[refactor](fs)(step3)use filesystem instead of old storage, new storage just access remote object storage (#19098)
see #18960 PR1: add new storage file system template and move old storage to new package PR2: extract some method in old storage to new file system. PR3: use storages to access remote object storage, and use file systems to access file in local or remote location. Will add some unit tests. this is PR3.
This commit is contained in:
@ -46,7 +46,8 @@ import org.apache.doris.common.ErrorReport;
|
||||
import org.apache.doris.common.Pair;
|
||||
import org.apache.doris.common.io.Writable;
|
||||
import org.apache.doris.common.util.MasterDaemon;
|
||||
import org.apache.doris.fs.obj.BlobStorage;
|
||||
import org.apache.doris.fs.FileSystemFactory;
|
||||
import org.apache.doris.fs.remote.RemoteFileSystem;
|
||||
import org.apache.doris.task.DirMoveTask;
|
||||
import org.apache.doris.task.DownloadTask;
|
||||
import org.apache.doris.task.SnapshotTask;
|
||||
@ -197,9 +198,10 @@ public class BackupHandler extends MasterDaemon implements Writable {
|
||||
"broker does not exist: " + stmt.getBrokerName());
|
||||
}
|
||||
|
||||
BlobStorage storage = BlobStorage.create(stmt.getBrokerName(), stmt.getStorageType(), stmt.getProperties());
|
||||
RemoteFileSystem fileSystem = FileSystemFactory.get(stmt.getBrokerName(), stmt.getStorageType(),
|
||||
stmt.getProperties());
|
||||
long repoId = env.getNextId();
|
||||
Repository repo = new Repository(repoId, stmt.getName(), stmt.isReadOnly(), stmt.getLocation(), storage);
|
||||
Repository repo = new Repository(repoId, stmt.getName(), stmt.isReadOnly(), stmt.getLocation(), fileSystem);
|
||||
|
||||
Status st = repoMgr.addAndInitRepoIfNotExist(repo, false);
|
||||
if (!st.ok()) {
|
||||
|
||||
@ -612,8 +612,8 @@ public class BackupJob extends AbstractJob {
|
||||
}
|
||||
long signature = env.getNextId();
|
||||
UploadTask task = new UploadTask(null, beId, signature, jobId, dbId, srcToDest,
|
||||
brokers.get(0), repo.getStorage().getProperties(), repo.getStorage().getStorageType(),
|
||||
repo.getLocation());
|
||||
brokers.get(0), repo.getRemoteFileSystem().getProperties(),
|
||||
repo.getRemoteFileSystem().getStorageType(), repo.getLocation());
|
||||
LOG.info("yy debug upload location: " + repo.getLocation());
|
||||
batchTask.addTask(task);
|
||||
unfinishedTaskIds.put(signature, beId);
|
||||
|
||||
@ -28,10 +28,12 @@ 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.common.util.TimeUtils;
|
||||
import org.apache.doris.fs.obj.BlobStorage;
|
||||
import org.apache.doris.fs.obj.BrokerStorage;
|
||||
import org.apache.doris.fs.obj.HdfsStorage;
|
||||
import org.apache.doris.fs.obj.S3Storage;
|
||||
import org.apache.doris.fs.PersistentFileSystem;
|
||||
import org.apache.doris.fs.remote.BrokerFileSystem;
|
||||
import org.apache.doris.fs.remote.RemoteFile;
|
||||
import org.apache.doris.fs.remote.RemoteFileSystem;
|
||||
import org.apache.doris.fs.remote.S3FileSystem;
|
||||
import org.apache.doris.fs.remote.dfs.DFSFileSystem;
|
||||
import org.apache.doris.system.Backend;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
@ -112,18 +114,18 @@ public class Repository implements Writable {
|
||||
// and the specified bucket should exist.
|
||||
private String location;
|
||||
|
||||
private BlobStorage storage;
|
||||
private RemoteFileSystem fileSystem;
|
||||
|
||||
private Repository() {
|
||||
// for persist
|
||||
}
|
||||
|
||||
public Repository(long id, String name, boolean isReadOnly, String location, BlobStorage storage) {
|
||||
public Repository(long id, String name, boolean isReadOnly, String location, RemoteFileSystem fileSystem) {
|
||||
this.id = id;
|
||||
this.name = name;
|
||||
this.isReadOnly = isReadOnly;
|
||||
this.location = location;
|
||||
this.storage = storage;
|
||||
this.fileSystem = fileSystem;
|
||||
this.createTime = System.currentTimeMillis();
|
||||
}
|
||||
|
||||
@ -199,8 +201,8 @@ public class Repository implements Writable {
|
||||
return errMsg;
|
||||
}
|
||||
|
||||
public BlobStorage getStorage() {
|
||||
return storage;
|
||||
public RemoteFileSystem getRemoteFileSystem() {
|
||||
return fileSystem;
|
||||
}
|
||||
|
||||
public long getCreateTime() {
|
||||
@ -215,7 +217,7 @@ public class Repository implements Writable {
|
||||
String repoInfoFilePath = assembleRepoInfoFilePath();
|
||||
// check if the repo is already exist in remote
|
||||
List<RemoteFile> remoteFiles = Lists.newArrayList();
|
||||
Status st = storage.list(repoInfoFilePath, remoteFiles);
|
||||
Status st = fileSystem.list(repoInfoFilePath, remoteFiles);
|
||||
if (!st.ok()) {
|
||||
return st;
|
||||
}
|
||||
@ -228,7 +230,7 @@ public class Repository implements Writable {
|
||||
// exist, download and parse the repo info file
|
||||
String localFilePath = BackupHandler.BACKUP_ROOT_DIR + "/tmp_info_" + System.currentTimeMillis();
|
||||
try {
|
||||
st = storage.downloadWithFileSize(repoInfoFilePath, localFilePath, remoteFile.getSize());
|
||||
st = fileSystem.downloadWithFileSize(repoInfoFilePath, localFilePath, remoteFile.getSize());
|
||||
if (!st.ok()) {
|
||||
return st;
|
||||
}
|
||||
@ -260,7 +262,7 @@ public class Repository implements Writable {
|
||||
root.put("name", name);
|
||||
root.put("create_time", TimeUtils.longToTimeString(createTime));
|
||||
String repoInfoContent = root.toString();
|
||||
return storage.directUpload(repoInfoContent, repoInfoFilePath);
|
||||
return fileSystem.directUpload(repoInfoContent, repoInfoFilePath);
|
||||
}
|
||||
}
|
||||
|
||||
@ -328,7 +330,7 @@ public class Repository implements Writable {
|
||||
String path = location + "/" + joinPrefix(PREFIX_REPO, name) + "/" + FILE_REPO_INFO;
|
||||
try {
|
||||
URI checkUri = new URI(path);
|
||||
Status st = storage.checkPathExist(checkUri.normalize().toString());
|
||||
Status st = fileSystem.exists(checkUri.normalize().toString());
|
||||
if (!st.ok()) {
|
||||
errMsg = TimeUtils.longToTimeString(System.currentTimeMillis()) + ": " + st.getErrMsg();
|
||||
return false;
|
||||
@ -351,7 +353,7 @@ public class Repository implements Writable {
|
||||
String listPath = Joiner.on(PATH_DELIMITER).join(location, joinPrefix(PREFIX_REPO, name), PREFIX_SNAPSHOT_DIR)
|
||||
+ "*";
|
||||
List<RemoteFile> result = Lists.newArrayList();
|
||||
Status st = storage.list(listPath, result);
|
||||
Status st = fileSystem.list(listPath, result);
|
||||
if (!st.ok()) {
|
||||
return st;
|
||||
}
|
||||
@ -459,53 +461,53 @@ public class Repository implements Writable {
|
||||
String finalRemotePath = assembleFileNameWithSuffix(remoteFilePath, md5sum);
|
||||
|
||||
Status st = Status.OK;
|
||||
if (storage instanceof BrokerStorage) {
|
||||
if (fileSystem instanceof BrokerFileSystem) {
|
||||
// this may be a retry, so we should first delete remote file
|
||||
String tmpRemotePath = assembleFileNameWithSuffix(remoteFilePath, SUFFIX_TMP_FILE);
|
||||
LOG.debug("get md5sum of file: {}. tmp remote path: {}. final remote path: {}",
|
||||
localFilePath, tmpRemotePath, finalRemotePath);
|
||||
st = storage.delete(tmpRemotePath);
|
||||
st = fileSystem.delete(tmpRemotePath);
|
||||
if (!st.ok()) {
|
||||
return st;
|
||||
}
|
||||
|
||||
st = storage.delete(finalRemotePath);
|
||||
st = fileSystem.delete(finalRemotePath);
|
||||
if (!st.ok()) {
|
||||
return st;
|
||||
}
|
||||
|
||||
// upload tmp file
|
||||
st = storage.upload(localFilePath, tmpRemotePath);
|
||||
st = fileSystem.upload(localFilePath, tmpRemotePath);
|
||||
if (!st.ok()) {
|
||||
return st;
|
||||
}
|
||||
|
||||
// rename tmp file with checksum named file
|
||||
st = storage.rename(tmpRemotePath, finalRemotePath);
|
||||
st = fileSystem.rename(tmpRemotePath, finalRemotePath);
|
||||
if (!st.ok()) {
|
||||
return st;
|
||||
}
|
||||
} else if (storage instanceof S3Storage) {
|
||||
} else if (fileSystem instanceof S3FileSystem) {
|
||||
LOG.debug("get md5sum of file: {}. final remote path: {}", localFilePath, finalRemotePath);
|
||||
st = storage.delete(finalRemotePath);
|
||||
st = fileSystem.delete(finalRemotePath);
|
||||
if (!st.ok()) {
|
||||
return st;
|
||||
}
|
||||
|
||||
// upload final file
|
||||
st = storage.upload(localFilePath, finalRemotePath);
|
||||
st = fileSystem.upload(localFilePath, finalRemotePath);
|
||||
if (!st.ok()) {
|
||||
return st;
|
||||
}
|
||||
} else if (storage instanceof HdfsStorage) {
|
||||
} else if (fileSystem instanceof DFSFileSystem) {
|
||||
LOG.debug("hdfs get md5sum of file: {}. final remote path: {}", localFilePath, finalRemotePath);
|
||||
st = storage.delete(finalRemotePath);
|
||||
st = fileSystem.delete(finalRemotePath);
|
||||
if (!st.ok()) {
|
||||
return st;
|
||||
}
|
||||
|
||||
// upload final file
|
||||
st = storage.upload(localFilePath, finalRemotePath);
|
||||
st = fileSystem.upload(localFilePath, finalRemotePath);
|
||||
if (!st.ok()) {
|
||||
return st;
|
||||
}
|
||||
@ -519,7 +521,7 @@ public class Repository implements Writable {
|
||||
public Status download(String remoteFilePath, String localFilePath) {
|
||||
// 0. list to get to full name(with checksum)
|
||||
List<RemoteFile> remoteFiles = Lists.newArrayList();
|
||||
Status status = storage.list(remoteFilePath + "*", remoteFiles);
|
||||
Status status = fileSystem.list(remoteFilePath + "*", remoteFiles);
|
||||
if (!status.ok()) {
|
||||
return status;
|
||||
}
|
||||
@ -547,7 +549,8 @@ public class Repository implements Writable {
|
||||
String md5sum = pair.second;
|
||||
|
||||
// 2. download
|
||||
status = storage.downloadWithFileSize(remoteFilePathWithChecksum, localFilePath, remoteFiles.get(0).getSize());
|
||||
status = fileSystem.downloadWithFileSize(remoteFilePathWithChecksum, localFilePath,
|
||||
remoteFiles.get(0).getSize());
|
||||
if (!status.ok()) {
|
||||
return status;
|
||||
}
|
||||
@ -578,7 +581,7 @@ public class Repository implements Writable {
|
||||
+ "failed to send upload snapshot task");
|
||||
}
|
||||
// only Broker storage backend need to get broker addr, other type return a fake one;
|
||||
if (storage.getStorageType() != StorageBackend.StorageType.BROKER) {
|
||||
if (fileSystem.getStorageType() != StorageBackend.StorageType.BROKER) {
|
||||
brokerAddrs.add(new FsBroker("127.0.0.1", 0));
|
||||
return Status.OK;
|
||||
}
|
||||
@ -586,15 +589,15 @@ public class Repository implements Writable {
|
||||
// get proper broker for this backend
|
||||
FsBroker brokerAddr = null;
|
||||
try {
|
||||
brokerAddr = env.getBrokerMgr().getBroker(((BrokerStorage) storage).getBrokerName(), be.getIp());
|
||||
brokerAddr = env.getBrokerMgr().getBroker(fileSystem.getName(), be.getIp());
|
||||
} catch (AnalysisException e) {
|
||||
return new Status(ErrCode.COMMON_ERROR, "failed to get address of broker "
|
||||
+ ((BrokerStorage) storage).getBrokerName() + " when try to send upload snapshot task: "
|
||||
+ fileSystem.getName() + " when try to send upload snapshot task: "
|
||||
+ e.getMessage());
|
||||
}
|
||||
if (brokerAddr == null) {
|
||||
return new Status(ErrCode.COMMON_ERROR, "failed to get address of broker "
|
||||
+ ((BrokerStorage) storage).getBrokerName() + " when try to send upload snapshot task");
|
||||
+ fileSystem.getName() + " when try to send upload snapshot task");
|
||||
}
|
||||
brokerAddrs.add(brokerAddr);
|
||||
return Status.OK;
|
||||
@ -607,8 +610,8 @@ public class Repository implements Writable {
|
||||
info.add(TimeUtils.longToTimeString(createTime));
|
||||
info.add(String.valueOf(isReadOnly));
|
||||
info.add(location);
|
||||
info.add(storage.getType() != StorageBackend.StorageType.BROKER ? "-" : storage.getName());
|
||||
info.add(storage.getStorageType().name());
|
||||
info.add(fileSystem.getStorageType() != StorageBackend.StorageType.BROKER ? "-" : fileSystem.getName());
|
||||
info.add(fileSystem.getStorageType().name());
|
||||
info.add(errMsg == null ? FeConstants.null_string : errMsg);
|
||||
return info;
|
||||
}
|
||||
@ -647,14 +650,14 @@ public class Repository implements Writable {
|
||||
stmtBuilder.append("REPOSITORY ");
|
||||
stmtBuilder.append(this.name);
|
||||
stmtBuilder.append(" \nWITH ");
|
||||
StorageBackend.StorageType storageType = this.storage.getStorageType();
|
||||
StorageBackend.StorageType storageType = this.fileSystem.getStorageType();
|
||||
if (storageType == StorageBackend.StorageType.S3) {
|
||||
stmtBuilder.append(" S3 ");
|
||||
} else if (storageType == StorageBackend.StorageType.HDFS) {
|
||||
stmtBuilder.append(" HDFS ");
|
||||
} else if (storageType == StorageBackend.StorageType.BROKER) {
|
||||
stmtBuilder.append(" BROKER ");
|
||||
stmtBuilder.append(this.storage.getName());
|
||||
stmtBuilder.append(this.fileSystem.getName());
|
||||
} else {
|
||||
// should never reach here
|
||||
throw new UnsupportedOperationException(storageType.toString() + " backend is not implemented");
|
||||
@ -664,7 +667,7 @@ public class Repository implements Writable {
|
||||
stmtBuilder.append("\"");
|
||||
|
||||
stmtBuilder.append("\nPROPERTIES\n(");
|
||||
stmtBuilder.append(new PrintableMap<>(this.getStorage().getProperties(), " = ",
|
||||
stmtBuilder.append(new PrintableMap<>(this.getRemoteFileSystem().getProperties(), " = ",
|
||||
true, true));
|
||||
stmtBuilder.append("\n)");
|
||||
return stmtBuilder.toString();
|
||||
@ -678,7 +681,7 @@ public class Repository implements Writable {
|
||||
String infoFilePath = assembleJobInfoFilePath(snapshotName, -1);
|
||||
LOG.debug("assemble infoFilePath: {}, snapshot: {}", infoFilePath, snapshotName);
|
||||
List<RemoteFile> results = Lists.newArrayList();
|
||||
Status st = storage.list(infoFilePath + "*", results);
|
||||
Status st = fileSystem.list(infoFilePath + "*", results);
|
||||
if (!st.ok()) {
|
||||
info.add(snapshotName);
|
||||
info.add(FeConstants.null_string);
|
||||
@ -748,7 +751,7 @@ public class Repository implements Writable {
|
||||
Text.writeString(out, name);
|
||||
out.writeBoolean(isReadOnly);
|
||||
Text.writeString(out, location);
|
||||
storage.write(out);
|
||||
fileSystem.write(out);
|
||||
out.writeLong(createTime);
|
||||
}
|
||||
|
||||
@ -757,7 +760,7 @@ public class Repository implements Writable {
|
||||
name = Text.readString(in);
|
||||
isReadOnly = in.readBoolean();
|
||||
location = Text.readString(in);
|
||||
storage = BlobStorage.read(in);
|
||||
fileSystem = PersistentFileSystem.read(in);
|
||||
createTime = in.readLong();
|
||||
}
|
||||
}
|
||||
|
||||
@ -1376,8 +1376,8 @@ public class RestoreJob extends AbstractJob {
|
||||
}
|
||||
long signature = env.getNextId();
|
||||
DownloadTask task = new DownloadTask(null, beId, signature, jobId, dbId, srcToDest,
|
||||
brokerAddrs.get(0), repo.getStorage().getProperties(),
|
||||
repo.getStorage().getStorageType(), repo.getLocation());
|
||||
brokerAddrs.get(0), repo.getRemoteFileSystem().getProperties(),
|
||||
repo.getRemoteFileSystem().getStorageType(), repo.getLocation());
|
||||
batchTask.addTask(task);
|
||||
unfinishedSignatureToId.put(signature, beId);
|
||||
}
|
||||
|
||||
@ -36,7 +36,10 @@ import org.apache.doris.common.Config;
|
||||
import org.apache.doris.common.DdlException;
|
||||
import org.apache.doris.common.UserException;
|
||||
import org.apache.doris.datasource.property.constants.HMSProperties;
|
||||
import org.apache.doris.fs.obj.BlobStorage;
|
||||
import org.apache.doris.fs.FileSystemFactory;
|
||||
import org.apache.doris.fs.RemoteFiles;
|
||||
import org.apache.doris.fs.remote.RemoteFile;
|
||||
import org.apache.doris.fs.remote.RemoteFileSystem;
|
||||
import org.apache.doris.thrift.TBrokerFileStatus;
|
||||
import org.apache.doris.thrift.TExprOpcode;
|
||||
|
||||
@ -46,8 +49,7 @@ import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Queues;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
|
||||
@ -73,7 +75,6 @@ import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import shade.doris.hive.org.apache.thrift.TException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
@ -177,8 +178,8 @@ public class HiveMetaStoreClientHelper {
|
||||
public static String getHiveDataFiles(HiveTable hiveTable, ExprNodeGenericFuncDesc hivePartitionPredicate,
|
||||
List<TBrokerFileStatus> fileStatuses, Table remoteHiveTbl, StorageBackend.StorageType type)
|
||||
throws DdlException {
|
||||
BlobStorage storage = BlobStorage.create("HiveMetaStore", type, hiveTable.getHiveProperties());
|
||||
List<RemoteIterator<LocatedFileStatus>> remoteIterators = new ArrayList<>();
|
||||
RemoteFileSystem fs = FileSystemFactory.get("HiveMetaStore", type, hiveTable.getHiveProperties());
|
||||
List<RemoteFiles> remoteLocationsList = new ArrayList<>();
|
||||
try {
|
||||
if (remoteHiveTbl.getPartitionKeys().size() > 0) {
|
||||
String metaStoreUris = hiveTable.getHiveProperties().get(HMSProperties.HIVE_METASTORE_URIS);
|
||||
@ -187,14 +188,14 @@ public class HiveMetaStoreClientHelper {
|
||||
hivePartitionPredicate);
|
||||
for (Partition p : hivePartitions) {
|
||||
String location = normalizeS3LikeSchema(p.getSd().getLocation());
|
||||
remoteIterators.add(storage.listLocatedStatus(location));
|
||||
remoteLocationsList.add(fs.listLocatedFiles(location));
|
||||
}
|
||||
} else {
|
||||
// hive non-partitioned table, get file iterator from table sd info
|
||||
String location = normalizeS3LikeSchema(remoteHiveTbl.getSd().getLocation());
|
||||
remoteIterators.add(storage.listLocatedStatus(location));
|
||||
remoteLocationsList.add(fs.listLocatedFiles(location));
|
||||
}
|
||||
return getAllFileStatus(fileStatuses, remoteIterators, storage);
|
||||
return getAllFileStatus(fileStatuses, remoteLocationsList, fs);
|
||||
} catch (UserException e) {
|
||||
throw new DdlException(e.getMessage(), e);
|
||||
}
|
||||
@ -212,46 +213,41 @@ public class HiveMetaStoreClientHelper {
|
||||
}
|
||||
|
||||
private static String getAllFileStatus(List<TBrokerFileStatus> fileStatuses,
|
||||
List<RemoteIterator<LocatedFileStatus>> remoteIterators, BlobStorage storage) throws UserException {
|
||||
boolean needFullPath = storage.getStorageType() == StorageBackend.StorageType.S3
|
||||
|| storage.getStorageType() == StorageBackend.StorageType.OFS
|
||||
|| storage.getStorageType() == StorageBackend.StorageType.JFS;
|
||||
List<RemoteFiles> remoteLocationsList, RemoteFileSystem fs)
|
||||
throws UserException {
|
||||
String hdfsUrl = "";
|
||||
Queue<RemoteIterator<LocatedFileStatus>> queue = Queues.newArrayDeque(remoteIterators);
|
||||
Queue<RemoteFiles> queue = Queues.newArrayDeque(remoteLocationsList);
|
||||
while (queue.peek() != null) {
|
||||
RemoteIterator<LocatedFileStatus> iterator = queue.poll();
|
||||
RemoteFiles locs = queue.poll();
|
||||
try {
|
||||
while (iterator.hasNext()) {
|
||||
LocatedFileStatus fileStatus = iterator.next();
|
||||
if (fileStatus.isDirectory()) {
|
||||
for (RemoteFile fileLocation : locs.locations()) {
|
||||
Path filePath = fileLocation.getPath();
|
||||
// hdfs://host:port/path/to/partition/file_name
|
||||
String fullUri = filePath.toString();
|
||||
if (fileLocation.isDirectory()) {
|
||||
// recursive visit the directory to get the file path.
|
||||
queue.add(storage.listLocatedStatus(fileStatus.getPath().toString()));
|
||||
queue.add(fs.listLocatedFiles(fullUri));
|
||||
continue;
|
||||
}
|
||||
TBrokerFileStatus brokerFileStatus = new TBrokerFileStatus();
|
||||
brokerFileStatus.setIsDir(fileStatus.isDirectory());
|
||||
brokerFileStatus.setIsDir(fileLocation.isDirectory());
|
||||
brokerFileStatus.setIsSplitable(true);
|
||||
brokerFileStatus.setSize(fileStatus.getLen());
|
||||
// path = "/path/to/partition/file_name"
|
||||
brokerFileStatus.setSize(fileLocation.getSize());
|
||||
// filePath.toUri().getPath() = "/path/to/partition/file_name"
|
||||
// eg: /home/work/dev/hive/apache-hive-2.3.7-bin/data/warehouse
|
||||
// + /dae.db/customer/state=CA/city=SanJose/000000_0
|
||||
String path = fileStatus.getPath().toUri().getPath();
|
||||
if (needFullPath) {
|
||||
// Backend need full s3 path (with s3://bucket at the beginning) to read the data on s3.
|
||||
// path = "s3://bucket/path/to/partition/file_name"
|
||||
// eg: s3://hive-s3-test/region/region.tbl
|
||||
path = fileStatus.getPath().toString();
|
||||
}
|
||||
// fullUri: Backend need full s3 path (with s3://bucket at the beginning) to read the data on s3.
|
||||
// path = "s3://bucket/path/to/partition/file_name"
|
||||
// eg: s3://hive-s3-test/region/region.tbl
|
||||
String path = fs.needFullPath() ? fullUri : filePath.toUri().getPath();
|
||||
brokerFileStatus.setPath(path);
|
||||
fileStatuses.add(brokerFileStatus);
|
||||
if (StringUtils.isEmpty(hdfsUrl)) {
|
||||
// hdfs://host:port/path/to/partition/file_name
|
||||
String fullUri = fileStatus.getPath().toString();
|
||||
// hdfs://host:port
|
||||
hdfsUrl = fullUri.replace(path, "");
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
} catch (UserException e) {
|
||||
LOG.warn("List HDFS file IOException: {}", e.getMessage());
|
||||
throw new DdlException("List HDFS file failed. Error: " + e.getMessage());
|
||||
}
|
||||
|
||||
@ -87,4 +87,14 @@ public class FeConstants {
|
||||
public static String FS_PREFIX_FILE = "file";
|
||||
public static final String INTERNAL_DB_NAME = "__internal_schema";
|
||||
public static String TEMP_MATERIZLIZE_DVIEW_PREFIX = "internal_tmp_materialized_view_";
|
||||
|
||||
public static boolean isObjStorage(String location) {
|
||||
return location.startsWith(FeConstants.FS_PREFIX_S3)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_S3A)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_S3N)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_BOS)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_COS)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_OSS)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_OBS);
|
||||
}
|
||||
}
|
||||
|
||||
@ -18,7 +18,6 @@
|
||||
package org.apache.doris.common.util;
|
||||
|
||||
import org.apache.doris.analysis.BrokerDesc;
|
||||
import org.apache.doris.backup.RemoteFile;
|
||||
import org.apache.doris.backup.Status;
|
||||
import org.apache.doris.catalog.Env;
|
||||
import org.apache.doris.catalog.FsBroker;
|
||||
@ -29,7 +28,9 @@ import org.apache.doris.common.FeConstants;
|
||||
import org.apache.doris.common.Pair;
|
||||
import org.apache.doris.common.UserException;
|
||||
import org.apache.doris.datasource.hive.HiveMetaStoreCache;
|
||||
import org.apache.doris.fs.obj.BlobStorage;
|
||||
import org.apache.doris.fs.FileSystemFactory;
|
||||
import org.apache.doris.fs.remote.RemoteFile;
|
||||
import org.apache.doris.fs.remote.RemoteFileSystem;
|
||||
import org.apache.doris.service.FrontendOptions;
|
||||
import org.apache.doris.thrift.TBrokerCheckPathExistRequest;
|
||||
import org.apache.doris.thrift.TBrokerCheckPathExistResponse;
|
||||
@ -85,9 +86,9 @@ public class BrokerUtil {
|
||||
throws UserException {
|
||||
List<RemoteFile> rfiles = new ArrayList<>();
|
||||
try {
|
||||
BlobStorage storage = BlobStorage.create(
|
||||
RemoteFileSystem fileSystem = FileSystemFactory.get(
|
||||
brokerDesc.getName(), brokerDesc.getStorageType(), brokerDesc.getProperties());
|
||||
Status st = storage.list(path, rfiles, false);
|
||||
Status st = fileSystem.list(path, rfiles, false);
|
||||
if (!st.ok()) {
|
||||
throw new UserException(brokerDesc.getName() + " list path failed. path=" + path
|
||||
+ ",msg=" + st.getErrMsg());
|
||||
|
||||
@ -40,7 +40,8 @@ public class S3URI {
|
||||
public static final String PATH_DELIM = "/";
|
||||
private static final String QUERY_DELIM = "\\?";
|
||||
private static final String FRAGMENT_DELIM = "#";
|
||||
private static final Set<String> VALID_SCHEMES = ImmutableSet.of("http", "https", "s3", "s3a", "s3n", "bos");
|
||||
private static final Set<String> VALID_SCHEMES = ImmutableSet.of("http", "https", "s3", "s3a", "s3n",
|
||||
"bos", "oss", "cos", "obs");
|
||||
|
||||
private String scheme;
|
||||
private final String location;
|
||||
|
||||
@ -29,6 +29,7 @@ import org.apache.doris.common.FeConstants;
|
||||
import org.apache.doris.datasource.CacheException;
|
||||
import org.apache.doris.datasource.HMSExternalCatalog;
|
||||
import org.apache.doris.external.hive.util.HiveUtil;
|
||||
import org.apache.doris.fs.remote.RemoteFile;
|
||||
import org.apache.doris.metric.GaugeMetric;
|
||||
import org.apache.doris.metric.Metric;
|
||||
import org.apache.doris.metric.MetricLabel;
|
||||
@ -54,7 +55,6 @@ import lombok.Data;
|
||||
import org.apache.commons.lang3.math.NumberUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.BlockLocation;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.hive.metastore.api.Partition;
|
||||
@ -284,7 +284,7 @@ public class HiveMetaStoreCache {
|
||||
InputFormat<?, ?> inputFormat = HiveUtil.getInputFormat(jobConf, key.inputFormat, false);
|
||||
// TODO: This is a temp config, will remove it after the HiveSplitter is stable.
|
||||
if (key.useSelfSplitter) {
|
||||
result = HiveSplitter.getFileCache(new Path(finalLocation), inputFormat,
|
||||
result = HiveSplitter.getFileCache(finalLocation, inputFormat,
|
||||
jobConf, key.getPartitionValues());
|
||||
} else {
|
||||
InputSplit[] splits;
|
||||
@ -320,7 +320,6 @@ public class HiveMetaStoreCache {
|
||||
private String convertToS3IfNecessary(String location) {
|
||||
LOG.debug("try convert location to s3 prefix: " + location);
|
||||
if (location.startsWith(FeConstants.FS_PREFIX_COS)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_BOS)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_BOS)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_OSS)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_S3A)
|
||||
@ -770,14 +769,14 @@ public class HiveMetaStoreCache {
|
||||
// partitionValues would be ["part1", "part2"]
|
||||
protected List<String> partitionValues;
|
||||
|
||||
public void addFile(LocatedFileStatus file) {
|
||||
public void addFile(RemoteFile file) {
|
||||
if (files == null) {
|
||||
files = Lists.newArrayList();
|
||||
}
|
||||
HiveFileStatus status = new HiveFileStatus();
|
||||
status.setBlockLocations(file.getBlockLocations());
|
||||
status.setPath(file.getPath());
|
||||
status.length = file.getLen();
|
||||
status.length = file.getSize();
|
||||
status.blockSize = file.getBlockSize();
|
||||
files.add(status);
|
||||
}
|
||||
|
||||
@ -62,19 +62,25 @@ public class PropertyConverter {
|
||||
* s3.endpoint -> AWS_ENDPOINT
|
||||
* s3.access_key -> AWS_ACCESS_KEY
|
||||
* These properties will be used for catalog/resource, and persisted to catalog/resource properties.
|
||||
* Some properties like AWS_XXX will be hidden, can find from HIDDEN_KEY in PrintableMap
|
||||
* @see org.apache.doris.common.util.PrintableMap
|
||||
*/
|
||||
public static Map<String, String> convertToMetaProperties(Map<String, String> props) {
|
||||
Map<String, String> metaProperties = new HashMap<>();
|
||||
CloudCredential credential = GlueProperties.getCredential(props);
|
||||
if (!credential.isWhole()) {
|
||||
credential = GlueProperties.getCompatibleCredential(props);
|
||||
}
|
||||
if (props.containsKey(GlueProperties.ENDPOINT)
|
||||
|| props.containsKey(AWSGlueConfig.AWS_GLUE_ENDPOINT)) {
|
||||
CloudCredential credential = GlueProperties.getCredential(props);
|
||||
if (!credential.isWhole()) {
|
||||
credential = GlueProperties.getCompatibleCredential(props);
|
||||
}
|
||||
metaProperties = convertToGlueProperties(props, credential);
|
||||
} else if (props.containsKey(DLFProperties.ENDPOINT)
|
||||
|| props.containsKey(DataLakeConfig.CATALOG_ENDPOINT)) {
|
||||
metaProperties = convertToDLFProperties(props, credential);
|
||||
metaProperties = convertToDLFProperties(props, DLFProperties.getCredential(props));
|
||||
} else if (props.containsKey(S3Properties.Env.ENDPOINT)) {
|
||||
// checkout env in the end
|
||||
// if meet AWS_XXX properties, convert to s3 properties
|
||||
return convertToS3EnvProperties(props, S3Properties.getEnvironmentCredentialWithEndpoint(props), true);
|
||||
}
|
||||
metaProperties.putAll(props);
|
||||
metaProperties.putAll(S3ClientBEProperties.getBeFSProperties(props));
|
||||
@ -97,7 +103,7 @@ public class PropertyConverter {
|
||||
} else if (props.containsKey(S3Properties.Env.ENDPOINT)) {
|
||||
// checkout env in the end
|
||||
// compatible with the s3,obs,oss,cos when they use aws client.
|
||||
return convertToS3EnvProperties(props, S3Properties.getEnvironmentCredentialWithEndpoint(props));
|
||||
return convertToS3EnvProperties(props, S3Properties.getEnvironmentCredentialWithEndpoint(props), false);
|
||||
}
|
||||
return props;
|
||||
}
|
||||
@ -124,7 +130,8 @@ public class PropertyConverter {
|
||||
}
|
||||
|
||||
private static Map<String, String> convertToS3EnvProperties(Map<String, String> properties,
|
||||
CloudCredentialWithEndpoint credential) {
|
||||
CloudCredentialWithEndpoint credential,
|
||||
boolean isMeta) {
|
||||
// Old properties to new properties
|
||||
properties.put(S3Properties.ENDPOINT, credential.getEndpoint());
|
||||
properties.put(S3Properties.REGION,
|
||||
@ -143,6 +150,9 @@ public class PropertyConverter {
|
||||
if (properties.containsKey(S3Properties.Env.CONNECTION_TIMEOUT_MS)) {
|
||||
properties.put(S3Properties.REQUEST_TIMEOUT_MS, properties.get(S3Properties.Env.CONNECTION_TIMEOUT_MS));
|
||||
}
|
||||
if (isMeta) {
|
||||
return properties;
|
||||
}
|
||||
return convertToS3Properties(properties, credential);
|
||||
}
|
||||
|
||||
|
||||
@ -194,7 +194,9 @@ public class S3Properties extends BaseProperties {
|
||||
|
||||
public static void convertToStdProperties(Map<String, String> properties) {
|
||||
properties.putIfAbsent(S3Properties.ENDPOINT, properties.get(S3Properties.Env.ENDPOINT));
|
||||
properties.putIfAbsent(S3Properties.REGION, properties.get(S3Properties.Env.REGION));
|
||||
if (properties.containsKey(S3Properties.Env.REGION)) {
|
||||
properties.putIfAbsent(S3Properties.REGION, properties.get(S3Properties.Env.REGION));
|
||||
}
|
||||
properties.putIfAbsent(S3Properties.ACCESS_KEY, properties.get(S3Properties.Env.ACCESS_KEY));
|
||||
properties.putIfAbsent(S3Properties.SECRET_KEY, properties.get(S3Properties.Env.SECRET_KEY));
|
||||
if (properties.containsKey(S3Properties.Env.TOKEN)) {
|
||||
|
||||
@ -23,6 +23,7 @@ import org.apache.doris.catalog.ScalarType;
|
||||
import org.apache.doris.catalog.Type;
|
||||
import org.apache.doris.common.AnalysisException;
|
||||
import org.apache.doris.common.UserException;
|
||||
import org.apache.doris.fs.FileSystemFactory;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@ -42,6 +43,7 @@ import org.apache.hadoop.util.ReflectionUtils;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.List;
|
||||
@ -182,7 +184,7 @@ public final class HiveUtil {
|
||||
}
|
||||
}
|
||||
|
||||
public static boolean isSplittable(InputFormat<?, ?> inputFormat, FileSystem fileSystem, Path path) {
|
||||
public static boolean isSplittable(InputFormat<?, ?> inputFormat, Path path, JobConf jobConf) {
|
||||
// ORC uses a custom InputFormat but is always splittable
|
||||
if (inputFormat.getClass().getSimpleName().equals("OrcInputFormat")) {
|
||||
return true;
|
||||
@ -204,8 +206,8 @@ public final class HiveUtil {
|
||||
}
|
||||
try {
|
||||
method.setAccessible(true);
|
||||
return (boolean) method.invoke(inputFormat, fileSystem, path);
|
||||
} catch (InvocationTargetException | IllegalAccessException e) {
|
||||
return (boolean) method.invoke(inputFormat, FileSystemFactory.getNativeByPath(path, jobConf), path);
|
||||
} catch (InvocationTargetException | IllegalAccessException | IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@ -17,14 +17,12 @@
|
||||
|
||||
package org.apache.doris.fs;
|
||||
|
||||
import org.apache.doris.backup.RemoteFile;
|
||||
import org.apache.doris.backup.Status;
|
||||
import org.apache.doris.common.UserException;
|
||||
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.doris.fs.remote.RemoteFile;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* File system interface.
|
||||
@ -36,6 +34,8 @@ import java.util.List;
|
||||
* @see org.apache.doris.fs.operations.FileOperations
|
||||
*/
|
||||
public interface FileSystem {
|
||||
Map<String, String> getProperties();
|
||||
|
||||
Status exists(String remotePath);
|
||||
|
||||
Status downloadWithFileSize(String remoteFilePath, String localFilePath, long fileSize);
|
||||
@ -50,16 +50,20 @@ public interface FileSystem {
|
||||
|
||||
Status makeDir(String remotePath);
|
||||
|
||||
default RemoteIterator<LocatedFileStatus> listLocatedStatus(String remotePath) throws UserException {
|
||||
throw new UserException("Not support to listLocatedStatus.");
|
||||
default RemoteFiles listLocatedFiles(String remotePath) throws UserException {
|
||||
return listLocatedFiles(remotePath, false, false);
|
||||
}
|
||||
|
||||
// Get files and directories located status, not only files
|
||||
default RemoteFiles listLocatedFiles(String remotePath, boolean onlyFiles, boolean recursive) throws UserException {
|
||||
throw new UserException("Not support to listLocations.");
|
||||
}
|
||||
|
||||
// List files in remotePath
|
||||
// The remote file name will only contains file name only(Not full path)
|
||||
// The remote file name will only contain file name only(Not full path)
|
||||
default Status list(String remotePath, List<RemoteFile> result) {
|
||||
return list(remotePath, result, true);
|
||||
}
|
||||
|
||||
Status list(String remotePath, List<RemoteFile> result, boolean fileNameOnly);
|
||||
|
||||
}
|
||||
|
||||
@ -18,22 +18,24 @@
|
||||
package org.apache.doris.fs;
|
||||
|
||||
import org.apache.doris.analysis.StorageBackend;
|
||||
import org.apache.doris.common.FeConstants;
|
||||
import org.apache.doris.fs.remote.BrokerFileSystem;
|
||||
import org.apache.doris.fs.remote.RemoteFileSystem;
|
||||
import org.apache.doris.fs.remote.S3FileSystem;
|
||||
import org.apache.doris.fs.remote.dfs.DFSFileSystem;
|
||||
import org.apache.doris.fs.remote.dfs.JFSFileSystem;
|
||||
import org.apache.doris.fs.remote.dfs.OFSFileSystem;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class FileSystemFactory {
|
||||
|
||||
public static FileSystem get(StorageBackend.StorageType type, Map<String, String> properties) {
|
||||
// use for test
|
||||
return get(type.name(), type, properties);
|
||||
}
|
||||
|
||||
public static FileSystem get(String name, StorageBackend.StorageType type, Map<String, String> properties) {
|
||||
public static RemoteFileSystem get(String name, StorageBackend.StorageType type, Map<String, String> properties) {
|
||||
// TODO: rename StorageBackend.StorageType
|
||||
if (type == StorageBackend.StorageType.S3) {
|
||||
return new S3FileSystem(properties);
|
||||
@ -49,4 +51,29 @@ public class FileSystemFactory {
|
||||
throw new UnsupportedOperationException(type.toString() + "backend is not implemented");
|
||||
}
|
||||
}
|
||||
|
||||
public static RemoteFileSystem getByLocation(String location, Configuration conf) {
|
||||
// TODO: need optimize the method. the conf is converted many times.
|
||||
Map<String, String> properties = new HashMap<>();
|
||||
conf.iterator().forEachRemaining(e -> properties.put(e.getKey(), e.getValue()));
|
||||
if (location.startsWith(FeConstants.FS_PREFIX_S3)) {
|
||||
return new S3FileSystem(properties);
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_HDFS) || location.startsWith(FeConstants.FS_PREFIX_GFS)) {
|
||||
return new DFSFileSystem(properties);
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_OFS)) {
|
||||
return new OFSFileSystem(properties);
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_JFS)) {
|
||||
return new JFSFileSystem(properties);
|
||||
}
|
||||
throw new UnsupportedOperationException("Can not create file system for: " + location);
|
||||
}
|
||||
|
||||
public static RemoteFileSystem getS3FileSystem(Map<String, String> properties) {
|
||||
// use for test
|
||||
return get(StorageBackend.StorageType.S3.name(), StorageBackend.StorageType.S3, properties);
|
||||
}
|
||||
|
||||
public static org.apache.hadoop.fs.FileSystem getNativeByPath(Path path, Configuration conf) throws IOException {
|
||||
return path.getFileSystem(conf);
|
||||
}
|
||||
}
|
||||
|
||||
@ -17,49 +17,55 @@
|
||||
|
||||
package org.apache.doris.fs;
|
||||
|
||||
import org.apache.doris.backup.RemoteFile;
|
||||
import org.apache.doris.backup.Status;
|
||||
import org.apache.doris.fs.remote.RemoteFile;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class LocalFileSystem implements FileSystem {
|
||||
@Override
|
||||
public Status exists(String remotePath) {
|
||||
return null;
|
||||
throw new UnsupportedOperationException("Unsupported operation on local file system.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status downloadWithFileSize(String remoteFilePath, String localFilePath, long fileSize) {
|
||||
return null;
|
||||
throw new UnsupportedOperationException("Unsupported operation on local file system.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status upload(String localPath, String remotePath) {
|
||||
return null;
|
||||
throw new UnsupportedOperationException("Unsupported operation on local file system.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status directUpload(String content, String remoteFile) {
|
||||
return null;
|
||||
throw new UnsupportedOperationException("Unsupported operation on local file system.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status rename(String origFilePath, String destFilePath) {
|
||||
return null;
|
||||
throw new UnsupportedOperationException("Unsupported operation on local file system.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status delete(String remotePath) {
|
||||
return null;
|
||||
throw new UnsupportedOperationException("Unsupported operation on local file system.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status makeDir(String remotePath) {
|
||||
return null;
|
||||
throw new UnsupportedOperationException("Unsupported operation on local file system.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status list(String remotePath, List<RemoteFile> result, boolean fileNameOnly) {
|
||||
return null;
|
||||
throw new UnsupportedOperationException("Unsupported operation on local file system.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, String> getProperties() {
|
||||
throw new UnsupportedOperationException("Unsupported operation on local file system.");
|
||||
}
|
||||
}
|
||||
|
||||
@ -20,6 +20,7 @@ package org.apache.doris.fs;
|
||||
import org.apache.doris.analysis.StorageBackend;
|
||||
import org.apache.doris.common.io.Text;
|
||||
import org.apache.doris.common.io.Writable;
|
||||
import org.apache.doris.fs.remote.RemoteFileSystem;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
|
||||
@ -37,11 +38,21 @@ public abstract class PersistentFileSystem implements FileSystem, Writable {
|
||||
protected String name;
|
||||
protected StorageBackend.StorageType type;
|
||||
|
||||
public boolean needFullPath() {
|
||||
return type == StorageBackend.StorageType.S3
|
||||
|| type == StorageBackend.StorageType.OFS
|
||||
|| type == StorageBackend.StorageType.JFS;
|
||||
}
|
||||
|
||||
public PersistentFileSystem(String name, StorageBackend.StorageType type) {
|
||||
this.name = name;
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
public Map<String, String> getProperties() {
|
||||
return properties;
|
||||
}
|
||||
@ -55,7 +66,7 @@ public abstract class PersistentFileSystem implements FileSystem, Writable {
|
||||
* @param in persisted data
|
||||
* @return file systerm
|
||||
*/
|
||||
public static FileSystem read(DataInput in) throws IOException {
|
||||
public static RemoteFileSystem read(DataInput in) throws IOException {
|
||||
String name = Text.readString(in);
|
||||
Map<String, String> properties = Maps.newHashMap();
|
||||
StorageBackend.StorageType type = StorageBackend.StorageType.BROKER;
|
||||
|
||||
@ -0,0 +1,35 @@
|
||||
// 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.fs;
|
||||
|
||||
import org.apache.doris.fs.remote.RemoteFile;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class RemoteFiles {
|
||||
|
||||
private final List<RemoteFile> files;
|
||||
|
||||
public RemoteFiles(List<RemoteFile> files) {
|
||||
this.files = files;
|
||||
}
|
||||
|
||||
public List<RemoteFile> locations() {
|
||||
return files;
|
||||
}
|
||||
}
|
||||
@ -1,162 +0,0 @@
|
||||
// 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.fs.obj;
|
||||
|
||||
import org.apache.doris.analysis.StorageBackend;
|
||||
import org.apache.doris.backup.RemoteFile;
|
||||
import org.apache.doris.backup.Status;
|
||||
import org.apache.doris.common.Config;
|
||||
import org.apache.doris.common.UserException;
|
||||
import org.apache.doris.common.io.Text;
|
||||
import org.apache.doris.common.io.Writable;
|
||||
import org.apache.doris.service.FrontendOptions;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* @see org.apache.doris.fs.PersistentFileSystem
|
||||
* @see org.apache.doris.fs.FileSystemFactory
|
||||
*/
|
||||
@Deprecated
|
||||
public abstract class BlobStorage implements Writable {
|
||||
|
||||
public static final String STORAGE_TYPE = "_DORIS_STORAGE_TYPE_";
|
||||
private Map<String, String> properties = Maps.newHashMap();
|
||||
private String name;
|
||||
private StorageBackend.StorageType type;
|
||||
|
||||
public static String clientId() {
|
||||
return FrontendOptions.getLocalHostAddress() + ":" + Config.edit_log_port;
|
||||
}
|
||||
|
||||
public static BlobStorage create(String name, StorageBackend.StorageType type, Map<String, String> properties) {
|
||||
if (type == StorageBackend.StorageType.S3) {
|
||||
return new S3Storage(properties);
|
||||
} else if (type == StorageBackend.StorageType.HDFS
|
||||
|| type == StorageBackend.StorageType.OFS
|
||||
|| type == StorageBackend.StorageType.GFS
|
||||
|| type == StorageBackend.StorageType.JFS) {
|
||||
BlobStorage storage = new HdfsStorage(properties);
|
||||
// as of ofs files, use hdfs storage, but it's type should be ofs
|
||||
if (type == StorageBackend.StorageType.OFS || type == StorageBackend.StorageType.JFS) {
|
||||
storage.setType(type);
|
||||
storage.setName(type.name());
|
||||
}
|
||||
|
||||
return storage;
|
||||
} else if (type == StorageBackend.StorageType.BROKER) {
|
||||
return new BrokerStorage(name, properties);
|
||||
} else {
|
||||
throw new UnsupportedOperationException(type.toString() + "backend is not implemented");
|
||||
}
|
||||
}
|
||||
|
||||
public static BlobStorage read(DataInput in) throws IOException {
|
||||
String name = Text.readString(in);
|
||||
Map<String, String> properties = Maps.newHashMap();
|
||||
StorageBackend.StorageType type = StorageBackend.StorageType.BROKER;
|
||||
int size = in.readInt();
|
||||
for (int i = 0; i < size; i++) {
|
||||
String key = Text.readString(in);
|
||||
String value = Text.readString(in);
|
||||
properties.put(key, value);
|
||||
}
|
||||
if (properties.containsKey(STORAGE_TYPE)) {
|
||||
type = StorageBackend.StorageType.valueOf(properties.get(STORAGE_TYPE));
|
||||
properties.remove(STORAGE_TYPE);
|
||||
}
|
||||
return BlobStorage.create(name, type, properties);
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
public void setName(String name) {
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
public StorageBackend.StorageType getType() {
|
||||
return type;
|
||||
}
|
||||
|
||||
public void setType(StorageBackend.StorageType type) {
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
public Map<String, String> getProperties() {
|
||||
return properties;
|
||||
}
|
||||
|
||||
public void setProperties(Map<String, String> properties) {
|
||||
this.properties = properties;
|
||||
}
|
||||
|
||||
public FileSystem getFileSystem(String remotePath) throws UserException {
|
||||
throw new UserException("Not support to getFileSystem.");
|
||||
}
|
||||
|
||||
public abstract Status downloadWithFileSize(String remoteFilePath, String localFilePath, long fileSize);
|
||||
|
||||
// directly upload the content to remote file
|
||||
public abstract Status directUpload(String content, String remoteFile);
|
||||
|
||||
public abstract Status upload(String localPath, String remotePath);
|
||||
|
||||
public abstract Status rename(String origFilePath, String destFilePath);
|
||||
|
||||
public abstract Status delete(String remotePath);
|
||||
|
||||
// only for hdfs and s3
|
||||
public RemoteIterator<LocatedFileStatus> listLocatedStatus(String remotePath) throws UserException {
|
||||
throw new UserException("Not support to listLocatedStatus.");
|
||||
}
|
||||
|
||||
// List files in remotePath
|
||||
// The remote file name will only contains file name only(Not full path)
|
||||
public abstract Status list(String remotePath, List<RemoteFile> result);
|
||||
|
||||
public abstract Status list(String remotePath, List<RemoteFile> result, boolean fileNameOnly);
|
||||
|
||||
public abstract Status makeDir(String remotePath);
|
||||
|
||||
public abstract Status checkPathExist(String remotePath);
|
||||
|
||||
public abstract StorageBackend.StorageType getStorageType();
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
// must write type first
|
||||
Text.writeString(out, name);
|
||||
properties.put(STORAGE_TYPE, type.name());
|
||||
out.writeInt(getProperties().size());
|
||||
for (Map.Entry<String, String> entry : getProperties().entrySet()) {
|
||||
Text.writeString(out, entry.getKey());
|
||||
Text.writeString(out, entry.getValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1,710 +0,0 @@
|
||||
// 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.fs.obj;
|
||||
|
||||
import org.apache.doris.analysis.StorageBackend;
|
||||
import org.apache.doris.backup.RemoteFile;
|
||||
import org.apache.doris.backup.Status;
|
||||
import org.apache.doris.catalog.Env;
|
||||
import org.apache.doris.catalog.FsBroker;
|
||||
import org.apache.doris.common.AnalysisException;
|
||||
import org.apache.doris.common.ClientPool;
|
||||
import org.apache.doris.common.Pair;
|
||||
import org.apache.doris.common.util.BrokerUtil;
|
||||
import org.apache.doris.service.FrontendOptions;
|
||||
import org.apache.doris.thrift.TBrokerCheckPathExistRequest;
|
||||
import org.apache.doris.thrift.TBrokerCheckPathExistResponse;
|
||||
import org.apache.doris.thrift.TBrokerCloseReaderRequest;
|
||||
import org.apache.doris.thrift.TBrokerCloseWriterRequest;
|
||||
import org.apache.doris.thrift.TBrokerDeletePathRequest;
|
||||
import org.apache.doris.thrift.TBrokerFD;
|
||||
import org.apache.doris.thrift.TBrokerFileStatus;
|
||||
import org.apache.doris.thrift.TBrokerListPathRequest;
|
||||
import org.apache.doris.thrift.TBrokerListResponse;
|
||||
import org.apache.doris.thrift.TBrokerOpenMode;
|
||||
import org.apache.doris.thrift.TBrokerOpenReaderRequest;
|
||||
import org.apache.doris.thrift.TBrokerOpenReaderResponse;
|
||||
import org.apache.doris.thrift.TBrokerOpenWriterRequest;
|
||||
import org.apache.doris.thrift.TBrokerOpenWriterResponse;
|
||||
import org.apache.doris.thrift.TBrokerOperationStatus;
|
||||
import org.apache.doris.thrift.TBrokerOperationStatusCode;
|
||||
import org.apache.doris.thrift.TBrokerPReadRequest;
|
||||
import org.apache.doris.thrift.TBrokerPWriteRequest;
|
||||
import org.apache.doris.thrift.TBrokerReadResponse;
|
||||
import org.apache.doris.thrift.TBrokerRenamePathRequest;
|
||||
import org.apache.doris.thrift.TBrokerVersion;
|
||||
import org.apache.doris.thrift.TNetworkAddress;
|
||||
import org.apache.doris.thrift.TPaloBrokerService;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.thrift.TException;
|
||||
import org.apache.thrift.transport.TTransportException;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.BufferedOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.FileVisitOption;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* @see org.apache.doris.fs.remote.BrokerFileSystem
|
||||
*/
|
||||
@Deprecated
|
||||
public class BrokerStorage extends BlobStorage {
|
||||
private static final Logger LOG = LogManager.getLogger(BrokerStorage.class);
|
||||
|
||||
public BrokerStorage(String brokerName, Map<String, String> properties) {
|
||||
setName(brokerName);
|
||||
setProperties(properties);
|
||||
setType(StorageBackend.StorageType.BROKER);
|
||||
}
|
||||
|
||||
public String getBrokerName() {
|
||||
return getName();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status downloadWithFileSize(String remoteFilePath, String localFilePath, long fileSize) {
|
||||
LOG.debug("download from {} to {}, file size: {}.", remoteFilePath, localFilePath, fileSize);
|
||||
|
||||
long start = System.currentTimeMillis();
|
||||
|
||||
// 1. get a proper broker
|
||||
Pair<TPaloBrokerService.Client, TNetworkAddress> pair = getBroker();
|
||||
if (pair == null) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "failed to get broker client");
|
||||
}
|
||||
TPaloBrokerService.Client client = pair.first;
|
||||
TNetworkAddress address = pair.second;
|
||||
|
||||
// 2. open file reader with broker
|
||||
TBrokerFD fd;
|
||||
try {
|
||||
TBrokerOpenReaderRequest req = new TBrokerOpenReaderRequest(TBrokerVersion.VERSION_ONE, remoteFilePath,
|
||||
0, clientId(), getProperties());
|
||||
TBrokerOpenReaderResponse rep = client.openReader(req);
|
||||
TBrokerOperationStatus opst = rep.getOpStatus();
|
||||
if (opst.getStatusCode() != TBrokerOperationStatusCode.OK) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to open reader on broker " + BrokerUtil.printBroker(getName(), address)
|
||||
+ " for file: " + remoteFilePath + ". msg: " + opst.getMessage());
|
||||
}
|
||||
|
||||
fd = rep.getFd();
|
||||
LOG.info("finished to open reader. fd: {}. download {} to {}.",
|
||||
fd, remoteFilePath, localFilePath);
|
||||
} catch (TException e) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to open reader on broker " + BrokerUtil.printBroker(getName(), address)
|
||||
+ " for file: " + remoteFilePath + ". msg: " + e.getMessage());
|
||||
}
|
||||
Preconditions.checkNotNull(fd);
|
||||
|
||||
// 3. delete local file if exist
|
||||
File localFile = new File(localFilePath);
|
||||
if (localFile.exists()) {
|
||||
try {
|
||||
Files.walk(Paths.get(localFilePath), FileVisitOption.FOLLOW_LINKS)
|
||||
.sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete);
|
||||
} catch (IOException e) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "failed to delete exist local file: " + localFilePath);
|
||||
}
|
||||
}
|
||||
|
||||
// 4. create local file
|
||||
Status status = Status.OK;
|
||||
try {
|
||||
if (!localFile.createNewFile()) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "failed to create local file: " + localFilePath);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "failed to create local file: "
|
||||
+ localFilePath + ", msg: " + e.getMessage());
|
||||
}
|
||||
|
||||
// 5. read remote file with broker and write to local
|
||||
String lastErrMsg = null;
|
||||
try (BufferedOutputStream out = new BufferedOutputStream(new FileOutputStream(localFile))) {
|
||||
final long bufSize = 1024 * 1024; // 1MB
|
||||
long leftSize = fileSize;
|
||||
long readOffset = 0;
|
||||
while (leftSize > 0) {
|
||||
long readLen = Math.min(leftSize, bufSize);
|
||||
TBrokerReadResponse rep = null;
|
||||
// We only retry if we encounter a timeout thrift exception.
|
||||
int tryTimes = 0;
|
||||
while (tryTimes < 3) {
|
||||
try {
|
||||
TBrokerPReadRequest req = new TBrokerPReadRequest(TBrokerVersion.VERSION_ONE,
|
||||
fd, readOffset, readLen);
|
||||
rep = client.pread(req);
|
||||
if (rep.getOpStatus().getStatusCode() != TBrokerOperationStatusCode.OK) {
|
||||
// pread return failure.
|
||||
lastErrMsg = String.format("failed to read via broker %s. "
|
||||
+ "current read offset: %d, read length: %d,"
|
||||
+ " file size: %d, file: %s, err code: %d, msg: %s",
|
||||
BrokerUtil.printBroker(getName(), address),
|
||||
readOffset, readLen, fileSize,
|
||||
remoteFilePath, rep.getOpStatus().getStatusCode().getValue(),
|
||||
rep.getOpStatus().getMessage());
|
||||
LOG.warn(lastErrMsg);
|
||||
status = new Status(Status.ErrCode.COMMON_ERROR, lastErrMsg);
|
||||
}
|
||||
if (rep.opStatus.statusCode != TBrokerOperationStatusCode.END_OF_FILE) {
|
||||
LOG.debug("download. readLen: {}, read data len: {}, left size:{}. total size: {}",
|
||||
readLen, rep.getData().length, leftSize, fileSize);
|
||||
} else {
|
||||
LOG.debug("read eof: " + remoteFilePath);
|
||||
}
|
||||
break;
|
||||
} catch (TTransportException e) {
|
||||
if (e.getType() == TTransportException.TIMED_OUT) {
|
||||
// we only retry when we encounter timeout exception.
|
||||
lastErrMsg = String.format("failed to read via broker %s. "
|
||||
+ "current read offset: %d, read length: %d,"
|
||||
+ " file size: %d, file: %s, timeout.",
|
||||
BrokerUtil.printBroker(getName(), address),
|
||||
readOffset, readLen, fileSize,
|
||||
remoteFilePath);
|
||||
tryTimes++;
|
||||
continue;
|
||||
}
|
||||
|
||||
lastErrMsg = String.format("failed to read via broker %s. "
|
||||
+ "current read offset: %d, read length: %d,"
|
||||
+ " file size: %d, file: %s. msg: %s",
|
||||
BrokerUtil.printBroker(getName(), address),
|
||||
readOffset, readLen, fileSize,
|
||||
remoteFilePath, e.getMessage());
|
||||
LOG.warn(lastErrMsg);
|
||||
status = new Status(Status.ErrCode.COMMON_ERROR, lastErrMsg);
|
||||
break;
|
||||
} catch (TException e) {
|
||||
lastErrMsg = String.format("failed to read via broker %s. "
|
||||
+ "current read offset: %d, read length: %d,"
|
||||
+ " file size: %d, file: %s. msg: %s",
|
||||
BrokerUtil.printBroker(getName(), address),
|
||||
readOffset, readLen, fileSize,
|
||||
remoteFilePath, e.getMessage());
|
||||
LOG.warn(lastErrMsg);
|
||||
status = new Status(Status.ErrCode.COMMON_ERROR, lastErrMsg);
|
||||
break;
|
||||
}
|
||||
} // end of retry loop
|
||||
|
||||
if (status.ok() && tryTimes < 3) {
|
||||
// read succeed, write to local file
|
||||
Preconditions.checkNotNull(rep);
|
||||
// NOTICE(cmy): Sometimes the actual read length does not equal to the expected read length,
|
||||
// even if the broker's read buffer size is large enough.
|
||||
// I don't know why, but have to adapt to it.
|
||||
if (rep.getData().length != readLen) {
|
||||
LOG.warn("the actual read length does not equal to "
|
||||
+ "the expected read length: {} vs. {}, file: {}, broker: {}",
|
||||
rep.getData().length, readLen, remoteFilePath,
|
||||
BrokerUtil.printBroker(getName(), address));
|
||||
}
|
||||
|
||||
out.write(rep.getData());
|
||||
readOffset += rep.getData().length;
|
||||
leftSize -= rep.getData().length;
|
||||
} else {
|
||||
status = new Status(Status.ErrCode.COMMON_ERROR, lastErrMsg);
|
||||
break;
|
||||
}
|
||||
} // end of reading remote file
|
||||
} catch (IOException e) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "Got exception: " + e.getMessage() + ", broker: "
|
||||
+ BrokerUtil.printBroker(getName(), address));
|
||||
} finally {
|
||||
// close broker reader
|
||||
Status closeStatus = closeReader(client, address, fd);
|
||||
if (!closeStatus.ok()) {
|
||||
LOG.warn(closeStatus.getErrMsg());
|
||||
if (status.ok()) {
|
||||
// we return close write error only if no other error has been encountered.
|
||||
status = closeStatus;
|
||||
}
|
||||
ClientPool.brokerPool.invalidateObject(address, client);
|
||||
} else {
|
||||
ClientPool.brokerPool.returnObject(address, client);
|
||||
}
|
||||
}
|
||||
|
||||
LOG.info("finished to download from {} to {} with size: {}. cost {} ms",
|
||||
remoteFilePath, localFilePath, fileSize, (System.currentTimeMillis() - start));
|
||||
return status;
|
||||
}
|
||||
|
||||
// directly upload the content to remote file
|
||||
@Override
|
||||
public Status directUpload(String content, String remoteFile) {
|
||||
// 1. get a proper broker
|
||||
Pair<TPaloBrokerService.Client, TNetworkAddress> pair = getBroker();
|
||||
if (pair == null) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "failed to get broker client");
|
||||
}
|
||||
TPaloBrokerService.Client client = pair.first;
|
||||
TNetworkAddress address = pair.second;
|
||||
|
||||
TBrokerFD fd = new TBrokerFD();
|
||||
Status status = Status.OK;
|
||||
try {
|
||||
// 2. open file writer with broker
|
||||
status = openWriter(client, address, remoteFile, fd);
|
||||
if (!status.ok()) {
|
||||
return status;
|
||||
}
|
||||
|
||||
// 3. write content
|
||||
try {
|
||||
ByteBuffer bb = ByteBuffer.wrap(content.getBytes(StandardCharsets.UTF_8));
|
||||
TBrokerPWriteRequest req = new TBrokerPWriteRequest(TBrokerVersion.VERSION_ONE, fd, 0, bb);
|
||||
TBrokerOperationStatus opst = client.pwrite(req);
|
||||
if (opst.getStatusCode() != TBrokerOperationStatusCode.OK) {
|
||||
// pwrite return failure.
|
||||
status = new Status(Status.ErrCode.COMMON_ERROR, "write failed: " + opst.getMessage()
|
||||
+ ", broker: " + BrokerUtil.printBroker(getName(), address));
|
||||
}
|
||||
} catch (TException e) {
|
||||
status = new Status(Status.ErrCode.BAD_CONNECTION, "write exception: " + e.getMessage()
|
||||
+ ", broker: " + BrokerUtil.printBroker(getName(), address));
|
||||
}
|
||||
} finally {
|
||||
Status closeStatus = closeWriter(client, address, fd);
|
||||
if (closeStatus.getErrCode() == Status.ErrCode.BAD_CONNECTION
|
||||
|| status.getErrCode() == Status.ErrCode.BAD_CONNECTION) {
|
||||
ClientPool.brokerPool.invalidateObject(address, client);
|
||||
} else {
|
||||
ClientPool.brokerPool.returnObject(address, client);
|
||||
}
|
||||
}
|
||||
|
||||
return status;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status upload(String localPath, String remotePath) {
|
||||
long start = System.currentTimeMillis();
|
||||
// 1. get a proper broker
|
||||
Pair<TPaloBrokerService.Client, TNetworkAddress> pair = getBroker();
|
||||
if (pair == null) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "failed to get broker client");
|
||||
}
|
||||
TPaloBrokerService.Client client = pair.first;
|
||||
TNetworkAddress address = pair.second;
|
||||
|
||||
// 2. open file write with broker
|
||||
TBrokerFD fd = new TBrokerFD();
|
||||
Status status = openWriter(client, address, remotePath, fd);
|
||||
if (!status.ok()) {
|
||||
return status;
|
||||
}
|
||||
|
||||
// 3. read local file and write to remote with broker
|
||||
File localFile = new File(localPath);
|
||||
long fileLength = localFile.length();
|
||||
byte[] readBuf = new byte[1024];
|
||||
try (BufferedInputStream in = new BufferedInputStream(new FileInputStream(localFile))) {
|
||||
// save the last err msg
|
||||
String lastErrMsg = null;
|
||||
// save the current write offset of remote file
|
||||
long writeOffset = 0;
|
||||
// read local file, 1MB at a time
|
||||
int bytesRead;
|
||||
while ((bytesRead = in.read(readBuf)) != -1) {
|
||||
ByteBuffer bb = ByteBuffer.wrap(readBuf, 0, bytesRead);
|
||||
|
||||
// We only retry if we encounter a timeout thrift exception.
|
||||
int tryTimes = 0;
|
||||
while (tryTimes < 3) {
|
||||
try {
|
||||
TBrokerPWriteRequest req
|
||||
= new TBrokerPWriteRequest(TBrokerVersion.VERSION_ONE, fd, writeOffset, bb);
|
||||
TBrokerOperationStatus opst = client.pwrite(req);
|
||||
if (opst.getStatusCode() != TBrokerOperationStatusCode.OK) {
|
||||
// pwrite return failure.
|
||||
lastErrMsg = String.format("failed to write via broker %s. "
|
||||
+ "current write offset: %d, write length: %d,"
|
||||
+ " file length: %d, file: %s, err code: %d, msg: %s",
|
||||
BrokerUtil.printBroker(getName(), address),
|
||||
writeOffset, bytesRead, fileLength,
|
||||
remotePath, opst.getStatusCode().getValue(), opst.getMessage());
|
||||
LOG.warn(lastErrMsg);
|
||||
status = new Status(Status.ErrCode.COMMON_ERROR, lastErrMsg);
|
||||
}
|
||||
break;
|
||||
} catch (TTransportException e) {
|
||||
if (e.getType() == TTransportException.TIMED_OUT) {
|
||||
// we only retry when we encounter timeout exception.
|
||||
lastErrMsg = String.format("failed to write via broker %s. "
|
||||
+ "current write offset: %d, write length: %d,"
|
||||
+ " file length: %d, file: %s. timeout",
|
||||
BrokerUtil.printBroker(getName(), address),
|
||||
writeOffset, bytesRead, fileLength,
|
||||
remotePath);
|
||||
tryTimes++;
|
||||
continue;
|
||||
}
|
||||
|
||||
lastErrMsg = String.format("failed to write via broker %s. "
|
||||
+ "current write offset: %d, write length: %d,"
|
||||
+ " file length: %d, file: %s. encounter TTransportException: %s",
|
||||
BrokerUtil.printBroker(getName(), address),
|
||||
writeOffset, bytesRead, fileLength,
|
||||
remotePath, e.getMessage());
|
||||
LOG.warn(lastErrMsg, e);
|
||||
status = new Status(Status.ErrCode.COMMON_ERROR, lastErrMsg);
|
||||
break;
|
||||
} catch (TException e) {
|
||||
lastErrMsg = String.format("failed to write via broker %s. "
|
||||
+ "current write offset: %d, write length: %d,"
|
||||
+ " file length: %d, file: %s. encounter TException: %s",
|
||||
BrokerUtil.printBroker(getName(), address),
|
||||
writeOffset, bytesRead, fileLength,
|
||||
remotePath, e.getMessage());
|
||||
LOG.warn(lastErrMsg, e);
|
||||
status = new Status(Status.ErrCode.COMMON_ERROR, lastErrMsg);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (status.ok() && tryTimes < 3) {
|
||||
// write succeed, update current write offset
|
||||
writeOffset += bytesRead;
|
||||
} else {
|
||||
status = new Status(Status.ErrCode.COMMON_ERROR, lastErrMsg);
|
||||
break;
|
||||
}
|
||||
} // end of read local file loop
|
||||
} catch (FileNotFoundException e1) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "encounter file not found exception: " + e1.getMessage()
|
||||
+ ", broker: " + BrokerUtil.printBroker(getName(), address));
|
||||
} catch (IOException e1) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "encounter io exception: " + e1.getMessage()
|
||||
+ ", broker: " + BrokerUtil.printBroker(getName(), address));
|
||||
} finally {
|
||||
// close write
|
||||
Status closeStatus = closeWriter(client, address, fd);
|
||||
if (!closeStatus.ok()) {
|
||||
LOG.warn(closeStatus.getErrMsg());
|
||||
if (status.ok()) {
|
||||
// we return close write error only if no other error has been encountered.
|
||||
status = closeStatus;
|
||||
}
|
||||
ClientPool.brokerPool.invalidateObject(address, client);
|
||||
} else {
|
||||
ClientPool.brokerPool.returnObject(address, client);
|
||||
}
|
||||
}
|
||||
|
||||
if (status.ok()) {
|
||||
LOG.info("finished to upload {} to remote path {}. cost: {} ms",
|
||||
localPath, remotePath, (System.currentTimeMillis() - start));
|
||||
}
|
||||
return status;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status rename(String origFilePath, String destFilePath) {
|
||||
long start = System.currentTimeMillis();
|
||||
// 1. get a proper broker
|
||||
Pair<TPaloBrokerService.Client, TNetworkAddress> pair = getBroker();
|
||||
if (pair == null) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "failed to get broker client");
|
||||
}
|
||||
TPaloBrokerService.Client client = pair.first;
|
||||
TNetworkAddress address = pair.second;
|
||||
|
||||
// 2. rename
|
||||
boolean needReturn = true;
|
||||
try {
|
||||
TBrokerRenamePathRequest req = new TBrokerRenamePathRequest(TBrokerVersion.VERSION_ONE,
|
||||
origFilePath, destFilePath, getProperties());
|
||||
TBrokerOperationStatus ost = client.renamePath(req);
|
||||
if (ost.getStatusCode() != TBrokerOperationStatusCode.OK) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to rename " + origFilePath + " to " + destFilePath + ", msg: " + ost.getMessage()
|
||||
+ ", broker: " + BrokerUtil.printBroker(getName(), address));
|
||||
}
|
||||
} catch (TException e) {
|
||||
needReturn = false;
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to rename " + origFilePath + " to " + destFilePath + ", msg: " + e.getMessage()
|
||||
+ ", broker: " + BrokerUtil.printBroker(getName(), address));
|
||||
} finally {
|
||||
if (needReturn) {
|
||||
ClientPool.brokerPool.returnObject(address, client);
|
||||
} else {
|
||||
ClientPool.brokerPool.invalidateObject(address, client);
|
||||
}
|
||||
}
|
||||
|
||||
LOG.info("finished to rename {} to {}. cost: {} ms",
|
||||
origFilePath, destFilePath, (System.currentTimeMillis() - start));
|
||||
return Status.OK;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status delete(String remotePath) {
|
||||
// get a proper broker
|
||||
Pair<TPaloBrokerService.Client, TNetworkAddress> pair = getBroker();
|
||||
if (pair == null) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "failed to get broker client");
|
||||
}
|
||||
TPaloBrokerService.Client client = pair.first;
|
||||
TNetworkAddress address = pair.second;
|
||||
|
||||
// delete
|
||||
boolean needReturn = true;
|
||||
try {
|
||||
TBrokerDeletePathRequest req = new TBrokerDeletePathRequest(TBrokerVersion.VERSION_ONE,
|
||||
remotePath, getProperties());
|
||||
TBrokerOperationStatus opst = client.deletePath(req);
|
||||
if (opst.getStatusCode() != TBrokerOperationStatusCode.OK) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to delete remote path: " + remotePath + ". msg: " + opst.getMessage()
|
||||
+ ", broker: " + BrokerUtil.printBroker(getName(), address));
|
||||
}
|
||||
|
||||
LOG.info("finished to delete remote path {}.", remotePath);
|
||||
} catch (TException e) {
|
||||
needReturn = false;
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to delete remote path: " + remotePath + ". msg: " + e.getMessage()
|
||||
+ ", broker: " + BrokerUtil.printBroker(getName(), address));
|
||||
} finally {
|
||||
if (needReturn) {
|
||||
ClientPool.brokerPool.returnObject(address, client);
|
||||
} else {
|
||||
ClientPool.brokerPool.invalidateObject(address, client);
|
||||
}
|
||||
}
|
||||
|
||||
return Status.OK;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status list(String remotePath, List<RemoteFile> result) {
|
||||
return list(remotePath, result, true);
|
||||
}
|
||||
|
||||
// List files in remotePath
|
||||
@Override
|
||||
public Status list(String remotePath, List<RemoteFile> result, boolean fileNameOnly) {
|
||||
// get a proper broker
|
||||
Pair<TPaloBrokerService.Client, TNetworkAddress> pair = getBroker();
|
||||
if (pair == null) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "failed to get broker client");
|
||||
}
|
||||
TPaloBrokerService.Client client = pair.first;
|
||||
TNetworkAddress address = pair.second;
|
||||
|
||||
// list
|
||||
boolean needReturn = true;
|
||||
try {
|
||||
TBrokerListPathRequest req = new TBrokerListPathRequest(TBrokerVersion.VERSION_ONE, remotePath,
|
||||
false /* not recursive */, getProperties());
|
||||
req.setFileNameOnly(fileNameOnly);
|
||||
TBrokerListResponse rep = client.listPath(req);
|
||||
TBrokerOperationStatus opst = rep.getOpStatus();
|
||||
if (opst.getStatusCode() != TBrokerOperationStatusCode.OK) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to list remote path: " + remotePath + ". msg: " + opst.getMessage()
|
||||
+ ", broker: " + BrokerUtil.printBroker(getName(), address));
|
||||
}
|
||||
|
||||
List<TBrokerFileStatus> fileStatus = rep.getFiles();
|
||||
for (TBrokerFileStatus tFile : fileStatus) {
|
||||
RemoteFile file = new RemoteFile(tFile.path, !tFile.isDir, tFile.size, 0);
|
||||
result.add(file);
|
||||
}
|
||||
LOG.info("finished to list remote path {}. get files: {}", remotePath, result);
|
||||
} catch (TException e) {
|
||||
needReturn = false;
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to list remote path: " + remotePath + ". msg: " + e.getMessage()
|
||||
+ ", broker: " + BrokerUtil.printBroker(getName(), address));
|
||||
} finally {
|
||||
if (needReturn) {
|
||||
ClientPool.brokerPool.returnObject(address, client);
|
||||
} else {
|
||||
ClientPool.brokerPool.invalidateObject(address, client);
|
||||
}
|
||||
}
|
||||
|
||||
return Status.OK;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status makeDir(String remotePath) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "mkdir is not implemented.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status checkPathExist(String remotePath) {
|
||||
// 1. get a proper broker
|
||||
Pair<TPaloBrokerService.Client, TNetworkAddress> pair = getBroker();
|
||||
if (pair == null) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "failed to get broker client");
|
||||
}
|
||||
TPaloBrokerService.Client client = pair.first;
|
||||
TNetworkAddress address = pair.second;
|
||||
|
||||
// check path
|
||||
boolean needReturn = true;
|
||||
try {
|
||||
TBrokerCheckPathExistRequest req = new TBrokerCheckPathExistRequest(TBrokerVersion.VERSION_ONE,
|
||||
remotePath, getProperties());
|
||||
TBrokerCheckPathExistResponse rep = client.checkPathExist(req);
|
||||
TBrokerOperationStatus opst = rep.getOpStatus();
|
||||
if (opst.getStatusCode() != TBrokerOperationStatusCode.OK) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to check remote path exist: " + remotePath
|
||||
+ ", broker: " + BrokerUtil.printBroker(getName(), address)
|
||||
+ ". msg: " + opst.getMessage());
|
||||
}
|
||||
|
||||
if (!rep.isIsPathExist()) {
|
||||
return new Status(Status.ErrCode.NOT_FOUND, "remote path does not exist: " + remotePath);
|
||||
}
|
||||
|
||||
return Status.OK;
|
||||
} catch (TException e) {
|
||||
needReturn = false;
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to check remote path exist: " + remotePath
|
||||
+ ", broker: " + BrokerUtil.printBroker(getName(), address)
|
||||
+ ". msg: " + e.getMessage());
|
||||
} finally {
|
||||
if (needReturn) {
|
||||
ClientPool.brokerPool.returnObject(address, client);
|
||||
} else {
|
||||
ClientPool.brokerPool.invalidateObject(address, client);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public StorageBackend.StorageType getStorageType() {
|
||||
return StorageBackend.StorageType.BROKER;
|
||||
}
|
||||
|
||||
public Pair<TPaloBrokerService.Client, TNetworkAddress> getBroker() {
|
||||
Pair<TPaloBrokerService.Client, TNetworkAddress> result = Pair.of(null, null);
|
||||
FsBroker broker;
|
||||
try {
|
||||
String localIP = FrontendOptions.getLocalHostAddress();
|
||||
broker = Env.getCurrentEnv().getBrokerMgr().getBroker(getName(), localIP);
|
||||
} catch (AnalysisException e) {
|
||||
LOG.warn("failed to get a broker address: " + e.getMessage());
|
||||
return null;
|
||||
}
|
||||
TNetworkAddress address = new TNetworkAddress(broker.ip, broker.port);
|
||||
TPaloBrokerService.Client client;
|
||||
try {
|
||||
client = ClientPool.brokerPool.borrowObject(address);
|
||||
} catch (Exception e) {
|
||||
LOG.warn("failed to get broker client: " + e.getMessage());
|
||||
return null;
|
||||
}
|
||||
|
||||
result.first = client;
|
||||
result.second = address;
|
||||
LOG.info("get broker: {}", BrokerUtil.printBroker(getName(), address));
|
||||
return result;
|
||||
}
|
||||
|
||||
private Status openWriter(TPaloBrokerService.Client client, TNetworkAddress address, String remoteFile,
|
||||
TBrokerFD fd) {
|
||||
try {
|
||||
TBrokerOpenWriterRequest req = new TBrokerOpenWriterRequest(TBrokerVersion.VERSION_ONE,
|
||||
remoteFile, TBrokerOpenMode.APPEND, clientId(), getProperties());
|
||||
TBrokerOpenWriterResponse rep = client.openWriter(req);
|
||||
TBrokerOperationStatus opst = rep.getOpStatus();
|
||||
if (opst.getStatusCode() != TBrokerOperationStatusCode.OK) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to open writer on broker " + BrokerUtil.printBroker(getName(), address)
|
||||
+ " for file: " + remoteFile + ". msg: " + opst.getMessage());
|
||||
}
|
||||
|
||||
fd.setHigh(rep.getFd().getHigh());
|
||||
fd.setLow(rep.getFd().getLow());
|
||||
LOG.info("finished to open writer. fd: {}. directly upload to remote path {}.", fd, remoteFile);
|
||||
} catch (TException e) {
|
||||
return new Status(Status.ErrCode.BAD_CONNECTION,
|
||||
"failed to open writer on broker " + BrokerUtil.printBroker(getName(), address)
|
||||
+ ", err: " + e.getMessage());
|
||||
}
|
||||
|
||||
return Status.OK;
|
||||
}
|
||||
|
||||
private Status closeWriter(TPaloBrokerService.Client client, TNetworkAddress address, TBrokerFD fd) {
|
||||
try {
|
||||
TBrokerCloseWriterRequest req = new TBrokerCloseWriterRequest(TBrokerVersion.VERSION_ONE, fd);
|
||||
TBrokerOperationStatus st = client.closeWriter(req);
|
||||
if (st.getStatusCode() != TBrokerOperationStatusCode.OK) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to close writer on broker " + BrokerUtil.printBroker(getName(), address)
|
||||
+ " for fd: " + fd);
|
||||
}
|
||||
|
||||
LOG.info("finished to close writer. fd: {}.", fd);
|
||||
} catch (TException e) {
|
||||
return new Status(Status.ErrCode.BAD_CONNECTION,
|
||||
"failed to close writer on broker " + BrokerUtil.printBroker(getName(), address)
|
||||
+ ", fd " + fd + ", msg: " + e.getMessage());
|
||||
}
|
||||
|
||||
return Status.OK;
|
||||
}
|
||||
|
||||
private Status closeReader(TPaloBrokerService.Client client, TNetworkAddress address, TBrokerFD fd) {
|
||||
try {
|
||||
TBrokerCloseReaderRequest req = new TBrokerCloseReaderRequest(TBrokerVersion.VERSION_ONE, fd);
|
||||
TBrokerOperationStatus st = client.closeReader(req);
|
||||
if (st.getStatusCode() != TBrokerOperationStatusCode.OK) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to close reader on broker " + BrokerUtil.printBroker(getName(), address)
|
||||
+ " for fd: " + fd);
|
||||
}
|
||||
|
||||
LOG.info("finished to close reader. fd: {}.", fd);
|
||||
} catch (TException e) {
|
||||
return new Status(Status.ErrCode.BAD_CONNECTION,
|
||||
"failed to close reader on broker " + BrokerUtil.printBroker(getName(), address)
|
||||
+ ", fd " + fd + ", msg: " + e.getMessage());
|
||||
}
|
||||
|
||||
return Status.OK;
|
||||
}
|
||||
}
|
||||
@ -1,570 +0,0 @@
|
||||
// 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.fs.obj;
|
||||
|
||||
import org.apache.doris.analysis.StorageBackend;
|
||||
import org.apache.doris.backup.RemoteFile;
|
||||
import org.apache.doris.backup.Status;
|
||||
import org.apache.doris.catalog.AuthType;
|
||||
import org.apache.doris.catalog.HdfsResource;
|
||||
import org.apache.doris.common.UserException;
|
||||
import org.apache.doris.common.util.URI;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.BufferedOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.file.FileVisitOption;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* HdfsStorage encapsulate interfaces accessing HDFS directly.
|
||||
* @see org.apache.doris.fs.remote.dfs.DFSFileSystem
|
||||
*/
|
||||
@Deprecated
|
||||
public class HdfsStorage extends BlobStorage {
|
||||
private static final Logger LOG = LogManager.getLogger(HdfsStorage.class);
|
||||
private final Map<String, String> hdfsProperties;
|
||||
|
||||
private final int readBufferSize = 128 << 10; // 128k
|
||||
private final int writeBufferSize = 128 << 10; // 128k
|
||||
|
||||
private FileSystem dfsFileSystem = null;
|
||||
|
||||
/**
|
||||
* init HdfsStorage with properties.
|
||||
*
|
||||
* @param properties parameters to access HDFS.
|
||||
*/
|
||||
public HdfsStorage(Map<String, String> properties) {
|
||||
hdfsProperties = new HashMap<>();
|
||||
setProperties(properties);
|
||||
setType(StorageBackend.StorageType.HDFS);
|
||||
setName(StorageBackend.StorageType.HDFS.name());
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileSystem getFileSystem(String remotePath) throws UserException {
|
||||
if (dfsFileSystem != null) {
|
||||
return dfsFileSystem;
|
||||
}
|
||||
String username = hdfsProperties.get(HdfsResource.HADOOP_USER_NAME);
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
boolean isSecurityEnabled = false;
|
||||
for (Map.Entry<String, String> propEntry : hdfsProperties.entrySet()) {
|
||||
conf.set(propEntry.getKey(), propEntry.getValue());
|
||||
if (propEntry.getKey().equals(HdfsResource.HADOOP_SECURITY_AUTHENTICATION)
|
||||
&& propEntry.getValue().equals(AuthType.KERBEROS.getDesc())) {
|
||||
isSecurityEnabled = true;
|
||||
}
|
||||
}
|
||||
try {
|
||||
if (isSecurityEnabled) {
|
||||
UserGroupInformation.setConfiguration(conf);
|
||||
UserGroupInformation.loginUserFromKeytab(
|
||||
hdfsProperties.get(HdfsResource.HADOOP_KERBEROS_PRINCIPAL),
|
||||
hdfsProperties.get(HdfsResource.HADOOP_KERBEROS_KEYTAB));
|
||||
}
|
||||
if (username == null) {
|
||||
dfsFileSystem = FileSystem.get(java.net.URI.create(remotePath), conf);
|
||||
} else {
|
||||
dfsFileSystem = FileSystem.get(java.net.URI.create(remotePath), conf, username);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error("errors while connect to " + remotePath, e);
|
||||
throw new UserException("errors while connect to " + remotePath, e);
|
||||
}
|
||||
return dfsFileSystem;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setProperties(Map<String, String> properties) {
|
||||
super.setProperties(properties);
|
||||
hdfsProperties.putAll(properties);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status downloadWithFileSize(String remoteFilePath, String localFilePath, long fileSize) {
|
||||
LOG.debug("download from {} to {}, file size: {}.", remoteFilePath, localFilePath, fileSize);
|
||||
final long start = System.currentTimeMillis();
|
||||
FSDataInputStream fsDataInputStream = null;
|
||||
try {
|
||||
fsDataInputStream = openReader(remoteFilePath, 0);
|
||||
} catch (Exception e) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, e.getMessage());
|
||||
}
|
||||
LOG.info("finished to open reader. download {} to {}.", remoteFilePath, localFilePath);
|
||||
|
||||
// delete local file if exist
|
||||
File localFile = new File(localFilePath);
|
||||
if (localFile.exists()) {
|
||||
try {
|
||||
Files.walk(Paths.get(localFilePath), FileVisitOption.FOLLOW_LINKS).sorted(Comparator.reverseOrder())
|
||||
.map(java.nio.file.Path::toFile).forEach(File::delete);
|
||||
} catch (IOException e) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to delete exist local file: " + localFilePath + ", msg: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
// create local file
|
||||
try {
|
||||
if (!localFile.createNewFile()) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "failed to create local file: " + localFilePath);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to create local file: " + localFilePath + ", msg: " + e.getMessage());
|
||||
}
|
||||
|
||||
String lastErrMsg = null;
|
||||
Status status = Status.OK;
|
||||
try (BufferedOutputStream out = new BufferedOutputStream(new FileOutputStream(localFile))) {
|
||||
final long bufSize = 1024 * 1024; // 1MB
|
||||
long leftSize = fileSize;
|
||||
long readOffset = 0;
|
||||
while (leftSize > 0) {
|
||||
long readLen = Math.min(leftSize, bufSize);
|
||||
try {
|
||||
ByteBuffer data = pread(fsDataInputStream, readOffset, readLen);
|
||||
if (readLen != data.array().length) {
|
||||
LOG.warn(
|
||||
"the actual read length does not equal to "
|
||||
+ "the expected read length: {} vs. {}, file: {}",
|
||||
data.array().length, readLen, remoteFilePath);
|
||||
}
|
||||
// write local file
|
||||
out.write(data.array());
|
||||
readOffset += data.array().length;
|
||||
leftSize -= data.array().length;
|
||||
} catch (Exception e) {
|
||||
lastErrMsg = String.format(
|
||||
"failed to read. " + "current read offset: %d, read length: %d,"
|
||||
+ " file size: %d, file: %s. msg: %s",
|
||||
readOffset, readLen, fileSize, remoteFilePath, e.getMessage());
|
||||
LOG.warn(lastErrMsg);
|
||||
status = new Status(Status.ErrCode.COMMON_ERROR, lastErrMsg);
|
||||
break;
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "Got exception: " + e.getMessage());
|
||||
} finally {
|
||||
Status closeStatus = closeReader(fsDataInputStream);
|
||||
if (!closeStatus.ok()) {
|
||||
LOG.warn(closeStatus.getErrMsg());
|
||||
if (status.ok()) {
|
||||
// we return close write error only if no other error has been encountered.
|
||||
status = closeStatus;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
LOG.info("finished to download from {} to {} with size: {}. cost {} ms", remoteFilePath, localFilePath,
|
||||
fileSize, (System.currentTimeMillis() - start));
|
||||
return status;
|
||||
}
|
||||
|
||||
private int readBytesFully(FSDataInputStream is, byte[] dest) throws IOException {
|
||||
int readLength = 0;
|
||||
while (readLength < dest.length) {
|
||||
int availableReadLength = dest.length - readLength;
|
||||
int n = is.read(dest, readLength, availableReadLength);
|
||||
if (n <= 0) {
|
||||
break;
|
||||
}
|
||||
readLength += n;
|
||||
}
|
||||
return readLength;
|
||||
}
|
||||
|
||||
/**
|
||||
* read data from fsDataInputStream.
|
||||
*
|
||||
* @param fsDataInputStream input stream for read.
|
||||
* @param readOffset read offset.
|
||||
* @param length read length.
|
||||
* @return ByteBuffer
|
||||
* @throws IOException when read data error.
|
||||
*/
|
||||
public ByteBuffer pread(FSDataInputStream fsDataInputStream, long readOffset, long length) throws IOException {
|
||||
synchronized (fsDataInputStream) {
|
||||
long currentStreamOffset;
|
||||
try {
|
||||
currentStreamOffset = fsDataInputStream.getPos();
|
||||
} catch (IOException e) {
|
||||
LOG.error("errors while get file pos from output stream", e);
|
||||
throw new IOException("errors while get file pos from output stream", e);
|
||||
}
|
||||
if (currentStreamOffset != readOffset) {
|
||||
// it's ok, when reading some format like parquet, it is not a sequential read
|
||||
LOG.debug("invalid offset, current read offset is " + currentStreamOffset
|
||||
+ " is not equal to request offset " + readOffset + " seek to it");
|
||||
try {
|
||||
fsDataInputStream.seek(readOffset);
|
||||
} catch (IOException e) {
|
||||
throw new IOException(String.format(
|
||||
"current read offset %d is not equal to %d, and could not seek to it, msg: %s",
|
||||
currentStreamOffset, readOffset, e.getMessage()));
|
||||
}
|
||||
}
|
||||
// Avoid using the ByteBuffer based read for Hadoop because some
|
||||
// FSDataInputStream
|
||||
// implementations are not ByteBufferReadable,
|
||||
// See https://issues.apache.org/jira/browse/HADOOP-14603
|
||||
byte[] buf;
|
||||
if (length > readBufferSize) {
|
||||
buf = new byte[readBufferSize];
|
||||
} else {
|
||||
buf = new byte[(int) length];
|
||||
}
|
||||
try {
|
||||
int readLength = readBytesFully(fsDataInputStream, buf);
|
||||
if (readLength < 0) {
|
||||
throw new IOException("end of file reached");
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(
|
||||
"read buffer from input stream, buffer size:" + buf.length + ", read length:" + readLength);
|
||||
}
|
||||
return ByteBuffer.wrap(buf, 0, readLength);
|
||||
} catch (IOException e) {
|
||||
LOG.error("errors while read data from stream", e);
|
||||
throw new IOException("errors while read data from stream " + e.getMessage());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status directUpload(String content, String remoteFile) {
|
||||
FSDataOutputStream fsDataOutputStream = null;
|
||||
try {
|
||||
fsDataOutputStream = openWriter(remoteFile);
|
||||
} catch (Exception e) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, e.getMessage());
|
||||
}
|
||||
LOG.info("finished to open writer. directly upload to remote path {}.", remoteFile);
|
||||
|
||||
Status status = Status.OK;
|
||||
try {
|
||||
fsDataOutputStream.writeBytes(content);
|
||||
} catch (IOException e) {
|
||||
LOG.error("errors while write data to output stream", e);
|
||||
status = new Status(Status.ErrCode.COMMON_ERROR, "write exception: " + e.getMessage());
|
||||
} finally {
|
||||
Status closeStatus = closeWriter(fsDataOutputStream);
|
||||
if (!closeStatus.ok()) {
|
||||
LOG.warn(closeStatus.getErrMsg());
|
||||
if (status.ok()) {
|
||||
status = closeStatus;
|
||||
}
|
||||
}
|
||||
}
|
||||
return status;
|
||||
}
|
||||
|
||||
/**
|
||||
* open remotePath for write.
|
||||
*
|
||||
* @param remotePath hdfs://namenode:port/path.
|
||||
* @return FSDataOutputStream
|
||||
* @throws UserException when get filesystem failed.
|
||||
* @throws IOException when open path error.
|
||||
*/
|
||||
public FSDataOutputStream openWriter(String remotePath) throws UserException, IOException {
|
||||
URI pathUri = URI.create(remotePath);
|
||||
Path inputFilePath = new Path(pathUri.getPath());
|
||||
FileSystem fileSystem = getFileSystem(remotePath);
|
||||
try {
|
||||
return fileSystem.create(inputFilePath, true, writeBufferSize);
|
||||
} catch (IOException e) {
|
||||
LOG.error("errors while open path", e);
|
||||
throw new IOException(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* close for write.
|
||||
*
|
||||
* @param fsDataOutputStream output stream.
|
||||
* @return Status.OK if success.
|
||||
*/
|
||||
public Status closeWriter(FSDataOutputStream fsDataOutputStream) {
|
||||
synchronized (fsDataOutputStream) {
|
||||
try {
|
||||
fsDataOutputStream.flush();
|
||||
fsDataOutputStream.close();
|
||||
LOG.info("finished to close writer");
|
||||
} catch (IOException e) {
|
||||
LOG.error("errors while close file output stream", e);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "failed to close writer, msg:" + e.getMessage());
|
||||
}
|
||||
}
|
||||
return Status.OK;
|
||||
}
|
||||
|
||||
/**
|
||||
* open remotePath for read.
|
||||
*
|
||||
* @param remotePath hdfs://namenode:port/path.
|
||||
* @param startOffset the offset to read.
|
||||
* @return FSDataInputStream if success.
|
||||
* @throws UserException when get filesystem failed.
|
||||
* @throws IOException when open file error.
|
||||
*/
|
||||
public FSDataInputStream openReader(String remotePath, long startOffset) throws UserException, IOException {
|
||||
URI pathUri = URI.create(remotePath);
|
||||
Path inputFilePath = new Path(pathUri.getPath());
|
||||
FileSystem fileSystem = getFileSystem(remotePath);
|
||||
try {
|
||||
FSDataInputStream fsDataInputStream = fileSystem.open(inputFilePath, readBufferSize);
|
||||
fsDataInputStream.seek(startOffset);
|
||||
return fsDataInputStream;
|
||||
} catch (IOException e) {
|
||||
LOG.error("errors while open path", e);
|
||||
throw new IOException(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* close for read.
|
||||
*
|
||||
* @param fsDataInputStream the input stream.
|
||||
* @return Status.OK if success.
|
||||
*/
|
||||
public Status closeReader(FSDataInputStream fsDataInputStream) {
|
||||
synchronized (fsDataInputStream) {
|
||||
try {
|
||||
fsDataInputStream.close();
|
||||
} catch (IOException e) {
|
||||
LOG.error("errors while close file input stream", e);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"errors while close file input stream, msg: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
return Status.OK;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status upload(String localPath, String remotePath) {
|
||||
long start = System.currentTimeMillis();
|
||||
LOG.debug("local path {}, remote path {}", localPath, remotePath);
|
||||
FSDataOutputStream fsDataOutputStream = null;
|
||||
try {
|
||||
fsDataOutputStream = openWriter(remotePath);
|
||||
} catch (Exception e) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, e.getMessage());
|
||||
}
|
||||
|
||||
LOG.info("finished to open writer. directly upload to remote path {}.", remotePath);
|
||||
// read local file and write remote
|
||||
File localFile = new File(localPath);
|
||||
long fileLength = localFile.length();
|
||||
byte[] readBuf = new byte[1024];
|
||||
Status status = new Status(Status.ErrCode.OK, "");
|
||||
try (BufferedInputStream in = new BufferedInputStream(new FileInputStream(localFile))) {
|
||||
// save the last err msg
|
||||
String lastErrMsg = null;
|
||||
// save the current write offset of remote file
|
||||
long writeOffset = 0;
|
||||
// read local file, 1MB at a time
|
||||
int bytesRead;
|
||||
while ((bytesRead = in.read(readBuf)) != -1) {
|
||||
try {
|
||||
fsDataOutputStream.write(readBuf, 0, bytesRead);
|
||||
} catch (IOException e) {
|
||||
LOG.error("errors while write data to output stream", e);
|
||||
lastErrMsg = String.format(
|
||||
"failed to write hdfs. current write offset: %d, write length: %d, "
|
||||
+ "file length: %d, file: %s, msg: errors while write data to output stream",
|
||||
writeOffset, bytesRead, fileLength, remotePath);
|
||||
status = new Status(Status.ErrCode.COMMON_ERROR, lastErrMsg);
|
||||
break;
|
||||
}
|
||||
|
||||
// write succeed, update current write offset
|
||||
writeOffset += bytesRead;
|
||||
} // end of read local file loop
|
||||
} catch (FileNotFoundException e1) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "encounter file not found exception: " + e1.getMessage());
|
||||
} catch (IOException e1) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "encounter io exception: " + e1.getMessage());
|
||||
} finally {
|
||||
Status closeStatus = closeWriter(fsDataOutputStream);
|
||||
if (!closeStatus.ok()) {
|
||||
LOG.warn(closeStatus.getErrMsg());
|
||||
if (status.ok()) {
|
||||
// we return close write error only if no other error has been encountered.
|
||||
status = closeStatus;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (status.ok()) {
|
||||
LOG.info("finished to upload {} to remote path {}. cost: {} ms", localPath, remotePath,
|
||||
(System.currentTimeMillis() - start));
|
||||
}
|
||||
return status;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status rename(String srcPath, String destPath) {
|
||||
long start = System.currentTimeMillis();
|
||||
try {
|
||||
URI srcPathUri = URI.create(srcPath);
|
||||
URI destPathUri = URI.create(destPath);
|
||||
if (!srcPathUri.getAuthority().trim().equals(destPathUri.getAuthority().trim())) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "only allow rename in same file system");
|
||||
}
|
||||
FileSystem fileSystem = getFileSystem(destPath);
|
||||
Path srcfilePath = new Path(srcPathUri.getPath());
|
||||
Path destfilePath = new Path(destPathUri.getPath());
|
||||
boolean isRenameSuccess = fileSystem.rename(srcfilePath, destfilePath);
|
||||
if (!isRenameSuccess) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "failed to rename " + srcPath + " to " + destPath);
|
||||
}
|
||||
} catch (UserException e) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, e.getMessage());
|
||||
} catch (IOException e) {
|
||||
LOG.error("errors while rename path from " + srcPath + " to " + destPath);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to rename remote " + srcPath + " to " + destPath + ", msg: " + e.getMessage());
|
||||
}
|
||||
LOG.info("finished to rename {} to {}. cost: {} ms", srcPath, destPath, (System.currentTimeMillis() - start));
|
||||
return Status.OK;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status delete(String remotePath) {
|
||||
try {
|
||||
URI pathUri = URI.create(remotePath);
|
||||
Path inputFilePath = new Path(pathUri.getPath());
|
||||
FileSystem fileSystem = getFileSystem(remotePath);
|
||||
fileSystem.delete(inputFilePath, true);
|
||||
} catch (UserException e) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, e.getMessage());
|
||||
} catch (IOException e) {
|
||||
LOG.error("errors while delete path " + remotePath);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to delete remote path: " + remotePath + ", msg: " + e.getMessage());
|
||||
}
|
||||
LOG.info("finished to delete remote path {}.", remotePath);
|
||||
return Status.OK;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<LocatedFileStatus> listLocatedStatus(String remotePath) throws UserException {
|
||||
FileSystem fileSystem = getFileSystem(remotePath);
|
||||
try {
|
||||
return fileSystem.listLocatedStatus(new Path(remotePath));
|
||||
} catch (IOException e) {
|
||||
throw new UserException("Failed to list located status for path: " + remotePath, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status list(String remotePath, List<RemoteFile> result) {
|
||||
return list(remotePath, result, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* get files in remotePath of HDFS.
|
||||
*
|
||||
* @param remotePath hdfs://namenode:port/path.
|
||||
* @param result files in remotePath.
|
||||
* @param fileNameOnly means get file only in remotePath if true.
|
||||
* @return Status.OK if success.
|
||||
*/
|
||||
@Override
|
||||
public Status list(String remotePath, List<RemoteFile> result, boolean fileNameOnly) {
|
||||
try {
|
||||
URI pathUri = URI.create(remotePath);
|
||||
FileSystem fileSystem = getFileSystem(remotePath);
|
||||
Path pathPattern = new Path(pathUri.getPath());
|
||||
FileStatus[] files = fileSystem.globStatus(pathPattern);
|
||||
if (files == null) {
|
||||
LOG.info("no files in path " + remotePath);
|
||||
return Status.OK;
|
||||
}
|
||||
for (FileStatus fileStatus : files) {
|
||||
RemoteFile remoteFile = new RemoteFile(
|
||||
fileNameOnly ? fileStatus.getPath().getName() : fileStatus.getPath().toString(),
|
||||
!fileStatus.isDirectory(), fileStatus.isDirectory() ? -1 : fileStatus.getLen(),
|
||||
fileStatus.getBlockSize());
|
||||
result.add(remoteFile);
|
||||
}
|
||||
} catch (FileNotFoundException e) {
|
||||
LOG.info("file not found: " + e.getMessage());
|
||||
return new Status(Status.ErrCode.NOT_FOUND, "file not found: " + e.getMessage());
|
||||
} catch (Exception e) {
|
||||
LOG.error("errors while get file status ", e);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "errors while get file status " + e.getMessage());
|
||||
}
|
||||
LOG.info("finish list path {}", remotePath);
|
||||
return Status.OK;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status makeDir(String remotePath) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "mkdir is not implemented.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status checkPathExist(String remotePath) {
|
||||
try {
|
||||
URI pathUri = URI.create(remotePath);
|
||||
Path inputFilePath = new Path(pathUri.getPath());
|
||||
FileSystem fileSystem = getFileSystem(remotePath);
|
||||
boolean isPathExist = fileSystem.exists(inputFilePath);
|
||||
if (!isPathExist) {
|
||||
return new Status(Status.ErrCode.NOT_FOUND, "remote path does not exist: " + remotePath);
|
||||
}
|
||||
return Status.OK;
|
||||
} catch (Exception e) {
|
||||
LOG.error("errors while check path exist " + remotePath, e);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR,
|
||||
"failed to check remote path exist: " + remotePath + ". msg: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public StorageBackend.StorageType getStorageType() {
|
||||
return this.getType();
|
||||
}
|
||||
}
|
||||
@ -44,6 +44,7 @@ import software.amazon.awssdk.regions.Region;
|
||||
import software.amazon.awssdk.services.s3.S3Client;
|
||||
import software.amazon.awssdk.services.s3.S3Configuration;
|
||||
import software.amazon.awssdk.services.s3.model.CopyObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.CopyObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.DeleteObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.DeleteObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
|
||||
@ -86,7 +87,11 @@ public class S3ObjStorage implements ObjStorage<S3Client> {
|
||||
setProperties(properties);
|
||||
}
|
||||
|
||||
private void setProperties(Map<String, String> properties) {
|
||||
public Map<String, String> getProperties() {
|
||||
return properties;
|
||||
}
|
||||
|
||||
protected void setProperties(Map<String, String> properties) {
|
||||
this.properties.putAll(properties);
|
||||
try {
|
||||
S3Properties.requiredS3Properties(this.properties);
|
||||
@ -111,9 +116,12 @@ public class S3ObjStorage implements ObjStorage<S3Client> {
|
||||
// That is, for S3 endpoint, ignore the `use_path_style` property, and the s3 client will automatically use
|
||||
// virtual hosted-sytle.
|
||||
// And for other endpoint, if `use_path_style` is true, use path style. Otherwise, use virtual hosted-sytle.
|
||||
// 'forceHostedStyle==false' means that use path style.
|
||||
if (!this.properties.get(S3Properties.ENDPOINT).toLowerCase().contains(S3Properties.S3_PREFIX)) {
|
||||
forceHostedStyle = !this.properties.getOrDefault(PropertyConverter.USE_PATH_STYLE, "false")
|
||||
.equalsIgnoreCase("true");
|
||||
String usePathStyle = this.properties.getOrDefault(PropertyConverter.USE_PATH_STYLE, "false");
|
||||
boolean isUsePathStyle = usePathStyle.equalsIgnoreCase("true");
|
||||
// when it's path style, we will not use virtual hosted-style
|
||||
forceHostedStyle = !isUsePathStyle;
|
||||
} else {
|
||||
forceHostedStyle = false;
|
||||
}
|
||||
@ -227,7 +235,7 @@ public class S3ObjStorage implements ObjStorage<S3Client> {
|
||||
.putObject(
|
||||
PutObjectRequest.builder().bucket(uri.getBucket()).key(uri.getKey()).build(),
|
||||
requestBody);
|
||||
LOG.info("put object success: " + response.eTag());
|
||||
LOG.info("put object success: " + response.toString());
|
||||
return Status.OK;
|
||||
} catch (S3Exception e) {
|
||||
LOG.error("put object failed:", e);
|
||||
@ -264,13 +272,14 @@ public class S3ObjStorage implements ObjStorage<S3Client> {
|
||||
try {
|
||||
S3URI origUri = S3URI.create(origFilePath);
|
||||
S3URI descUri = S3URI.create(destFilePath, forceHostedStyle);
|
||||
getClient(descUri.getVirtualBucket())
|
||||
CopyObjectResponse response = getClient(descUri.getVirtualBucket())
|
||||
.copyObject(
|
||||
CopyObjectRequest.builder()
|
||||
.copySource(origUri.getBucket() + "/" + origUri.getKey())
|
||||
.destinationBucket(descUri.getBucket())
|
||||
.destinationKey(descUri.getKey())
|
||||
.build());
|
||||
LOG.info("copy file from " + origFilePath + " to " + destFilePath + " success: " + response.toString());
|
||||
return Status.OK;
|
||||
} catch (S3Exception e) {
|
||||
LOG.error("copy file failed: ", e);
|
||||
|
||||
@ -1,433 +0,0 @@
|
||||
// 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.fs.obj;
|
||||
|
||||
import org.apache.doris.analysis.StorageBackend;
|
||||
import org.apache.doris.backup.RemoteFile;
|
||||
import org.apache.doris.backup.Status;
|
||||
import org.apache.doris.common.DdlException;
|
||||
import org.apache.doris.common.UserException;
|
||||
import org.apache.doris.common.util.S3URI;
|
||||
import org.apache.doris.datasource.property.PropertyConverter;
|
||||
import org.apache.doris.datasource.property.constants.S3Properties;
|
||||
|
||||
import org.apache.commons.collections.map.CaseInsensitiveMap;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.http.HttpStatus;
|
||||
import org.apache.http.client.utils.URIBuilder;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
|
||||
import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
|
||||
import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
|
||||
import software.amazon.awssdk.auth.signer.AwsS3V4Signer;
|
||||
import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration;
|
||||
import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption;
|
||||
import software.amazon.awssdk.core.retry.RetryPolicy;
|
||||
import software.amazon.awssdk.core.retry.backoff.EqualJitterBackoffStrategy;
|
||||
import software.amazon.awssdk.core.sync.RequestBody;
|
||||
import software.amazon.awssdk.regions.Region;
|
||||
import software.amazon.awssdk.services.s3.S3Client;
|
||||
import software.amazon.awssdk.services.s3.S3Configuration;
|
||||
import software.amazon.awssdk.services.s3.model.CopyObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.DeleteObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.DeleteObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.GetObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.PutObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.S3Exception;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.nio.file.FileVisitOption;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.time.Duration;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* @see S3ObjStorage
|
||||
* @see org.apache.doris.fs.remote.S3FileSystem
|
||||
*/
|
||||
@Deprecated
|
||||
public class S3Storage extends BlobStorage {
|
||||
private static final Logger LOG = LogManager.getLogger(S3Storage.class);
|
||||
private FileSystem dfsFileSystem = null;
|
||||
private final Map<String, String> caseInsensitiveProperties;
|
||||
private S3Client client;
|
||||
// false: the s3 client will automatically convert endpoint to virtual-hosted style, eg:
|
||||
// endpoint: http://s3.us-east-2.amazonaws.com
|
||||
// bucket/path: my_bucket/file.txt
|
||||
// auto convert: http://my_bucket.s3.us-east-2.amazonaws.com/file.txt
|
||||
// true: the s3 client will NOT automatically convert endpoint to virtual-hosted style, we need to do some tricks:
|
||||
// endpoint: http://cos.ap-beijing.myqcloud.com
|
||||
// bucket/path: my_bucket/file.txt
|
||||
// convert manually: See S3URI()
|
||||
private boolean forceHostedStyle = false;
|
||||
|
||||
public S3Storage(Map<String, String> properties) {
|
||||
caseInsensitiveProperties = new CaseInsensitiveMap();
|
||||
client = null;
|
||||
setProperties(properties);
|
||||
setType(StorageBackend.StorageType.S3);
|
||||
setName(StorageBackend.StorageType.S3.name());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setProperties(Map<String, String> properties) {
|
||||
super.setProperties(properties);
|
||||
caseInsensitiveProperties.putAll(properties);
|
||||
try {
|
||||
S3Properties.requiredS3Properties(caseInsensitiveProperties);
|
||||
} catch (DdlException e) {
|
||||
throw new IllegalArgumentException(e);
|
||||
}
|
||||
// Virtual hosted-style is recommended in the s3 protocol.
|
||||
// The path-style has been abandoned, but for some unexplainable reasons,
|
||||
// the s3 client will determine whether the endpiont starts with `s3`
|
||||
// when generating a virtual hosted-sytle request.
|
||||
// If not, it will not be converted ( https://github.com/aws/aws-sdk-java-v2/pull/763),
|
||||
// but the endpoints of many cloud service providers for object storage do not start with s3,
|
||||
// so they cannot be converted to virtual hosted-sytle.
|
||||
// Some of them, such as aliyun's oss, only support virtual hosted-style,
|
||||
// and some of them(ceph) may only support
|
||||
// path-style, so we need to do some additional conversion.
|
||||
//
|
||||
// use_path_style | !use_path_style
|
||||
// S3 forceHostedStyle=false | forceHostedStyle=false
|
||||
// !S3 forceHostedStyle=false | forceHostedStyle=true
|
||||
//
|
||||
// That is, for S3 endpoint, ignore the `use_path_style` property, and the s3 client will automatically use
|
||||
// virtual hosted-sytle.
|
||||
// And for other endpoint, if `use_path_style` is true, use path style. Otherwise, use virtual hosted-sytle.
|
||||
if (!caseInsensitiveProperties.get(S3Properties.ENDPOINT).toLowerCase().startsWith("s3")) {
|
||||
forceHostedStyle = !caseInsensitiveProperties.getOrDefault(PropertyConverter.USE_PATH_STYLE, "false")
|
||||
.equalsIgnoreCase("true");
|
||||
} else {
|
||||
forceHostedStyle = false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileSystem getFileSystem(String remotePath) throws UserException {
|
||||
if (dfsFileSystem == null) {
|
||||
Configuration conf = new Configuration();
|
||||
System.setProperty("com.amazonaws.services.s3.enableV4", "true");
|
||||
PropertyConverter.convertToHadoopFSProperties(caseInsensitiveProperties).forEach(conf::set);
|
||||
try {
|
||||
dfsFileSystem = FileSystem.get(new URI(remotePath), conf);
|
||||
} catch (Exception e) {
|
||||
throw new UserException("Failed to get S3 FileSystem for " + e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
return dfsFileSystem;
|
||||
}
|
||||
|
||||
private S3Client getClient(String bucket) throws UserException {
|
||||
if (client == null) {
|
||||
URI tmpEndpoint = URI.create(caseInsensitiveProperties.get(S3Properties.ENDPOINT));
|
||||
StaticCredentialsProvider scp;
|
||||
if (!caseInsensitiveProperties.containsKey(S3Properties.SESSION_TOKEN)) {
|
||||
AwsBasicCredentials awsBasic = AwsBasicCredentials.create(
|
||||
caseInsensitiveProperties.get(S3Properties.ACCESS_KEY),
|
||||
caseInsensitiveProperties.get(S3Properties.SECRET_KEY));
|
||||
scp = StaticCredentialsProvider.create(awsBasic);
|
||||
} else {
|
||||
AwsSessionCredentials awsSession = AwsSessionCredentials.create(
|
||||
caseInsensitiveProperties.get(S3Properties.ACCESS_KEY),
|
||||
caseInsensitiveProperties.get(S3Properties.SECRET_KEY),
|
||||
caseInsensitiveProperties.get(S3Properties.SESSION_TOKEN));
|
||||
scp = StaticCredentialsProvider.create(awsSession);
|
||||
}
|
||||
EqualJitterBackoffStrategy backoffStrategy = EqualJitterBackoffStrategy
|
||||
.builder()
|
||||
.baseDelay(Duration.ofSeconds(1))
|
||||
.maxBackoffTime(Duration.ofMinutes(1))
|
||||
.build();
|
||||
// retry 3 time with Equal backoff
|
||||
RetryPolicy retryPolicy = RetryPolicy
|
||||
.builder()
|
||||
.numRetries(3)
|
||||
.backoffStrategy(backoffStrategy)
|
||||
.build();
|
||||
ClientOverrideConfiguration clientConf = ClientOverrideConfiguration
|
||||
.builder()
|
||||
// set retry policy
|
||||
.retryPolicy(retryPolicy)
|
||||
// using AwsS3V4Signer
|
||||
.putAdvancedOption(SdkAdvancedClientOption.SIGNER, AwsS3V4Signer.create())
|
||||
.build();
|
||||
URI endpoint = StringUtils.isEmpty(bucket) ? tmpEndpoint :
|
||||
URI.create(new URIBuilder(tmpEndpoint).setHost(bucket + "." + tmpEndpoint.getHost()).toString());
|
||||
client = S3Client.builder()
|
||||
.endpointOverride(endpoint)
|
||||
.credentialsProvider(scp)
|
||||
.region(Region.of(caseInsensitiveProperties.get(S3Properties.REGION)))
|
||||
.overrideConfiguration(clientConf)
|
||||
// disable chunkedEncoding because of bos not supported
|
||||
// use virtual hosted-style access
|
||||
.serviceConfiguration(S3Configuration.builder()
|
||||
.chunkedEncodingEnabled(false)
|
||||
.pathStyleAccessEnabled(false)
|
||||
.build())
|
||||
.build();
|
||||
}
|
||||
return client;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status downloadWithFileSize(String remoteFilePath, String localFilePath, long fileSize) {
|
||||
long start = System.currentTimeMillis();
|
||||
// Write the data to a local file
|
||||
File localFile = new File(localFilePath);
|
||||
if (localFile.exists()) {
|
||||
try {
|
||||
Files.walk(Paths.get(localFilePath), FileVisitOption.FOLLOW_LINKS)
|
||||
.sorted(Comparator.reverseOrder())
|
||||
.map(Path::toFile)
|
||||
.forEach(File::delete);
|
||||
} catch (IOException e) {
|
||||
return new Status(
|
||||
Status.ErrCode.COMMON_ERROR, "failed to delete exist local file: " + localFilePath);
|
||||
}
|
||||
}
|
||||
try {
|
||||
S3URI uri = S3URI.create(remoteFilePath, forceHostedStyle);
|
||||
GetObjectResponse response = getClient(uri.getVirtualBucket()).getObject(
|
||||
GetObjectRequest.builder().bucket(uri.getBucket()).key(uri.getKey()).build(), localFile.toPath());
|
||||
if (localFile.length() == fileSize) {
|
||||
LOG.info(
|
||||
"finished to download from {} to {} with size: {}. cost {} ms",
|
||||
remoteFilePath,
|
||||
localFilePath,
|
||||
fileSize,
|
||||
(System.currentTimeMillis() - start));
|
||||
return Status.OK;
|
||||
} else {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, response.toString());
|
||||
}
|
||||
} catch (S3Exception s3Exception) {
|
||||
return new Status(
|
||||
Status.ErrCode.COMMON_ERROR,
|
||||
"get file from s3 error: " + s3Exception.awsErrorDetails().errorMessage());
|
||||
} catch (UserException ue) {
|
||||
LOG.warn("connect to s3 failed: ", ue);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "connect to s3 failed: " + ue.getMessage());
|
||||
} catch (Exception e) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, e.toString());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status directUpload(String content, String remoteFile) {
|
||||
try {
|
||||
S3URI uri = S3URI.create(remoteFile, forceHostedStyle);
|
||||
PutObjectResponse response =
|
||||
getClient(uri.getVirtualBucket())
|
||||
.putObject(
|
||||
PutObjectRequest.builder().bucket(uri.getBucket()).key(uri.getKey()).build(),
|
||||
RequestBody.fromBytes(content.getBytes()));
|
||||
LOG.info("upload content success: " + response.eTag());
|
||||
return Status.OK;
|
||||
} catch (S3Exception e) {
|
||||
LOG.warn("write content failed:", e);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "write content failed: " + e.getMessage());
|
||||
} catch (UserException ue) {
|
||||
LOG.warn("connect to s3 failed: ", ue);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "connect to s3 failed: " + ue.getMessage());
|
||||
} catch (Exception e) {
|
||||
LOG.warn("connect to s3 failed: ", e);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "connect to s3 failed: " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public Status copy(String origFilePath, String destFilePath) {
|
||||
try {
|
||||
S3URI origUri = S3URI.create(origFilePath);
|
||||
S3URI descUri = S3URI.create(destFilePath, forceHostedStyle);
|
||||
getClient(descUri.getVirtualBucket())
|
||||
.copyObject(
|
||||
CopyObjectRequest.builder()
|
||||
.copySource(origUri.getBucket() + "/" + origUri.getKey())
|
||||
.destinationBucket(descUri.getBucket())
|
||||
.destinationKey(descUri.getKey())
|
||||
.build());
|
||||
return Status.OK;
|
||||
} catch (S3Exception e) {
|
||||
LOG.error("copy file failed: ", e);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "copy file failed: " + e.getMessage());
|
||||
} catch (UserException ue) {
|
||||
LOG.error("copy to s3 failed: ", ue);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "connect to s3 failed: " + ue.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status upload(String localPath, String remotePath) {
|
||||
try {
|
||||
S3URI uri = S3URI.create(remotePath, forceHostedStyle);
|
||||
PutObjectResponse response =
|
||||
getClient(uri.getVirtualBucket())
|
||||
.putObject(
|
||||
PutObjectRequest.builder().bucket(uri.getBucket()).key(uri.getKey()).build(),
|
||||
RequestBody.fromFile(new File(localPath)));
|
||||
LOG.info("upload file " + localPath + " success: " + response.eTag());
|
||||
return Status.OK;
|
||||
} catch (S3Exception e) {
|
||||
LOG.error("write file failed:", e);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "write file failed: " + e.getMessage());
|
||||
} catch (UserException ue) {
|
||||
LOG.error("connect to s3 failed: ", ue);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "connect to s3 failed: " + ue.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status rename(String origFilePath, String destFilePath) {
|
||||
Status status = copy(origFilePath, destFilePath);
|
||||
if (status.ok()) {
|
||||
return delete(origFilePath);
|
||||
} else {
|
||||
return status;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status delete(String remotePath) {
|
||||
try {
|
||||
S3URI uri = S3URI.create(remotePath, forceHostedStyle);
|
||||
DeleteObjectResponse response =
|
||||
getClient(uri.getVirtualBucket())
|
||||
.deleteObject(
|
||||
DeleteObjectRequest.builder().bucket(uri.getBucket()).key(uri.getKey()).build());
|
||||
LOG.info("delete file " + remotePath + " success: " + response.toString());
|
||||
return Status.OK;
|
||||
} catch (S3Exception e) {
|
||||
LOG.warn("delete file failed: ", e);
|
||||
if (e.statusCode() == HttpStatus.SC_NOT_FOUND) {
|
||||
return Status.OK;
|
||||
}
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "delete file failed: " + e.getMessage());
|
||||
} catch (UserException ue) {
|
||||
LOG.warn("connect to s3 failed: ", ue);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "connect to s3 failed: " + ue.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<LocatedFileStatus> listLocatedStatus(String remotePath) throws UserException {
|
||||
FileSystem fileSystem = getFileSystem(remotePath);
|
||||
try {
|
||||
return fileSystem.listLocatedStatus(new org.apache.hadoop.fs.Path(remotePath));
|
||||
} catch (IOException e) {
|
||||
throw new UserException("Failed to list located status for path: " + remotePath, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status list(String remotePath, List<RemoteFile> result) {
|
||||
return list(remotePath, result, true);
|
||||
}
|
||||
|
||||
// broker file pattern glob is too complex, so we use hadoop directly
|
||||
@Override
|
||||
public Status list(String remotePath, List<RemoteFile> result, boolean fileNameOnly) {
|
||||
try {
|
||||
FileSystem s3AFileSystem = getFileSystem(remotePath);
|
||||
org.apache.hadoop.fs.Path pathPattern = new org.apache.hadoop.fs.Path(remotePath);
|
||||
FileStatus[] files = s3AFileSystem.globStatus(pathPattern);
|
||||
if (files == null) {
|
||||
return Status.OK;
|
||||
}
|
||||
for (FileStatus fileStatus : files) {
|
||||
RemoteFile remoteFile = new RemoteFile(
|
||||
fileNameOnly ? fileStatus.getPath().getName() : fileStatus.getPath().toString(),
|
||||
!fileStatus.isDirectory(), fileStatus.isDirectory() ? -1 : fileStatus.getLen(),
|
||||
fileStatus.getBlockSize());
|
||||
result.add(remoteFile);
|
||||
}
|
||||
} catch (FileNotFoundException e) {
|
||||
LOG.info("file not found: " + e.getMessage());
|
||||
return new Status(Status.ErrCode.NOT_FOUND, "file not found: " + e.getMessage());
|
||||
} catch (Exception e) {
|
||||
LOG.error("errors while get file status ", e);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "errors while get file status " + e.getMessage());
|
||||
}
|
||||
return Status.OK;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status makeDir(String remotePath) {
|
||||
if (!remotePath.endsWith("/")) {
|
||||
remotePath += "/";
|
||||
}
|
||||
try {
|
||||
S3URI uri = S3URI.create(remotePath, forceHostedStyle);
|
||||
PutObjectResponse response =
|
||||
getClient(uri.getVirtualBucket())
|
||||
.putObject(
|
||||
PutObjectRequest.builder().bucket(uri.getBucket()).key(uri.getKey()).build(),
|
||||
RequestBody.empty());
|
||||
LOG.info("makeDir success: " + response.eTag());
|
||||
return Status.OK;
|
||||
} catch (S3Exception e) {
|
||||
LOG.error("makeDir failed:", e);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "makeDir failed: " + e.getMessage());
|
||||
} catch (UserException ue) {
|
||||
LOG.error("connect to s3 failed: ", ue);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "connect to s3 failed: " + ue.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status checkPathExist(String remotePath) {
|
||||
try {
|
||||
S3URI uri = S3URI.create(remotePath, forceHostedStyle);
|
||||
getClient(uri.getVirtualBucket())
|
||||
.headObject(HeadObjectRequest.builder().bucket(uri.getBucket()).key(uri.getKey()).build());
|
||||
return Status.OK;
|
||||
} catch (S3Exception e) {
|
||||
if (e.statusCode() == HttpStatus.SC_NOT_FOUND) {
|
||||
return new Status(Status.ErrCode.NOT_FOUND, "remote path does not exist: " + remotePath);
|
||||
} else {
|
||||
LOG.warn("headObject failed:", e);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "headObject failed: " + e.getMessage());
|
||||
}
|
||||
} catch (UserException ue) {
|
||||
LOG.warn("connect to s3 failed: ", ue);
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "connect to s3 failed: " + ue.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public StorageBackend.StorageType getStorageType() {
|
||||
return StorageBackend.StorageType.S3;
|
||||
}
|
||||
}
|
||||
@ -18,7 +18,6 @@
|
||||
package org.apache.doris.fs.remote;
|
||||
|
||||
import org.apache.doris.analysis.StorageBackend;
|
||||
import org.apache.doris.backup.RemoteFile;
|
||||
import org.apache.doris.backup.Status;
|
||||
import org.apache.doris.catalog.Env;
|
||||
import org.apache.doris.catalog.FsBroker;
|
||||
|
||||
@ -48,6 +48,13 @@ public abstract class ObjFileSystem extends RemoteFileSystem {
|
||||
return objStorage.headObject(remotePath);
|
||||
}
|
||||
|
||||
/**
|
||||
* download data from remote file and check data size with expected file size.
|
||||
* @param remoteFilePath remote file path
|
||||
* @param localFilePath local file path
|
||||
* @param fileSize download data size
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public Status downloadWithFileSize(String remoteFilePath, String localFilePath, long fileSize) {
|
||||
long start = System.currentTimeMillis();
|
||||
|
||||
@ -15,38 +15,63 @@
|
||||
// specific language governing permissions and limitations
|
||||
// under the License.
|
||||
|
||||
package org.apache.doris.backup;
|
||||
package org.apache.doris.fs.remote;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
import org.apache.hadoop.fs.BlockLocation;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
// represent a file or a dir in remote storage
|
||||
public class RemoteFile {
|
||||
// Only file name, not full path
|
||||
private String name;
|
||||
private boolean isFile;
|
||||
private long size;
|
||||
private final String name;
|
||||
private final boolean isFile;
|
||||
private final boolean isDirectory;
|
||||
private final long size;
|
||||
// Block size of underlying file system. e.g. HDFS and S3.
|
||||
// A large file will split into multiple blocks. The blocks are transparent to the user.
|
||||
// Default block size for HDFS 2.x is 128M.
|
||||
private long blockSize;
|
||||
private final long blockSize;
|
||||
private Path path;
|
||||
BlockLocation[] blockLocations;
|
||||
|
||||
public RemoteFile(String name, boolean isFile, long size, long blockSize) {
|
||||
this(name, null, isFile, !isFile, size, blockSize, null);
|
||||
}
|
||||
|
||||
public RemoteFile(Path path, boolean isDirectory, long size, long blockSize, BlockLocation[] blockLocations) {
|
||||
this(path.getName(), path, !isDirectory, isDirectory, size, blockSize, blockLocations);
|
||||
}
|
||||
|
||||
public RemoteFile(String name, Path path, boolean isFile, boolean isDirectory,
|
||||
long size, long blockSize, BlockLocation[] blockLocations) {
|
||||
Preconditions.checkState(!Strings.isNullOrEmpty(name));
|
||||
this.name = name;
|
||||
this.isFile = isFile;
|
||||
this.isDirectory = isDirectory;
|
||||
this.size = size;
|
||||
this.blockSize = blockSize;
|
||||
this.path = path;
|
||||
this.blockLocations = blockLocations;
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
public Path getPath() {
|
||||
return path;
|
||||
}
|
||||
|
||||
public boolean isFile() {
|
||||
return isFile;
|
||||
}
|
||||
|
||||
public boolean isDirectory() {
|
||||
return isDirectory;
|
||||
}
|
||||
|
||||
public long getSize() {
|
||||
return size;
|
||||
}
|
||||
@ -55,6 +80,10 @@ public class RemoteFile {
|
||||
return blockSize;
|
||||
}
|
||||
|
||||
public BlockLocation[] getBlockLocations() {
|
||||
return blockLocations;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "[name: " + name + ", is file: " + isFile + "]";
|
||||
@ -20,12 +20,15 @@ package org.apache.doris.fs.remote;
|
||||
import org.apache.doris.analysis.StorageBackend;
|
||||
import org.apache.doris.common.UserException;
|
||||
import org.apache.doris.fs.PersistentFileSystem;
|
||||
import org.apache.doris.fs.RemoteFiles;
|
||||
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public abstract class RemoteFileSystem extends PersistentFileSystem {
|
||||
protected org.apache.hadoop.fs.FileSystem dfsFileSystem = null;
|
||||
@ -34,17 +37,31 @@ public abstract class RemoteFileSystem extends PersistentFileSystem {
|
||||
super(name, type);
|
||||
}
|
||||
|
||||
protected org.apache.hadoop.fs.FileSystem getFileSystem(String remotePath) throws UserException {
|
||||
protected org.apache.hadoop.fs.FileSystem nativeFileSystem(String remotePath) throws UserException {
|
||||
throw new UserException("Not support to getFileSystem.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<LocatedFileStatus> listLocatedStatus(String remotePath) throws UserException {
|
||||
org.apache.hadoop.fs.FileSystem fileSystem = getFileSystem(remotePath);
|
||||
public RemoteFiles listLocatedFiles(String remotePath, boolean onlyFiles, boolean recursive) throws UserException {
|
||||
org.apache.hadoop.fs.FileSystem fileSystem = nativeFileSystem(remotePath);
|
||||
try {
|
||||
return fileSystem.listLocatedStatus(new Path(remotePath));
|
||||
Path locatedPath = new Path(remotePath);
|
||||
RemoteIterator<LocatedFileStatus> locatedFiles = onlyFiles ? fileSystem.listFiles(locatedPath, recursive)
|
||||
: fileSystem.listLocatedStatus(locatedPath);
|
||||
return getFileLocations(locatedFiles);
|
||||
} catch (IOException e) {
|
||||
throw new UserException("Failed to list located status for path: " + remotePath, e);
|
||||
}
|
||||
}
|
||||
|
||||
private RemoteFiles getFileLocations(RemoteIterator<LocatedFileStatus> locatedFiles) throws IOException {
|
||||
List<RemoteFile> locations = new ArrayList<>();
|
||||
while (locatedFiles.hasNext()) {
|
||||
LocatedFileStatus fileStatus = locatedFiles.next();
|
||||
RemoteFile location = new RemoteFile(fileStatus.getPath(), fileStatus.isDirectory(),
|
||||
fileStatus.getLen(), fileStatus.getBlockSize(), fileStatus.getBlockLocations());
|
||||
locations.add(location);
|
||||
}
|
||||
return new RemoteFiles(locations);
|
||||
}
|
||||
}
|
||||
|
||||
@ -18,12 +18,12 @@
|
||||
package org.apache.doris.fs.remote;
|
||||
|
||||
import org.apache.doris.analysis.StorageBackend;
|
||||
import org.apache.doris.backup.RemoteFile;
|
||||
import org.apache.doris.backup.Status;
|
||||
import org.apache.doris.common.UserException;
|
||||
import org.apache.doris.datasource.property.PropertyConverter;
|
||||
import org.apache.doris.fs.obj.S3ObjStorage;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@ -35,7 +35,6 @@ import java.io.FileNotFoundException;
|
||||
import java.net.URI;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
public class S3FileSystem extends ObjFileSystem {
|
||||
|
||||
@ -43,11 +42,17 @@ public class S3FileSystem extends ObjFileSystem {
|
||||
|
||||
public S3FileSystem(Map<String, String> properties) {
|
||||
super(StorageBackend.StorageType.S3.name(), StorageBackend.StorageType.S3, new S3ObjStorage(properties));
|
||||
this.properties = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
|
||||
this.properties.putAll(properties);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public S3FileSystem(S3ObjStorage storage) {
|
||||
super(StorageBackend.StorageType.S3.name(), StorageBackend.StorageType.S3, storage);
|
||||
this.properties.putAll(storage.getProperties());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FileSystem getFileSystem(String remotePath) throws UserException {
|
||||
protected FileSystem nativeFileSystem(String remotePath) throws UserException {
|
||||
if (dfsFileSystem == null) {
|
||||
Configuration conf = new Configuration();
|
||||
System.setProperty("com.amazonaws.services.s3.enableV4", "true");
|
||||
@ -65,7 +70,7 @@ public class S3FileSystem extends ObjFileSystem {
|
||||
@Override
|
||||
public Status list(String remotePath, List<RemoteFile> result, boolean fileNameOnly) {
|
||||
try {
|
||||
FileSystem s3AFileSystem = getFileSystem(remotePath);
|
||||
FileSystem s3AFileSystem = nativeFileSystem(remotePath);
|
||||
Path pathPattern = new Path(remotePath);
|
||||
FileStatus[] files = s3AFileSystem.globStatus(pathPattern);
|
||||
if (files == null) {
|
||||
|
||||
@ -18,7 +18,6 @@
|
||||
package org.apache.doris.fs.remote.dfs;
|
||||
|
||||
import org.apache.doris.analysis.StorageBackend;
|
||||
import org.apache.doris.backup.RemoteFile;
|
||||
import org.apache.doris.backup.Status;
|
||||
import org.apache.doris.catalog.AuthType;
|
||||
import org.apache.doris.catalog.HdfsResource;
|
||||
@ -27,6 +26,7 @@ import org.apache.doris.common.util.URI;
|
||||
import org.apache.doris.fs.operations.HDFSFileOperations;
|
||||
import org.apache.doris.fs.operations.HDFSOpParams;
|
||||
import org.apache.doris.fs.operations.OpParams;
|
||||
import org.apache.doris.fs.remote.RemoteFile;
|
||||
import org.apache.doris.fs.remote.RemoteFileSystem;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
@ -70,7 +70,7 @@ public class DFSFileSystem extends RemoteFileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FileSystem getFileSystem(String remotePath) throws UserException {
|
||||
protected FileSystem nativeFileSystem(String remotePath) throws UserException {
|
||||
if (dfsFileSystem != null) {
|
||||
return dfsFileSystem;
|
||||
}
|
||||
@ -261,7 +261,7 @@ public class DFSFileSystem extends RemoteFileSystem {
|
||||
try {
|
||||
URI pathUri = URI.create(remotePath);
|
||||
Path inputFilePath = new Path(pathUri.getPath());
|
||||
FileSystem fileSystem = getFileSystem(remotePath);
|
||||
FileSystem fileSystem = nativeFileSystem(remotePath);
|
||||
boolean isPathExist = fileSystem.exists(inputFilePath);
|
||||
if (!isPathExist) {
|
||||
return new Status(Status.ErrCode.NOT_FOUND, "remote path does not exist: " + remotePath);
|
||||
@ -372,7 +372,7 @@ public class DFSFileSystem extends RemoteFileSystem {
|
||||
if (!srcPathUri.getAuthority().trim().equals(destPathUri.getAuthority().trim())) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, "only allow rename in same file system");
|
||||
}
|
||||
FileSystem fileSystem = getFileSystem(destPath);
|
||||
FileSystem fileSystem = nativeFileSystem(destPath);
|
||||
Path srcfilePath = new Path(srcPathUri.getPath());
|
||||
Path destfilePath = new Path(destPathUri.getPath());
|
||||
boolean isRenameSuccess = fileSystem.rename(srcfilePath, destfilePath);
|
||||
@ -395,7 +395,7 @@ public class DFSFileSystem extends RemoteFileSystem {
|
||||
try {
|
||||
URI pathUri = URI.create(remotePath);
|
||||
Path inputFilePath = new Path(pathUri.getPath());
|
||||
FileSystem fileSystem = getFileSystem(remotePath);
|
||||
FileSystem fileSystem = nativeFileSystem(remotePath);
|
||||
fileSystem.delete(inputFilePath, true);
|
||||
} catch (UserException e) {
|
||||
return new Status(Status.ErrCode.COMMON_ERROR, e.getMessage());
|
||||
@ -420,7 +420,7 @@ public class DFSFileSystem extends RemoteFileSystem {
|
||||
public Status list(String remotePath, List<RemoteFile> result, boolean fileNameOnly) {
|
||||
try {
|
||||
URI pathUri = URI.create(remotePath);
|
||||
FileSystem fileSystem = getFileSystem(remotePath);
|
||||
FileSystem fileSystem = nativeFileSystem(remotePath);
|
||||
Path pathPattern = new Path(pathUri.getPath());
|
||||
FileStatus[] files = fileSystem.globStatus(pathPattern);
|
||||
if (files == null) {
|
||||
|
||||
@ -22,7 +22,6 @@ import org.apache.doris.catalog.HiveMetaStoreClientHelper;
|
||||
import org.apache.doris.catalog.TableIf;
|
||||
import org.apache.doris.catalog.external.HMSExternalTable;
|
||||
import org.apache.doris.common.DdlException;
|
||||
import org.apache.doris.common.FeConstants;
|
||||
import org.apache.doris.common.MetaNotFoundException;
|
||||
import org.apache.doris.common.UserException;
|
||||
import org.apache.doris.planner.ColumnRange;
|
||||
@ -86,28 +85,8 @@ public class HiveScanProvider extends HMSTableScanProvider {
|
||||
@Override
|
||||
public TFileType getLocationType() throws DdlException, MetaNotFoundException {
|
||||
String location = hmsTable.getRemoteTable().getSd().getLocation();
|
||||
if (location != null && !location.isEmpty()) {
|
||||
if (location.startsWith(FeConstants.FS_PREFIX_S3)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_S3A)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_S3N)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_BOS)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_COS)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_OSS)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_OBS)) {
|
||||
return TFileType.FILE_S3;
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_HDFS)) {
|
||||
return TFileType.FILE_HDFS;
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_FILE)) {
|
||||
return TFileType.FILE_LOCAL;
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_OFS)) {
|
||||
return TFileType.FILE_BROKER;
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_GFS)) {
|
||||
return TFileType.FILE_BROKER;
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_JFS)) {
|
||||
return TFileType.FILE_BROKER;
|
||||
}
|
||||
}
|
||||
throw new DdlException("Unknown file location " + location + " for hms table " + hmsTable.getName());
|
||||
return getTFileType(location).orElseThrow(() ->
|
||||
new DdlException("Unknown file location " + location + " for hms table " + hmsTable.getName()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@ -29,6 +29,9 @@ import org.apache.doris.datasource.HMSExternalCatalog;
|
||||
import org.apache.doris.datasource.hive.HiveMetaStoreCache;
|
||||
import org.apache.doris.datasource.hive.HivePartition;
|
||||
import org.apache.doris.external.hive.util.HiveUtil;
|
||||
import org.apache.doris.fs.FileSystemFactory;
|
||||
import org.apache.doris.fs.RemoteFiles;
|
||||
import org.apache.doris.fs.remote.RemoteFileSystem;
|
||||
import org.apache.doris.planner.ColumnRange;
|
||||
import org.apache.doris.planner.ListPartitionPrunerV2;
|
||||
import org.apache.doris.planner.Split;
|
||||
@ -37,10 +40,7 @@ import org.apache.doris.qe.ConnectContext;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.fs.BlockLocation;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.mapred.InputFormat;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
@ -193,17 +193,14 @@ public class HiveSplitter implements Splitter {
|
||||
}
|
||||
|
||||
// Get File Status by using FileSystem API.
|
||||
public static HiveMetaStoreCache.FileCacheValue getFileCache(Path path, InputFormat<?, ?> inputFormat,
|
||||
public static HiveMetaStoreCache.FileCacheValue getFileCache(String location, InputFormat<?, ?> inputFormat,
|
||||
JobConf jobConf,
|
||||
List<String> partitionValues) throws IOException {
|
||||
FileSystem fs = path.getFileSystem(jobConf);
|
||||
boolean splittable = HiveUtil.isSplittable(inputFormat, fs, path);
|
||||
RemoteIterator<LocatedFileStatus> locatedFileStatusRemoteIterator = fs.listFiles(path, false);
|
||||
List<String> partitionValues) throws UserException {
|
||||
HiveMetaStoreCache.FileCacheValue result = new HiveMetaStoreCache.FileCacheValue();
|
||||
result.setSplittable(splittable);
|
||||
while (locatedFileStatusRemoteIterator.hasNext()) {
|
||||
result.addFile(locatedFileStatusRemoteIterator.next());
|
||||
}
|
||||
result.setSplittable(HiveUtil.isSplittable(inputFormat, new Path(location), jobConf));
|
||||
RemoteFileSystem fs = FileSystemFactory.getByLocation(location, jobConf);
|
||||
RemoteFiles locatedFiles = fs.listLocatedFiles(location, true, false);
|
||||
locatedFiles.locations().forEach(result::addFile);
|
||||
result.setPartitionValues(partitionValues);
|
||||
return result;
|
||||
}
|
||||
|
||||
@ -23,6 +23,7 @@ import org.apache.doris.catalog.Env;
|
||||
import org.apache.doris.catalog.FsBroker;
|
||||
import org.apache.doris.catalog.HdfsResource;
|
||||
import org.apache.doris.common.DdlException;
|
||||
import org.apache.doris.common.FeConstants;
|
||||
import org.apache.doris.common.MetaNotFoundException;
|
||||
import org.apache.doris.common.UserException;
|
||||
import org.apache.doris.common.util.BrokerUtil;
|
||||
@ -51,6 +52,7 @@ import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
||||
public abstract class QueryScanProvider implements FileScanProviderIf {
|
||||
public static final Logger LOG = LogManager.getLogger(QueryScanProvider.class);
|
||||
@ -201,4 +203,23 @@ public abstract class QueryScanProvider implements FileScanProviderIf {
|
||||
}
|
||||
return rangeDesc;
|
||||
}
|
||||
|
||||
protected static Optional<TFileType> getTFileType(String location) {
|
||||
if (location != null && !location.isEmpty()) {
|
||||
if (FeConstants.isObjStorage(location)) {
|
||||
return Optional.of(TFileType.FILE_S3);
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_HDFS)) {
|
||||
return Optional.of(TFileType.FILE_HDFS);
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_FILE)) {
|
||||
return Optional.of(TFileType.FILE_LOCAL);
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_OFS)) {
|
||||
return Optional.of(TFileType.FILE_BROKER);
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_GFS)) {
|
||||
return Optional.of(TFileType.FILE_BROKER);
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_JFS)) {
|
||||
return Optional.of(TFileType.FILE_BROKER);
|
||||
}
|
||||
}
|
||||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
|
||||
@ -20,7 +20,6 @@ package org.apache.doris.planner.external.iceberg;
|
||||
import org.apache.doris.analysis.Analyzer;
|
||||
import org.apache.doris.catalog.TableIf;
|
||||
import org.apache.doris.common.DdlException;
|
||||
import org.apache.doris.common.FeConstants;
|
||||
import org.apache.doris.common.MetaNotFoundException;
|
||||
import org.apache.doris.common.UserException;
|
||||
import org.apache.doris.planner.external.IcebergSplitter;
|
||||
@ -95,30 +94,10 @@ public class IcebergScanProvider extends QueryScanProvider {
|
||||
|
||||
@Override
|
||||
public TFileType getLocationType() throws DdlException, MetaNotFoundException {
|
||||
String location = icebergSource.getIcebergTable().location();
|
||||
if (location != null && !location.isEmpty()) {
|
||||
if (location.startsWith(FeConstants.FS_PREFIX_S3)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_S3A)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_S3N)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_BOS)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_COS)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_OSS)
|
||||
|| location.startsWith(FeConstants.FS_PREFIX_OBS)) {
|
||||
return TFileType.FILE_S3;
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_HDFS)) {
|
||||
return TFileType.FILE_HDFS;
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_FILE)) {
|
||||
return TFileType.FILE_LOCAL;
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_OFS)) {
|
||||
return TFileType.FILE_BROKER;
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_GFS)) {
|
||||
return TFileType.FILE_BROKER;
|
||||
} else if (location.startsWith(FeConstants.FS_PREFIX_JFS)) {
|
||||
return TFileType.FILE_BROKER;
|
||||
}
|
||||
}
|
||||
throw new DdlException("Unknown file location " + location
|
||||
+ " for hms table " + icebergSource.getIcebergTable().name());
|
||||
org.apache.iceberg.Table table = icebergSource.getIcebergTable();
|
||||
String location = table.location();
|
||||
return getTFileType(location).orElseThrow(() ->
|
||||
new DdlException("Unknown file location " + location + " for iceberg table " + table.name()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@ -18,7 +18,6 @@
|
||||
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.common.AnalysisException;
|
||||
import org.apache.doris.common.FeConstants;
|
||||
@ -88,7 +87,7 @@ public class S3TableValuedFunction extends ExternalFileTableValuedFunction {
|
||||
locationProperties.put(PropertyConverter.USE_PATH_STYLE, usePathStyle);
|
||||
if (FeConstants.runningUnitTest) {
|
||||
// Just check
|
||||
FileSystemFactory.get(StorageBackend.StorageType.S3, locationProperties);
|
||||
FileSystemFactory.getS3FileSystem(locationProperties);
|
||||
} else {
|
||||
parseFile();
|
||||
}
|
||||
|
||||
@ -31,7 +31,7 @@ import org.apache.doris.common.FeConstants;
|
||||
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.obj.BlobStorage;
|
||||
import org.apache.doris.fs.FileSystemFactory;
|
||||
import org.apache.doris.persist.EditLog;
|
||||
import org.apache.doris.task.AgentBatchTask;
|
||||
import org.apache.doris.task.AgentTask;
|
||||
@ -121,7 +121,7 @@ public class BackupJobTest {
|
||||
private EditLog editLog;
|
||||
|
||||
private Repository repo = new Repository(repoId, "repo", false, "my_repo",
|
||||
BlobStorage.create("broker", StorageBackend.StorageType.BROKER, Maps.newHashMap()));
|
||||
FileSystemFactory.get("broker", StorageBackend.StorageType.BROKER, Maps.newHashMap()));
|
||||
|
||||
@BeforeClass
|
||||
public static void start() {
|
||||
|
||||
@ -21,7 +21,8 @@ import org.apache.doris.common.ClientPool;
|
||||
import org.apache.doris.common.GenericPool;
|
||||
import org.apache.doris.common.Pair;
|
||||
import org.apache.doris.common.jmockit.Deencapsulation;
|
||||
import org.apache.doris.fs.obj.BrokerStorage;
|
||||
import org.apache.doris.fs.remote.BrokerFileSystem;
|
||||
import org.apache.doris.fs.remote.RemoteFile;
|
||||
import org.apache.doris.thrift.TNetworkAddress;
|
||||
import org.apache.doris.thrift.TPaloBrokerService;
|
||||
|
||||
@ -62,7 +63,7 @@ public class BrokerStorageTest {
|
||||
private Map<String, String> properties;
|
||||
|
||||
@Tested
|
||||
private BrokerStorage storage;
|
||||
private BrokerFileSystem fileSystem;
|
||||
private String testFile;
|
||||
private String content;
|
||||
private Pair<TPaloBrokerService.Client, TNetworkAddress> pair;
|
||||
@ -86,7 +87,7 @@ public class BrokerStorageTest {
|
||||
properties.put("bos_accesskey", System.getenv().getOrDefault("AWS_AK", ""));
|
||||
properties.put("bos_secret_accesskey", System.getenv().getOrDefault("AWS_SK", ""));
|
||||
properties.put("bos_endpoint", "http://bj.bcebos.com");
|
||||
storage = new BrokerStorage("bos_broker", properties);
|
||||
fileSystem = new BrokerFileSystem("bos_broker", properties);
|
||||
testFile = bucket + basePath + "/Ode_to_the_West_Wind";
|
||||
content =
|
||||
"O wild West Wind, thou breath of Autumn's being\n"
|
||||
@ -103,9 +104,9 @@ public class BrokerStorageTest {
|
||||
+ "With living hues and odors plain and hill:\n"
|
||||
+ "Wild Spirit, which art moving everywhere;\n"
|
||||
+ "Destroyer and preserver; hear, oh, hear!";
|
||||
new MockUp<BrokerStorage>() {
|
||||
new MockUp<BrokerFileSystem>() {
|
||||
@Mock
|
||||
private Pair<TPaloBrokerService.Client, TNetworkAddress> getBroker() {
|
||||
public Pair<TPaloBrokerService.Client, TNetworkAddress> getBroker() {
|
||||
return pair;
|
||||
}
|
||||
};
|
||||
@ -116,17 +117,17 @@ public class BrokerStorageTest {
|
||||
}
|
||||
};
|
||||
Deencapsulation.setField(ClientPool.class, "brokerPool", pool);
|
||||
Assert.assertEquals(Status.OK, storage.directUpload(content, testFile));
|
||||
Assert.assertEquals(Status.OK, fileSystem.directUpload(content, testFile));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void downloadWithFileSize() throws IOException {
|
||||
File localFile = File.createTempFile("brokerunittest", ".dat");
|
||||
localFile.deleteOnExit();
|
||||
Status status = storage.downloadWithFileSize(testFile, localFile.getAbsolutePath(), content.getBytes().length);
|
||||
Status status = fileSystem.downloadWithFileSize(testFile, localFile.getAbsolutePath(), content.getBytes().length);
|
||||
Assert.assertEquals(Status.OK, status);
|
||||
Assert.assertEquals(DigestUtils.md5Hex(content.getBytes()), DigestUtils.md5Hex(new FileInputStream(localFile)));
|
||||
status = storage.downloadWithFileSize(bucket + basePath + "/Ode_to_the_West_Wind", localFile.getAbsolutePath(), content.getBytes().length + 1);
|
||||
status = fileSystem.downloadWithFileSize(bucket + basePath + "/Ode_to_the_West_Wind", localFile.getAbsolutePath(), content.getBytes().length + 1);
|
||||
Assert.assertNotEquals(Status.OK, status);
|
||||
}
|
||||
|
||||
@ -141,11 +142,11 @@ public class BrokerStorageTest {
|
||||
os.write(buf);
|
||||
os.close();
|
||||
String remote = bucket + basePath + "/" + localFile.getName();
|
||||
Status status = storage.upload(localFile.getAbsolutePath(), remote);
|
||||
Status status = fileSystem.upload(localFile.getAbsolutePath(), remote);
|
||||
Assert.assertEquals(Status.OK, status);
|
||||
File localFile2 = File.createTempFile("brokerunittest", ".dat");
|
||||
localFile2.deleteOnExit();
|
||||
status = storage.downloadWithFileSize(remote, localFile2.getAbsolutePath(), 1024 * 1024);
|
||||
status = fileSystem.downloadWithFileSize(remote, localFile2.getAbsolutePath(), 1024 * 1024);
|
||||
Assert.assertEquals(Status.OK, status);
|
||||
Assert.assertEquals(DigestUtils.md5Hex(new FileInputStream(localFile)),
|
||||
DigestUtils.md5Hex(new FileInputStream(localFile2)));
|
||||
@ -153,37 +154,37 @@ public class BrokerStorageTest {
|
||||
|
||||
@Test
|
||||
public void rename() {
|
||||
Assert.assertEquals(Status.OK, storage.directUpload(content, testFile + ".bak"));
|
||||
storage.rename(testFile + ".bak", testFile + ".bak1");
|
||||
Assert.assertEquals(Status.OK, storage.checkPathExist(testFile + ".bak1"));
|
||||
Assert.assertEquals(Status.OK, fileSystem.directUpload(content, testFile + ".bak"));
|
||||
fileSystem.rename(testFile + ".bak", testFile + ".bak1");
|
||||
Assert.assertEquals(Status.OK, fileSystem.exists(testFile + ".bak1"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void delete() {
|
||||
String deleteFile = testFile + ".to_be_delete";
|
||||
Assert.assertEquals(Status.OK, storage.delete(deleteFile + "xxxx"));
|
||||
Assert.assertEquals(Status.OK, storage.directUpload(content, deleteFile));
|
||||
Assert.assertEquals(Status.OK, storage.delete(deleteFile));
|
||||
Assert.assertEquals(Status.ErrCode.NOT_FOUND, storage.checkPathExist(deleteFile).getErrCode());
|
||||
Assert.assertEquals(Status.OK, storage.delete(deleteFile + "xxxx"));
|
||||
Assert.assertEquals(Status.OK, fileSystem.delete(deleteFile + "xxxx"));
|
||||
Assert.assertEquals(Status.OK, fileSystem.directUpload(content, deleteFile));
|
||||
Assert.assertEquals(Status.OK, fileSystem.delete(deleteFile));
|
||||
Assert.assertEquals(Status.ErrCode.NOT_FOUND, fileSystem.exists(deleteFile).getErrCode());
|
||||
Assert.assertEquals(Status.OK, fileSystem.delete(deleteFile + "xxxx"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void list() {
|
||||
List<RemoteFile> result = new ArrayList<>();
|
||||
String listPath = bucket + basePath + "_list" + "/Ode_to_the_West_Wind";
|
||||
Assert.assertEquals(Status.OK, storage.directUpload(content, listPath + ".1"));
|
||||
Assert.assertEquals(Status.OK, storage.directUpload(content, listPath + ".2"));
|
||||
Assert.assertEquals(Status.OK, storage.directUpload(content, listPath + ".3"));
|
||||
Assert.assertEquals(Status.OK, storage.list(bucket + basePath + "_list/*", result));
|
||||
Assert.assertEquals(Status.OK, fileSystem.directUpload(content, listPath + ".1"));
|
||||
Assert.assertEquals(Status.OK, fileSystem.directUpload(content, listPath + ".2"));
|
||||
Assert.assertEquals(Status.OK, fileSystem.directUpload(content, listPath + ".3"));
|
||||
Assert.assertEquals(Status.OK, fileSystem.list(bucket + basePath + "_list/*", result));
|
||||
Assert.assertEquals(3, result.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void checkPathExist() {
|
||||
Status status = storage.checkPathExist(testFile);
|
||||
public void exists() {
|
||||
Status status = fileSystem.exists(testFile);
|
||||
Assert.assertEquals(Status.OK, status);
|
||||
status = storage.checkPathExist(testFile + ".NOT_EXIST");
|
||||
status = fileSystem.exists(testFile + ".NOT_EXIST");
|
||||
Assert.assertEquals(Status.ErrCode.NOT_FOUND, status.getErrCode());
|
||||
}
|
||||
}
|
||||
|
||||
@ -22,7 +22,9 @@ 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.obj.BlobStorage;
|
||||
import org.apache.doris.fs.FileSystemFactory;
|
||||
import org.apache.doris.fs.remote.RemoteFile;
|
||||
import org.apache.doris.fs.remote.RemoteFileSystem;
|
||||
import org.apache.doris.service.FrontendOptions;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
@ -39,13 +41,12 @@ import org.junit.Test;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintWriter;
|
||||
import java.net.MalformedURLException;
|
||||
import java.net.URISyntaxException;
|
||||
import java.nio.file.Files;
|
||||
import java.sql.Timestamp;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@ -62,7 +63,7 @@ public class RepositoryTest {
|
||||
private SnapshotInfo info;
|
||||
|
||||
@Mocked
|
||||
private BlobStorage storage;
|
||||
private RemoteFileSystem fileSystem;
|
||||
|
||||
|
||||
@Before
|
||||
@ -91,7 +92,7 @@ public class RepositoryTest {
|
||||
|
||||
@Test
|
||||
public void testGet() {
|
||||
repo = new Repository(10000, "repo", false, location, storage);
|
||||
repo = new Repository(10000, "repo", false, location, fileSystem);
|
||||
|
||||
Assert.assertEquals(repoId, repo.getId());
|
||||
Assert.assertEquals(name, repo.getName());
|
||||
@ -105,7 +106,7 @@ public class RepositoryTest {
|
||||
public void testInit() {
|
||||
new Expectations() {
|
||||
{
|
||||
storage.list(anyString, (List<RemoteFile>) any);
|
||||
fileSystem.list(anyString, (List<RemoteFile>) any);
|
||||
minTimes = 0;
|
||||
result = new Delegate<Status>() {
|
||||
public Status list(String remotePath, List<RemoteFile> result) {
|
||||
@ -114,13 +115,13 @@ public class RepositoryTest {
|
||||
}
|
||||
};
|
||||
|
||||
storage.directUpload(anyString, anyString);
|
||||
fileSystem.directUpload(anyString, anyString);
|
||||
minTimes = 0;
|
||||
result = Status.OK;
|
||||
}
|
||||
};
|
||||
|
||||
repo = new Repository(10000, "repo", false, location, storage);
|
||||
repo = new Repository(10000, "repo", false, location, fileSystem);
|
||||
|
||||
Status st = repo.initRepository();
|
||||
System.out.println(st);
|
||||
@ -129,7 +130,7 @@ public class RepositoryTest {
|
||||
|
||||
@Test
|
||||
public void testassemnblePath() throws MalformedURLException, URISyntaxException {
|
||||
repo = new Repository(10000, "repo", false, location, storage);
|
||||
repo = new Repository(10000, "repo", false, location, fileSystem);
|
||||
|
||||
// job info
|
||||
String label = "label";
|
||||
@ -164,13 +165,13 @@ public class RepositoryTest {
|
||||
public void testPing() {
|
||||
new Expectations() {
|
||||
{
|
||||
storage.checkPathExist(anyString);
|
||||
fileSystem.exists(anyString);
|
||||
minTimes = 0;
|
||||
result = Status.OK;
|
||||
}
|
||||
};
|
||||
|
||||
repo = new Repository(10000, "repo", false, location, storage);
|
||||
repo = new Repository(10000, "repo", false, location, fileSystem);
|
||||
Assert.assertTrue(repo.ping());
|
||||
Assert.assertTrue(repo.getErrorMsg() == null);
|
||||
}
|
||||
@ -179,7 +180,7 @@ public class RepositoryTest {
|
||||
public void testListSnapshots() {
|
||||
new Expectations() {
|
||||
{
|
||||
storage.list(anyString, (List<RemoteFile>) any);
|
||||
fileSystem.list(anyString, (List<RemoteFile>) any);
|
||||
minTimes = 0;
|
||||
result = new Delegate() {
|
||||
public Status list(String remotePath, List<RemoteFile> result) {
|
||||
@ -191,7 +192,7 @@ public class RepositoryTest {
|
||||
}
|
||||
};
|
||||
|
||||
repo = new Repository(10000, "repo", false, location, storage);
|
||||
repo = new Repository(10000, "repo", false, location, fileSystem);
|
||||
List<String> snapshotNames = Lists.newArrayList();
|
||||
Status st = repo.listSnapshots(snapshotNames);
|
||||
Assert.assertTrue(st.ok());
|
||||
@ -203,21 +204,21 @@ public class RepositoryTest {
|
||||
public void testUpload() {
|
||||
new Expectations() {
|
||||
{
|
||||
storage.upload(anyString, anyString);
|
||||
fileSystem.upload(anyString, anyString);
|
||||
minTimes = 0;
|
||||
result = Status.OK;
|
||||
|
||||
storage.rename(anyString, anyString);
|
||||
fileSystem.rename(anyString, anyString);
|
||||
minTimes = 0;
|
||||
result = Status.OK;
|
||||
|
||||
storage.delete(anyString);
|
||||
fileSystem.delete(anyString);
|
||||
minTimes = 0;
|
||||
result = Status.OK;
|
||||
}
|
||||
};
|
||||
|
||||
repo = new Repository(10000, "repo", false, location, storage);
|
||||
repo = new Repository(10000, "repo", false, location, fileSystem);
|
||||
String localFilePath = "./tmp_" + System.currentTimeMillis();
|
||||
try (PrintWriter out = new PrintWriter(localFilePath)) {
|
||||
out.print("a");
|
||||
@ -249,7 +250,7 @@ public class RepositoryTest {
|
||||
|
||||
new Expectations() {
|
||||
{
|
||||
storage.list(anyString, (List<RemoteFile>) any);
|
||||
fileSystem.list(anyString, (List<RemoteFile>) any);
|
||||
minTimes = 0;
|
||||
result = new Delegate() {
|
||||
public Status list(String remotePath, List<RemoteFile> result) {
|
||||
@ -258,13 +259,13 @@ public class RepositoryTest {
|
||||
}
|
||||
};
|
||||
|
||||
storage.downloadWithFileSize(anyString, anyString, anyLong);
|
||||
fileSystem.downloadWithFileSize(anyString, anyString, anyLong);
|
||||
minTimes = 0;
|
||||
result = Status.OK;
|
||||
}
|
||||
};
|
||||
|
||||
repo = new Repository(10000, "repo", false, location, storage);
|
||||
repo = new Repository(10000, "repo", false, location, fileSystem);
|
||||
String remoteFilePath = location + "/remote_file";
|
||||
Status st = repo.download(remoteFilePath, localFilePath);
|
||||
Assert.assertTrue(st.ok());
|
||||
@ -275,7 +276,7 @@ public class RepositoryTest {
|
||||
|
||||
@Test
|
||||
public void testGetInfo() {
|
||||
repo = new Repository(10000, "repo", false, location, storage);
|
||||
repo = new Repository(10000, "repo", false, location, fileSystem);
|
||||
List<String> infos = repo.getInfo();
|
||||
Assert.assertTrue(infos.size() == ShowRepositoriesStmt.TITLE_NAMES.size());
|
||||
}
|
||||
@ -284,7 +285,7 @@ public class RepositoryTest {
|
||||
public void testGetSnapshotInfo() {
|
||||
new Expectations() {
|
||||
{
|
||||
storage.list(anyString, (List<RemoteFile>) any);
|
||||
fileSystem.list(anyString, (List<RemoteFile>) any);
|
||||
minTimes = 0;
|
||||
result = new Delegate() {
|
||||
public Status list(String remotePath, List<RemoteFile> result) {
|
||||
@ -303,7 +304,7 @@ public class RepositoryTest {
|
||||
}
|
||||
};
|
||||
|
||||
repo = new Repository(10000, "repo", false, location, storage);
|
||||
repo = new Repository(10000, "repo", false, location, fileSystem);
|
||||
String snapshotName = "";
|
||||
String timestamp = "";
|
||||
try {
|
||||
@ -322,17 +323,17 @@ public class RepositoryTest {
|
||||
properties.put("bos_endpoint", "http://gz.bcebos.com");
|
||||
properties.put("bos_accesskey", "a");
|
||||
properties.put("bos_secret_accesskey", "b");
|
||||
BlobStorage storage = BlobStorage.create(brokerName, StorageBackend.StorageType.BROKER, properties);
|
||||
repo = new Repository(10000, "repo", false, location, storage);
|
||||
RemoteFileSystem fs = FileSystemFactory.get(brokerName, StorageBackend.StorageType.BROKER, properties);
|
||||
repo = new Repository(10000, "repo", false, location, fs);
|
||||
|
||||
File file = new File("./Repository");
|
||||
try {
|
||||
DataOutputStream out = new DataOutputStream(new FileOutputStream(file));
|
||||
DataOutputStream out = new DataOutputStream(Files.newOutputStream(file.toPath()));
|
||||
repo.write(out);
|
||||
out.flush();
|
||||
out.close();
|
||||
|
||||
DataInputStream in = new DataInputStream(new FileInputStream(file));
|
||||
DataInputStream in = new DataInputStream(Files.newInputStream(file.toPath()));
|
||||
Repository newRepo = Repository.read(in);
|
||||
in.close();
|
||||
|
||||
@ -352,7 +353,7 @@ public class RepositoryTest {
|
||||
@Test
|
||||
public void testPathNormalize() {
|
||||
String newLoc = "bos://cmy_bucket/bos_repo/";
|
||||
repo = new Repository(10000, "repo", false, newLoc, storage);
|
||||
repo = new Repository(10000, "repo", false, newLoc, fileSystem);
|
||||
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);
|
||||
|
||||
@ -360,7 +361,7 @@ public class RepositoryTest {
|
||||
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, storage);
|
||||
repo = new Repository(10000, "repo", false, newLoc, fileSystem);
|
||||
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);
|
||||
|
||||
@ -37,7 +37,7 @@ import org.apache.doris.common.FeConstants;
|
||||
import org.apache.doris.common.MarkedCountDownLatch;
|
||||
import org.apache.doris.common.jmockit.Deencapsulation;
|
||||
import org.apache.doris.datasource.InternalCatalog;
|
||||
import org.apache.doris.fs.obj.BlobStorage;
|
||||
import org.apache.doris.fs.FileSystemFactory;
|
||||
import org.apache.doris.persist.EditLog;
|
||||
import org.apache.doris.system.SystemInfoService;
|
||||
import org.apache.doris.thrift.TStorageMedium;
|
||||
@ -111,7 +111,7 @@ public class RestoreJobTest {
|
||||
|
||||
@Injectable
|
||||
private Repository repo = new Repository(repoId, "repo", false, "bos://my_repo",
|
||||
BlobStorage.create("broker", StorageBackend.StorageType.BROKER, Maps.newHashMap()));
|
||||
FileSystemFactory.get("broker", StorageBackend.StorageType.BROKER, Maps.newHashMap()));
|
||||
|
||||
private BackupMeta backupMeta;
|
||||
|
||||
|
||||
@ -148,7 +148,7 @@ public class PropertyConverterTest extends TestWithFeService {
|
||||
CreateRepositoryStmt analyzedStmt = createStmt(s3Repo);
|
||||
Assertions.assertEquals(analyzedStmt.getProperties().size(), 4);
|
||||
Repository repository = getRepository(analyzedStmt, "s3_repo");
|
||||
Assertions.assertEquals(repository.getStorage().getProperties().size(), 5);
|
||||
Assertions.assertEquals(repository.getRemoteFileSystem().getProperties().size(), 5);
|
||||
|
||||
String s3RepoNew = "CREATE REPOSITORY `s3_repo_new`\n"
|
||||
+ "WITH S3\n"
|
||||
@ -162,7 +162,7 @@ public class PropertyConverterTest extends TestWithFeService {
|
||||
CreateRepositoryStmt analyzedStmtNew = createStmt(s3RepoNew);
|
||||
Assertions.assertEquals(analyzedStmtNew.getProperties().size(), 3);
|
||||
Repository repositoryNew = getRepository(analyzedStmtNew, "s3_repo_new");
|
||||
Assertions.assertEquals(repositoryNew.getStorage().getProperties().size(), 4);
|
||||
Assertions.assertEquals(repositoryNew.getRemoteFileSystem().getProperties().size(), 4);
|
||||
}
|
||||
|
||||
private static Repository getRepository(CreateRepositoryStmt analyzedStmt, String name) throws DdlException {
|
||||
@ -190,7 +190,7 @@ public class PropertyConverterTest extends TestWithFeService {
|
||||
Env.getCurrentEnv().getBrokerMgr().addBrokers("bos_broker", brokers);
|
||||
|
||||
Repository repositoryNew = getRepository(analyzedStmt, "bos_broker_repo");
|
||||
Assertions.assertEquals(repositoryNew.getStorage().getProperties().size(), 4);
|
||||
Assertions.assertEquals(repositoryNew.getRemoteFileSystem().getProperties().size(), 4);
|
||||
}
|
||||
|
||||
@Test
|
||||
@ -237,7 +237,7 @@ public class PropertyConverterTest extends TestWithFeService {
|
||||
CreateCatalogStmt analyzedStmt = createStmt(queryOld);
|
||||
HMSExternalCatalog catalog = createAndGetCatalog(analyzedStmt, "hms_s3_old");
|
||||
Map<String, String> properties = catalog.getCatalogProperty().getProperties();
|
||||
Assertions.assertEquals(properties.size(), 7);
|
||||
Assertions.assertEquals(properties.size(), 11);
|
||||
|
||||
Map<String, String> hdProps = catalog.getCatalogProperty().getHadoopProperties();
|
||||
Assertions.assertEquals(hdProps.size(), 20);
|
||||
|
||||
@ -0,0 +1,131 @@
|
||||
// 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.fs.obj;
|
||||
|
||||
import software.amazon.awssdk.awscore.exception.AwsServiceException;
|
||||
import software.amazon.awssdk.core.exception.SdkClientException;
|
||||
import software.amazon.awssdk.core.sync.RequestBody;
|
||||
import software.amazon.awssdk.services.s3.S3Client;
|
||||
import software.amazon.awssdk.services.s3.model.CopyObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.CopyObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.CopyObjectResult;
|
||||
import software.amazon.awssdk.services.s3.model.DeleteObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.DeleteObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.GetObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.InvalidObjectStateException;
|
||||
import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
|
||||
import software.amazon.awssdk.services.s3.model.ListObjectsV2Response;
|
||||
import software.amazon.awssdk.services.s3.model.NoSuchBucketException;
|
||||
import software.amazon.awssdk.services.s3.model.NoSuchKeyException;
|
||||
import software.amazon.awssdk.services.s3.model.ObjectNotInActiveTierErrorException;
|
||||
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
|
||||
import software.amazon.awssdk.services.s3.model.PutObjectResponse;
|
||||
import software.amazon.awssdk.services.s3.model.S3Exception;
|
||||
import software.amazon.awssdk.services.s3.model.S3Object;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* use for unit test
|
||||
*/
|
||||
public class MockedS3Client implements S3Client {
|
||||
|
||||
private byte[] mockedData;
|
||||
private boolean canMakeData;
|
||||
private final List<S3Object> mockedObjectList = new ArrayList<>();
|
||||
|
||||
@Override
|
||||
public String serviceName() {
|
||||
return "MockedS3";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {}
|
||||
|
||||
public void setMockedData(byte[] mockedData) {
|
||||
this.mockedData = mockedData;
|
||||
}
|
||||
|
||||
public void setCanMakeData(boolean canMakeData) {
|
||||
this.canMakeData = canMakeData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HeadObjectResponse headObject(HeadObjectRequest headObjectRequest) throws NoSuchKeyException,
|
||||
AwsServiceException, SdkClientException, S3Exception {
|
||||
return HeadObjectResponse.builder().deleteMarker(false).eTag("head-tag").build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public GetObjectResponse getObject(GetObjectRequest getObjectRequest, Path destinationPath) throws
|
||||
NoSuchKeyException, InvalidObjectStateException,
|
||||
AwsServiceException, SdkClientException, S3Exception {
|
||||
if (canMakeData) {
|
||||
try (OutputStream os = Files.newOutputStream(destinationPath)) {
|
||||
os.write(mockedData);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
return GetObjectResponse.builder().eTag("get-etag").build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public PutObjectResponse putObject(PutObjectRequest putObjectRequest, RequestBody requestBody)
|
||||
throws AwsServiceException, SdkClientException, S3Exception {
|
||||
Long size = requestBody.optionalContentLength().orElse(0L);
|
||||
mockedObjectList.add(S3Object.builder().key(putObjectRequest.key()).size(size).build());
|
||||
return PutObjectResponse.builder().eTag("put-etag").build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DeleteObjectResponse deleteObject(DeleteObjectRequest deleteObjectRequest) throws AwsServiceException,
|
||||
SdkClientException, S3Exception {
|
||||
mockedObjectList.removeIf(e -> Objects.equals(e.key(), deleteObjectRequest.key()));
|
||||
return DeleteObjectResponse.builder().deleteMarker(true).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CopyObjectResponse copyObject(CopyObjectRequest copyObjectRequest) throws ObjectNotInActiveTierErrorException,
|
||||
AwsServiceException, SdkClientException, S3Exception {
|
||||
CopyObjectResult result = CopyObjectResult.builder().eTag("copy-etag").build();
|
||||
return CopyObjectResponse.builder().copyObjectResult(result).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListObjectsV2Response listObjectsV2(ListObjectsV2Request listObjectsV2Request) throws NoSuchBucketException,
|
||||
AwsServiceException, SdkClientException, S3Exception {
|
||||
return ListObjectsV2Response.builder()
|
||||
.contents(mockedObjectList)
|
||||
.isTruncated(true)
|
||||
.delimiter(",")
|
||||
.nextContinuationToken("next-token")
|
||||
.prefix("prefix")
|
||||
.maxKeys(5)
|
||||
.build();
|
||||
}
|
||||
}
|
||||
@ -17,25 +17,31 @@
|
||||
|
||||
package org.apache.doris.fs.obj;
|
||||
|
||||
import org.apache.doris.analysis.StorageBackend;
|
||||
import org.apache.doris.backup.RemoteFile;
|
||||
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 mockit.Mock;
|
||||
import mockit.MockUp;
|
||||
import org.apache.commons.codec.digest.DigestUtils;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import software.amazon.awssdk.services.s3.S3Client;
|
||||
import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
|
||||
import software.amazon.awssdk.services.s3.model.ListObjectsV2Response;
|
||||
import software.amazon.awssdk.services.s3.model.S3Object;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.file.Files;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
@ -43,32 +49,30 @@ import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
|
||||
@Ignore
|
||||
public class S3FileSystemTest {
|
||||
private static String basePath;
|
||||
private final String bucket = "s3://doris-test/";
|
||||
private Map<String, String> properties;
|
||||
private S3FileSystem fileSystem;
|
||||
private MockedS3Client mockedClient;
|
||||
private String testFile;
|
||||
private String content;
|
||||
// we use mocked s3 client to test s3 file system by default.
|
||||
private boolean injectMockedClient = true;
|
||||
|
||||
@BeforeClass
|
||||
@BeforeAll
|
||||
public static void init() {
|
||||
basePath = "s3/" + UUID.randomUUID().toString();
|
||||
basePath = "s3/" + UUID.randomUUID();
|
||||
}
|
||||
|
||||
@Before
|
||||
@BeforeEach
|
||||
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(PropertyConverter.USE_PATH_STYLE, "false");
|
||||
|
||||
properties.put("AWS_REGION", "bj");
|
||||
fileSystem = (S3FileSystem) FileSystemFactory.get(StorageBackend.StorageType.S3, properties);
|
||||
testFile = bucket + basePath + "/Ode_to_the_West_Wind";
|
||||
|
||||
content =
|
||||
"O wild West Wind, thou breath of Autumn's being\n"
|
||||
+ "Thou, from whose unseen presence the leaves dead\n"
|
||||
@ -84,7 +88,46 @@ public class S3FileSystemTest {
|
||||
+ "With living hues and odors plain and hill:\n"
|
||||
+ "Wild Spirit, which art moving everywhere;\n"
|
||||
+ "Destroyer and preserver; hear, oh, hear!";
|
||||
Assert.assertEquals(Status.OK, fileSystem.directUpload(content, testFile));
|
||||
|
||||
if (injectMockedClient) {
|
||||
properties.put("AWS_ACCESS_KEY", "ak");
|
||||
properties.put("AWS_SECRET_KEY", "sk");
|
||||
// create storage
|
||||
mockedClient = new MockedS3Client();
|
||||
mockedClient.setCanMakeData(true);
|
||||
mockedClient.setMockedData(content.getBytes());
|
||||
new MockUp<S3ObjStorage>(S3ObjStorage.class) {
|
||||
@Mock
|
||||
public S3Client getClient(String bucket) throws UserException {
|
||||
return mockedClient;
|
||||
}
|
||||
};
|
||||
S3ObjStorage mockedStorage = new S3ObjStorage(properties);
|
||||
Assertions.assertTrue(mockedStorage.getClient("mocked") instanceof MockedS3Client);
|
||||
// inject storage to file system.
|
||||
fileSystem = new S3FileSystem(mockedStorage);
|
||||
new MockUp<S3FileSystem>(S3FileSystem.class) {
|
||||
@Mock
|
||||
public Status list(String remotePath, List<RemoteFile> result, boolean fileNameOnly) {
|
||||
try {
|
||||
S3URI uri = S3URI.create(remotePath, false);
|
||||
ListObjectsV2Request.Builder requestBuilder = ListObjectsV2Request.builder().bucket(uri.getBucket());
|
||||
ListObjectsV2Response response = mockedClient.listObjectsV2(requestBuilder.build());
|
||||
for (S3Object c : response.contents()) {
|
||||
result.add(new RemoteFile(c.key(), true, c.size(), 0));
|
||||
}
|
||||
} catch (UserException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return Status.OK;
|
||||
}
|
||||
};
|
||||
} else {
|
||||
// can also real file system to test.
|
||||
fileSystem = (S3FileSystem) FileSystemFactory.getS3FileSystem(properties);
|
||||
}
|
||||
testFile = bucket + basePath + "/Ode_to_the_West_Wind";
|
||||
Assertions.assertEquals(Status.OK, fileSystem.directUpload(content, testFile));
|
||||
}
|
||||
|
||||
@Test
|
||||
@ -92,83 +135,108 @@ public class S3FileSystemTest {
|
||||
File localFile = File.createTempFile("s3unittest", ".dat");
|
||||
localFile.deleteOnExit();
|
||||
Status status = fileSystem.downloadWithFileSize(testFile, localFile.getAbsolutePath(), content.getBytes().length);
|
||||
Assert.assertEquals(Status.OK, status);
|
||||
Assert.assertEquals(DigestUtils.md5Hex(content.getBytes()), DigestUtils.md5Hex(new FileInputStream(localFile)));
|
||||
Assertions.assertEquals(Status.OK, status);
|
||||
Assertions.assertEquals(DigestUtils.md5Hex(content.getBytes()),
|
||||
DigestUtils.md5Hex(Files.newInputStream(localFile.toPath())));
|
||||
status = fileSystem.downloadWithFileSize(bucket + basePath + "/Ode_to_the_West_Wind", localFile.getAbsolutePath(), content.getBytes().length + 1);
|
||||
Assert.assertNotEquals(Status.OK, status);
|
||||
Assertions.assertNotEquals(Status.OK, status);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void upload() throws IOException {
|
||||
File localFile = File.createTempFile("s3unittest", ".dat");
|
||||
localFile.deleteOnExit();
|
||||
OutputStream os = new FileOutputStream(localFile);
|
||||
OutputStream os = Files.newOutputStream(localFile.toPath());
|
||||
byte[] buf = new byte[1024 * 1024];
|
||||
Random r = new Random();
|
||||
r.nextBytes(buf);
|
||||
os.write(buf);
|
||||
os.close();
|
||||
if (injectMockedClient) {
|
||||
mockedClient.setMockedData(buf);
|
||||
}
|
||||
String remote = bucket + basePath + "/" + localFile.getName();
|
||||
Status status = fileSystem.upload(localFile.getAbsolutePath(), remote);
|
||||
Assert.assertEquals(Status.OK, status);
|
||||
Assertions.assertEquals(Status.OK, status);
|
||||
File localFile2 = File.createTempFile("s3unittest", ".dat");
|
||||
localFile2.deleteOnExit();
|
||||
status = fileSystem.downloadWithFileSize(remote, localFile2.getAbsolutePath(), 1024 * 1024);
|
||||
Assert.assertEquals(Status.OK, status);
|
||||
Assert.assertEquals(DigestUtils.md5Hex(new FileInputStream(localFile)),
|
||||
DigestUtils.md5Hex(new FileInputStream(localFile2)));
|
||||
Assertions.assertEquals(Status.OK, status);
|
||||
Assertions.assertEquals(DigestUtils.md5Hex(Files.newInputStream(localFile.toPath())),
|
||||
DigestUtils.md5Hex(Files.newInputStream(localFile2.toPath())));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRepositoryUpload() throws IOException {
|
||||
Repository repo = new Repository(10000, "repo", false, bucket + basePath, fileSystem);
|
||||
File localFile = File.createTempFile("s3unittest", ".dat");
|
||||
localFile.deleteOnExit();
|
||||
String remote = bucket + basePath + "/" + localFile.getName();
|
||||
Status status = repo.upload(localFile.getAbsolutePath(), remote);
|
||||
Assertions.assertEquals(Status.OK, status);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void copy() {
|
||||
Assert.assertEquals(Status.OK, fileSystem.copy(testFile, testFile + ".bak"));
|
||||
Assert.assertEquals(Status.OK, fileSystem.exists(testFile + ".bak"));
|
||||
Assert.assertNotEquals(Status.OK, fileSystem.copy(testFile + ".bakxxx", testFile + ".bak"));
|
||||
Assertions.assertEquals(Status.OK, fileSystem.copy(testFile, testFile + ".bak"));
|
||||
Assertions.assertEquals(Status.OK, fileSystem.exists(testFile + ".bak"));
|
||||
if (!injectMockedClient) {
|
||||
Assertions.assertNotEquals(Status.OK, fileSystem.copy(testFile + ".bakxxx", testFile + ".bak"));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void rename() {
|
||||
Assert.assertEquals(Status.OK, fileSystem.directUpload(content, testFile + ".bak"));
|
||||
Assertions.assertEquals(Status.OK, fileSystem.directUpload(content, testFile + ".bak"));
|
||||
fileSystem.rename(testFile + ".bak", testFile + ".bak1");
|
||||
Assert.assertEquals(Status.ErrCode.NOT_FOUND, fileSystem.exists(testFile + ".bak").getErrCode());
|
||||
Assert.assertEquals(Status.OK, fileSystem.exists(testFile + ".bak1"));
|
||||
if (!injectMockedClient) {
|
||||
Assertions.assertEquals(Status.ErrCode.NOT_FOUND, fileSystem.exists(testFile + ".bak").getErrCode());
|
||||
}
|
||||
Assertions.assertEquals(Status.OK, fileSystem.exists(testFile + ".bak1"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void delete() {
|
||||
String deleteFile = testFile + ".to_be_delete";
|
||||
Assert.assertEquals(Status.OK, fileSystem.directUpload(content, deleteFile));
|
||||
Assert.assertEquals(Status.OK, fileSystem.delete(deleteFile));
|
||||
Assert.assertEquals(Status.ErrCode.NOT_FOUND, fileSystem.exists(deleteFile).getErrCode());
|
||||
Assert.assertEquals(Status.OK, fileSystem.delete(deleteFile + "xxxx"));
|
||||
public void checkPathExist() {
|
||||
Status status = fileSystem.exists(testFile);
|
||||
Assertions.assertEquals(Status.OK, status);
|
||||
status = fileSystem.exists(testFile + ".NOT_EXIST");
|
||||
if (!injectMockedClient) {
|
||||
Assertions.assertEquals(Status.ErrCode.NOT_FOUND, status.getErrCode());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void makeDir() {
|
||||
String path = bucket + basePath + "/test_path";
|
||||
Assertions.assertEquals(Status.OK, fileSystem.makeDir(path));
|
||||
if (!injectMockedClient) {
|
||||
Assertions.assertNotEquals(Status.OK, fileSystem.exists(path));
|
||||
}
|
||||
String path1 = bucket + basePath + "/test_path1/";
|
||||
Assertions.assertEquals(Status.OK, fileSystem.makeDir(path1));
|
||||
Assertions.assertEquals(Status.OK, fileSystem.exists(path1));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void list() {
|
||||
List<RemoteFile> result = new ArrayList<>();
|
||||
String listPath = bucket + basePath + "_list" + "/Ode_to_the_West_Wind";
|
||||
Assert.assertEquals(Status.OK, fileSystem.directUpload(content, listPath + ".1"));
|
||||
Assert.assertEquals(Status.OK, fileSystem.directUpload(content, listPath + ".2"));
|
||||
Assert.assertEquals(Status.OK, fileSystem.directUpload(content, listPath + ".3"));
|
||||
Assert.assertEquals(Status.OK, fileSystem.list(bucket + basePath + "_list/*", result));
|
||||
Assert.assertEquals(3, result.size());
|
||||
Assertions.assertEquals(Status.OK, fileSystem.delete(testFile));
|
||||
Assertions.assertEquals(Status.OK, fileSystem.directUpload(content, listPath + ".1"));
|
||||
Assertions.assertEquals(Status.OK, fileSystem.directUpload(content, listPath + ".2"));
|
||||
Assertions.assertEquals(Status.OK, fileSystem.directUpload(content, listPath + ".3"));
|
||||
Assertions.assertEquals(Status.OK, fileSystem.list(bucket + basePath + "_list/*", result));
|
||||
Assertions.assertEquals(3, result.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void makeDir() {
|
||||
String path = bucket + basePath + "/test_path";
|
||||
Assert.assertEquals(Status.OK, fileSystem.makeDir(path));
|
||||
Assert.assertNotEquals(Status.OK, fileSystem.exists(path));
|
||||
String path1 = bucket + basePath + "/test_path1/";
|
||||
Assert.assertEquals(Status.OK, fileSystem.makeDir(path1));
|
||||
Assert.assertEquals(Status.OK, fileSystem.exists(path1));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void checkPathExist() {
|
||||
Status status = fileSystem.exists(testFile);
|
||||
Assert.assertEquals(Status.OK, status);
|
||||
status = fileSystem.exists(testFile + ".NOT_EXIST");
|
||||
Assert.assertEquals(Status.ErrCode.NOT_FOUND, status.getErrCode());
|
||||
public void delete() {
|
||||
String deleteFile = testFile + ".to_be_delete";
|
||||
Assertions.assertEquals(Status.OK, fileSystem.directUpload(content, deleteFile));
|
||||
Assertions.assertEquals(Status.OK, fileSystem.delete(deleteFile));
|
||||
if (!injectMockedClient) {
|
||||
Assertions.assertEquals(Status.ErrCode.NOT_FOUND, fileSystem.exists(deleteFile).getErrCode());
|
||||
}
|
||||
Assertions.assertEquals(Status.OK, fileSystem.delete(deleteFile + "xxxx"));
|
||||
}
|
||||
}
|
||||
|
||||
@ -0,0 +1,104 @@
|
||||
// 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.fs.obj;
|
||||
|
||||
import org.apache.doris.backup.Status;
|
||||
import org.apache.doris.common.UserException;
|
||||
import org.apache.doris.common.util.S3URI;
|
||||
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.TestInstance;
|
||||
import software.amazon.awssdk.core.sync.RequestBody;
|
||||
|
||||
import java.io.File;
|
||||
import java.lang.reflect.Field;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
||||
@TestInstance(TestInstance.Lifecycle.PER_CLASS)
|
||||
class S3ObjStorageTest {
|
||||
private S3ObjStorage storage;
|
||||
|
||||
private MockedS3Client mockedClient;
|
||||
|
||||
@BeforeAll
|
||||
public void beforeAll() throws Exception {
|
||||
Map<String, String> properties = new HashMap<>();
|
||||
properties.put("s3.endpoint", "s3.e.c");
|
||||
properties.put("s3.access_key", "abc");
|
||||
properties.put("s3.secret_key", "123");
|
||||
storage = new S3ObjStorage(properties);
|
||||
Field client = storage.getClass().getDeclaredField("client");
|
||||
client.setAccessible(true);
|
||||
mockedClient = new MockedS3Client();
|
||||
client.set(storage, mockedClient);
|
||||
Assertions.assertTrue(storage.getClient("mocked") instanceof MockedS3Client);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBaseOp() throws UserException {
|
||||
S3URI vUri = S3URI.create("s3://bucket/key", true);
|
||||
S3URI uri = S3URI.create("s3://bucket/key", false);
|
||||
Assertions.assertEquals(vUri.getVirtualBucket(), "bucket");
|
||||
Assertions.assertEquals(vUri.getBucket(), "key");
|
||||
Assertions.assertEquals(uri.getVirtualBucket(), "");
|
||||
Assertions.assertEquals(uri.getBucket(), "bucket");
|
||||
|
||||
Status st = storage.headObject("s3://bucket/key");
|
||||
Assertions.assertEquals(Status.OK, st);
|
||||
|
||||
mockedClient.setMockedData(new byte[0]);
|
||||
st = storage.getObject("s3://bucket/key", new File("/mocked/file"));
|
||||
Assertions.assertEquals(Status.OK, st);
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
st = storage.putObject("s3://bucket/keys/key" + i, RequestBody.fromString("mocked"));
|
||||
Assertions.assertEquals(Status.OK, st);
|
||||
}
|
||||
st = storage.copyObject("s3://bucket/key", "s3://bucket/key1");
|
||||
Assertions.assertEquals(Status.OK, st);
|
||||
|
||||
st = storage.deleteObject("s3://bucket/key");
|
||||
Assertions.assertEquals(Status.OK, st);
|
||||
|
||||
RemoteObjects remoteObjects = storage.listObjects("s3://bucket/keys", null);
|
||||
Assertions.assertEquals(5, remoteObjects.getObjectList().size());
|
||||
Assertions.assertTrue(remoteObjects.isTruncated());
|
||||
Assertions.assertEquals("next-token", remoteObjects.getContinuationToken());
|
||||
|
||||
List<RemoteObject> objectList = remoteObjects.getObjectList();
|
||||
for (int i = 0; i < objectList.size(); i++) {
|
||||
RemoteObject remoteObject = objectList.get(i);
|
||||
Assertions.assertEquals("key" + i, remoteObject.getRelativePath());
|
||||
}
|
||||
|
||||
storage.properties.put("use_path_style", "false");
|
||||
storage.properties.put("s3.endpoint", "oss.a.c");
|
||||
storage.setProperties(storage.properties);
|
||||
RemoteObjects remoteObjectsVBucket = storage.listObjects("oss://bucket/keys", null);
|
||||
List<RemoteObject> list = remoteObjectsVBucket.getObjectList();
|
||||
for (int i = 0; i < list.size(); i++) {
|
||||
RemoteObject remoteObject = list.get(i);
|
||||
Assertions.assertTrue(remoteObject.getRelativePath().startsWith("keys/key" + i));
|
||||
}
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user