diff --git a/cloud/aws/aws_env.cc b/cloud/aws/aws_env.cc index b57f03b52f5..61fba6dfe1e 100644 --- a/cloud/aws/aws_env.cc +++ b/cloud/aws/aws_env.cc @@ -10,25 +10,43 @@ #include #include +#include "port/port.h" #include "rocksdb/env.h" #include "rocksdb/status.h" #include "util/stderr_logger.h" #include "util/string_util.h" #ifdef USE_AWS +#include #include #endif +#include "cloud/aws/aws_env.h" #include "cloud/aws/aws_file.h" -#include "cloud/cloud_log_controller.h" #include "cloud/db_cloud_impl.h" +#include "cloud/filename.h" +#include "rocksdb/cloud/cloud_log_controller.h" +#include "rocksdb/cloud/cloud_storage_provider.h" +#include "rocksdb/utilities/object_registry.h" + +namespace { +template +bool ParseEnum(const std::unordered_map& type_map, + const std::string& type, T* value) { + auto iter = type_map.find(type); + if (iter != type_map.end()) { + *value = iter->second; + return true; + } + return false; +} +} namespace rocksdb { -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" - static const std::unordered_map AwsAccessTypeMap = { {"undefined", AwsAccessType::kUndefined}, + {"none", AwsAccessType::kDefault}, + {"default", AwsAccessType::kDefault}, {"simple", AwsAccessType::kSimple}, {"instance", AwsAccessType::kInstance}, {"EC2", AwsAccessType::kInstance}, @@ -37,16 +55,6 @@ static const std::unordered_map AwsAccessTypeMap = { {"anonymous", AwsAccessType::kAnonymous}, }; -template -bool ParseEnum(const std::unordered_map& type_map, - const std::string& type, T* value) { - auto iter = type_map.find(type); - if (iter != type_map.end()) { - *value = iter->second; - return true; - } - return false; -} AwsAccessType AwsCloudAccessCredentials::GetAccessType() const { if (type != AwsAccessType::kUndefined) { @@ -55,7 +63,10 @@ AwsAccessType AwsCloudAccessCredentials::GetAccessType() const { return AwsAccessType::kConfig; } else if (!access_key_id.empty() || !secret_key.empty()) { return AwsAccessType::kSimple; - } + } else if (getenv("AWS_ACCESS_KEY_ID") != nullptr && + getenv("AWS_SECRET_ACCESS_KEY") != nullptr) { + return AwsAccessType::kEnvironment; + } return AwsAccessType::kUndefined; } @@ -82,8 +93,10 @@ Status AwsCloudAccessCredentials::CheckCredentials( } else if (aws_type == AwsAccessType::kTaskRole) { return Status::InvalidArgument( "AWS access type: Task Role access is not supported."); + } else if (aws_type == AwsAccessType::kUndefined) { + return Status::InvalidArgument("Undefined credentials"); } - return Status::OK(); + return Status::OK(); #endif } @@ -144,7 +157,7 @@ Status AwsCloudAccessCredentials::GetCredentialsProvider( case AwsAccessType::kEnvironment: result->reset(new Aws::Auth::EnvironmentAWSCredentialsProvider()); break; - case AwsAccessType::kUndefined: + case AwsAccessType::kDefault: // Use AWS SDK's default credential chain result->reset(); break; @@ -158,242 +171,14 @@ Status AwsCloudAccessCredentials::GetCredentialsProvider( } #ifdef USE_AWS -namespace detail { - -using ScheduledJob = - std::pair>; -struct Comp { - bool operator()(const ScheduledJob& a, const ScheduledJob& b) const { - return a.first < b.first; - } -}; -struct JobHandle { - std::multiset::iterator itr; - JobHandle(std::multiset::iterator i) - : itr(std::move(i)) {} -}; - -class JobExecutor { - public: - std::shared_ptr ScheduleJob( - std::chrono::steady_clock::time_point time, - std::function callback); - void CancelJob(JobHandle* handle); - - JobExecutor(); - ~JobExecutor(); - - private: - void DoWork(); - - std::mutex mutex_; - // Notified when the earliest job to be scheduled has changed. - std::condition_variable jobs_changed_cv_; - std::multiset scheduled_jobs_; - bool shutting_down_{false}; - - std::thread thread_; -}; - -JobExecutor::JobExecutor() { - thread_ = std::thread([this]() { DoWork(); }); -} - -JobExecutor::~JobExecutor() { - { - std::lock_guard lk(mutex_); - shutting_down_ = true; - jobs_changed_cv_.notify_all(); - } - if (thread_.joinable()) { - thread_.join(); - } -} - -std::shared_ptr JobExecutor::ScheduleJob( - std::chrono::steady_clock::time_point time, - std::function callback) { - std::lock_guard lk(mutex_); - auto itr = scheduled_jobs_.emplace(time, std::move(callback)); - if (itr == scheduled_jobs_.begin()) { - jobs_changed_cv_.notify_all(); - } - return std::make_shared(itr); -} - -void JobExecutor::CancelJob(JobHandle* handle) { - std::lock_guard lk(mutex_); - if (scheduled_jobs_.begin() == handle->itr) { - jobs_changed_cv_.notify_all(); - } - scheduled_jobs_.erase(handle->itr); -} - -void JobExecutor::DoWork() { - while (true) { - std::unique_lock lk(mutex_); - if (shutting_down_) { - break; - } - if (scheduled_jobs_.empty()) { - jobs_changed_cv_.wait(lk); - continue; - } - auto earliest_job = scheduled_jobs_.begin(); - auto earliest_job_time = earliest_job->first; - if (earliest_job_time >= std::chrono::steady_clock::now()) { - jobs_changed_cv_.wait_until(lk, earliest_job_time); - continue; - } - // invoke the function - lk.unlock(); - earliest_job->second(); - lk.lock(); - scheduled_jobs_.erase(earliest_job); - } -} - -} // namespace detail - -detail::JobExecutor* GetJobExecutor() { - static detail::JobExecutor executor; - return &executor; -} - -Aws::Utils::Threading::Executor* GetAwsTransferManagerExecutor() { - static Aws::Utils::Threading::PooledThreadExecutor executor(8); - return &executor; -} - - -template -void SetEncryptionParameters(const CloudEnvOptions& cloud_env_options, - T& put_request) { - if (cloud_env_options.server_side_encryption) { - if (cloud_env_options.encryption_key_id.empty()) { - put_request.SetServerSideEncryption( - Aws::S3::Model::ServerSideEncryption::AES256); - } else { - put_request.SetServerSideEncryption( - Aws::S3::Model::ServerSideEncryption::aws_kms); - put_request.SetSSEKMSKeyId(cloud_env_options.encryption_key_id.c_str()); - } - } -} - -class CloudRequestCallbackGuard { - public: - CloudRequestCallbackGuard(CloudRequestCallback* callback, - CloudRequestOpType type, uint64_t size = 0) - : callback_(callback), type_(type), size_(size), start_(now()) {} - - ~CloudRequestCallbackGuard() { - if (callback_) { - (*callback_)(type_, size_, now() - start_, success_); - } - } - - void SetSize(uint64_t size) { size_ = size; } - void SetSuccess(bool success) { success_ = success; } - - private: - uint64_t now() { - return std::chrono::duration_cast( - std::chrono::system_clock::now() - - std::chrono::system_clock::from_time_t(0)) - .count(); - } - CloudRequestCallback* callback_; - CloudRequestOpType type_; - uint64_t size_; - bool success_{false}; - uint64_t start_; -}; - -AwsS3ClientWrapper::AwsS3ClientWrapper( - std::shared_ptr client, - std::shared_ptr cloud_request_callback) - : client_(std::move(client)), - cloud_request_callback_(std::move(cloud_request_callback)) {} - -Aws::S3::Model::ListObjectsOutcome AwsS3ClientWrapper::ListObjects( - const Aws::S3::Model::ListObjectsRequest& request) { - CloudRequestCallbackGuard t(cloud_request_callback_.get(), - CloudRequestOpType::kListOp); - auto outcome = client_->ListObjects(request); - t.SetSuccess(outcome.IsSuccess()); - return outcome; -} - -Aws::S3::Model::CreateBucketOutcome AwsS3ClientWrapper::CreateBucket( - const Aws::S3::Model::CreateBucketRequest& request) { - CloudRequestCallbackGuard t(cloud_request_callback_.get(), - CloudRequestOpType::kCreateOp); - return client_->CreateBucket(request); -} - -Aws::S3::Model::HeadBucketOutcome AwsS3ClientWrapper::HeadBucket( - const Aws::S3::Model::HeadBucketRequest& request) { - CloudRequestCallbackGuard t(cloud_request_callback_.get(), - CloudRequestOpType::kInfoOp); - return client_->HeadBucket(request); -} - -Aws::S3::Model::DeleteObjectOutcome AwsS3ClientWrapper::DeleteObject( - const Aws::S3::Model::DeleteObjectRequest& request) { - CloudRequestCallbackGuard t(cloud_request_callback_.get(), - CloudRequestOpType::kDeleteOp); - auto outcome = client_->DeleteObject(request); - t.SetSuccess(outcome.IsSuccess()); - return outcome; -} - -Aws::S3::Model::CopyObjectOutcome AwsS3ClientWrapper::CopyObject( - const Aws::S3::Model::CopyObjectRequest& request) { - CloudRequestCallbackGuard t(cloud_request_callback_.get(), - CloudRequestOpType::kCopyOp); - auto outcome = client_->CopyObject(request); - t.SetSuccess(outcome.IsSuccess()); - return outcome; -} - -Aws::S3::Model::GetObjectOutcome AwsS3ClientWrapper::GetObject( - const Aws::S3::Model::GetObjectRequest& request) { - CloudRequestCallbackGuard t(cloud_request_callback_.get(), - CloudRequestOpType::kReadOp); - auto outcome = client_->GetObject(request); - if (outcome.IsSuccess()) { - t.SetSize(outcome.GetResult().GetContentLength()); - t.SetSuccess(true); - } - return outcome; -} - -Aws::S3::Model::PutObjectOutcome AwsS3ClientWrapper::PutObject( - const Aws::S3::Model::PutObjectRequest& request, uint64_t size_hint) { - CloudRequestCallbackGuard t(cloud_request_callback_.get(), - CloudRequestOpType::kWriteOp, size_hint); - auto outcome = client_->PutObject(request); - t.SetSuccess(outcome.IsSuccess()); - return outcome; -} - -Aws::S3::Model::HeadObjectOutcome AwsS3ClientWrapper::HeadObject( - const Aws::S3::Model::HeadObjectRequest& request) { - CloudRequestCallbackGuard t(cloud_request_callback_.get(), - CloudRequestOpType::kInfoOp); - auto outcome = client_->HeadObject(request); - t.SetSuccess(outcome.IsSuccess()); - return outcome; -} - +void AwsEnv::Shutdown() { Aws::ShutdownAPI(Aws::SDKOptions()); } + // // The AWS credentials are specified to the constructor via // access_key_id and secret_key. // -AwsEnv::AwsEnv(Env* underlying_env, const CloudEnvOptions& _cloud_env_options, - const std::shared_ptr& info_log) - : CloudEnvImpl(_cloud_env_options, underlying_env, info_log) { + AwsEnv::AwsEnv(Env* underlying_env, const CloudEnvOptions& _cloud_env_options, const std::shared_ptr& logger) + : CloudEnvImpl(_cloud_env_options, underlying_env, logger) { Aws::InitAPI(Aws::SDKOptions()); if (cloud_env_options.src_bucket.GetRegion().empty() || cloud_env_options.dest_bucket.GetRegion().empty()) { @@ -409,166 +194,18 @@ AwsEnv::AwsEnv(Env* underlying_env, const CloudEnvOptions& _cloud_env_options, } } - std::shared_ptr creds; - create_bucket_status_ = - cloud_env_options.credentials.GetCredentialsProvider(&creds); - if (!create_bucket_status_.ok()) { - Log(InfoLogLevel::INFO_LEVEL, info_log, - "[aws] NewAwsEnv - Bad AWS credentials"); - } - - Header(info_log_, " AwsEnv.src_bucket_name: %s", - cloud_env_options.src_bucket.GetBucketName().c_str()); - Header(info_log_, " AwsEnv.src_object_path: %s", - cloud_env_options.src_bucket.GetObjectPath().c_str()); - Header(info_log_, " AwsEnv.src_bucket_region: %s", - cloud_env_options.src_bucket.GetRegion().c_str()); - Header(info_log_, " AwsEnv.dest_bucket_name: %s", - cloud_env_options.dest_bucket.GetBucketName().c_str()); - Header(info_log_, " AwsEnv.dest_object_path: %s", - cloud_env_options.dest_bucket.GetObjectPath().c_str()); - Header(info_log_, " AwsEnv.dest_bucket_region: %s", - cloud_env_options.dest_bucket.GetRegion().c_str()); - Header(info_log_, " AwsEnv.credentials: %s", - creds ? "[given]" : "[not given]"); - base_env_ = underlying_env; - - // TODO: support buckets being in different regions - if (!SrcMatchesDest() && HasSrcBucket() && HasDestBucket()) { - if (cloud_env_options.src_bucket.GetRegion() == cloud_env_options.dest_bucket.GetRegion()) { - // alls good - } else { - create_bucket_status_ = - Status::InvalidArgument("Two different regions not supported"); - Log(InfoLogLevel::ERROR_LEVEL, info_log, - "[aws] NewAwsEnv Buckets %s, %s in two different regions %s, %s " - "is not supported", - cloud_env_options.src_bucket.GetBucketName().c_str(), - cloud_env_options.dest_bucket.GetBucketName().c_str(), - cloud_env_options.src_bucket.GetRegion().c_str(), - cloud_env_options.dest_bucket.GetRegion().c_str()); - return; - } - } - // create AWS S3 client with appropriate timeouts - Aws::Client::ClientConfiguration config; - create_bucket_status_ = - AwsCloudOptions::GetClientConfiguration(this, - cloud_env_options.src_bucket.GetRegion(), - &config); - - Header(info_log_, "AwsEnv connection to endpoint in region: %s", - config.region.c_str()); - bucket_location_ = Aws::S3::Model::BucketLocationConstraintMapper:: - GetBucketLocationConstraintForName(config.region); - - { - auto s3client = creds ? std::make_shared(creds, config) - : std::make_shared(config); - - s3client_ = std::make_shared( - std::move(s3client), cloud_env_options.cloud_request_callback); - } - - { - Aws::Transfer::TransferManagerConfiguration transferManagerConfig( - GetAwsTransferManagerExecutor()); - transferManagerConfig.s3Client = s3client_->GetClient(); - SetEncryptionParameters(cloud_env_options, - transferManagerConfig.putObjectTemplate); - SetEncryptionParameters( - cloud_env_options, transferManagerConfig.createMultipartUploadTemplate); - - if (cloud_env_options.use_aws_transfer_manager) { - awsTransferManager_ = - Aws::Transfer::TransferManager::Create(transferManagerConfig); - } - } - - // create dest bucket if specified - if (HasDestBucket()) { - if (S3WritableFile::BucketExistsInS3(s3client_, GetDestBucketName(), - bucket_location_) - .ok()) { - Log(InfoLogLevel::INFO_LEVEL, info_log, - "[aws] NewAwsEnv Bucket %s already exists", - GetDestBucketName().c_str()); - } else if (cloud_env_options.create_bucket_if_missing) { - Log(InfoLogLevel::INFO_LEVEL, info_log, - "[aws] NewAwsEnv Going to create bucket %s", - GetDestBucketName().c_str()); - create_bucket_status_ = S3WritableFile::CreateBucketInS3( - s3client_, GetDestBucketName(), bucket_location_); - } else { - create_bucket_status_ = Status::NotFound( - "[aws] Bucket not found and create_bucket_if_missing is false"); - } - } - if (!create_bucket_status_.ok()) { - Log(InfoLogLevel::ERROR_LEVEL, info_log, - "[aws] NewAwsEnv Unable to create bucket %s %s", - GetDestBucketName().c_str(), - create_bucket_status_.ToString().c_str()); - } - - // create cloud log client for storing/reading logs - if (create_bucket_status_.ok() && !cloud_env_options.keep_local_log_files) { - if (cloud_env_options.log_type == kLogKinesis) { - create_bucket_status_ = CreateKinesisController(this, &cloud_log_controller_); - } else if (cloud_env_options.log_type == kLogKafka) { -#ifdef USE_KAFKA - create_bucket_status_ = CreateKafkaController(this, &cloud_log_controller_); -#else - create_bucket_status_ = Status::NotSupported( - "In order to use Kafka, make sure you're compiling with " - "USE_KAFKA=1"); - - Log(InfoLogLevel::ERROR_LEVEL, info_log, - "[aws] NewAwsEnv Unknown log type %d. %s", - cloud_env_options.log_type, - create_bucket_status_.ToString().c_str()); -#endif /* USE_KAFKA */ - } else { - create_bucket_status_ = - Status::NotSupported("We currently only support Kinesis and Kafka"); - - Log(InfoLogLevel::ERROR_LEVEL, info_log, - "[aws] NewAwsEnv Unknown log type %d. %s", cloud_env_options.log_type, - create_bucket_status_.ToString().c_str()); - } - - // Create Kinesis stream and wait for it to be ready - if (create_bucket_status_.ok()) { - create_bucket_status_ = - cloud_log_controller_->StartTailingStream(GetSrcBucketName()); - if (!create_bucket_status_.ok()) { - Log(InfoLogLevel::ERROR_LEVEL, info_log, - "[aws] NewAwsEnv Unable to create stream %s", - create_bucket_status_.ToString().c_str()); - } - } - } - if (!create_bucket_status_.ok()) { - Log(InfoLogLevel::ERROR_LEVEL, info_log, - "[aws] NewAwsEnv Unable to create environment %s", - create_bucket_status_.ToString().c_str()); - } } AwsEnv::~AwsEnv() { - { - std::lock_guard lk(files_to_delete_mutex_); - using std::swap; - for (auto& e : files_to_delete_) { - GetJobExecutor()->CancelJob(e.second.get()); - } - files_to_delete_.clear(); - } - - StopPurger(); } +Status AwsEnv::Prepare() { + if (create_bucket_status_.ok()) { + create_bucket_status_ = CloudEnvImpl::Prepare(); + } + return create_bucket_status_; +} Status AwsEnv::status() { return create_bucket_status_; } @@ -588,10 +225,10 @@ Status AwsEnv::CheckOption(const EnvOptions& options) { Status AwsEnv::NewSequentialFileCloud(const std::string& bucket, const std::string& fname, std::unique_ptr* result, - const EnvOptions& options) { + const EnvOptions& /*options*/) { assert(status().ok()); - std::unique_ptr file; - Status st = NewS3ReadableFile(bucket, fname, &file); + std::unique_ptr file; + Status st = cloud_env_options.storage_provider->NewCloudReadableFile(bucket, fname, &file); if (!st.ok()) { return st; } @@ -627,22 +264,26 @@ Status AwsEnv::NewSequentialFile(const std::string& logical_fname, if (cloud_env_options.keep_local_sst_files || !sstfile) { // copy the file to the local storage if keep_local_sst_files is true if (HasDestBucket()) { - st = GetObject(GetDestBucketName(), destname(fname), fname); + st = cloud_env_options.storage_provider->GetObject(GetDestBucketName(), + destname(fname), fname); } if (!st.ok() && HasSrcBucket() && !SrcMatchesDest()) { - st = GetObject(GetSrcBucketName(), srcname(fname), fname); + st = cloud_env_options.storage_provider->GetObject(GetSrcBucketName(), srcname(fname), + fname); } if (st.ok()) { // we successfully copied the file, try opening it locally now st = base_env_->NewSequentialFile(fname, result, options); } } else { - std::unique_ptr file; + std::unique_ptr file; if (!st.ok() && HasDestBucket()) { // read from destination S3 - st = NewS3ReadableFile(GetDestBucketName(), destname(fname), &file); + st = cloud_env_options.storage_provider->NewCloudReadableFile(GetDestBucketName(), + destname(fname), &file); } if (!st.ok() && HasSrcBucket()) { // read from src bucket - st = NewS3ReadableFile(GetSrcBucketName(), srcname(fname), &file); + st = cloud_env_options.storage_provider->NewCloudReadableFile(GetSrcBucketName(), + srcname(fname), &file); } if (st.ok()) { result->reset(dynamic_cast(file.release())); @@ -656,17 +297,17 @@ Status AwsEnv::NewSequentialFile(const std::string& logical_fname, } else if (logfile && !cloud_env_options.keep_local_log_files) { // read from Kinesis - st = cloud_log_controller_->status(); + st = cloud_env_options.log_controller->status(); if (st.ok()) { // map pathname to cache dir - std::string pathname = cloud_log_controller_->GetCachePath(Slice(fname)); + std::string pathname = cloud_env_options.log_controller->GetCachePath(Slice(fname)); Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[Kinesis] NewSequentialFile logfile %s %s", pathname.c_str(), "ok"); auto lambda = [this, pathname, &result, options]() -> Status { return base_env_->NewSequentialFile(pathname, result, options); }; - return cloud_log_controller_->Retry(lambda); + return cloud_env_options.log_controller->Retry(lambda); } } @@ -707,10 +348,12 @@ Status AwsEnv::NewRandomAccessFile(const std::string& logical_fname, if (!st.ok()) { // copy the file to the local storage if keep_local_sst_files is true if (HasDestBucket()) { - st = GetObject(GetDestBucketName(), destname(fname), fname); + st = cloud_env_options.storage_provider->GetObject(GetDestBucketName(), + destname(fname), fname); } if (!st.ok() && HasSrcBucket() && !SrcMatchesDest()) { - st = GetObject(GetSrcBucketName(), srcname(fname), fname); + st = cloud_env_options.storage_provider->GetObject(GetSrcBucketName(), srcname(fname), + fname); } if (st.ok()) { // we successfully copied the file, try opening it locally now @@ -728,12 +371,12 @@ Status AwsEnv::NewRandomAccessFile(const std::string& logical_fname, } stax = Status::NotFound(); if (HasDestBucket()) { - stax = HeadObject(GetDestBucketName(), destname(fname), nullptr, - &remote_size, nullptr); + cloud_env_options.storage_provider->GetObjectSize(GetDestBucketName(), destname(fname), + &remote_size); } if (stax.IsNotFound() && HasSrcBucket()) { - stax = HeadObject(GetSrcBucketName(), srcname(fname), nullptr, - &remote_size, nullptr); + stax = cloud_env_options.storage_provider->GetObjectSize(GetSrcBucketName(), + srcname(fname), &remote_size); } if (stax.IsNotFound() && !HasDestBucket()) { // It is legal for file to not be present in S3 if destination bucket @@ -750,28 +393,30 @@ Status AwsEnv::NewRandomAccessFile(const std::string& logical_fname, // Only execute this code path if keep_local_sst_files == false. If it's // true, we will never use S3ReadableFile to read; we copy the file // locally and read using base_env. - std::unique_ptr file; + std::unique_ptr file; if (!st.ok() && HasDestBucket()) { - st = NewS3ReadableFile(GetDestBucketName(), destname(fname), &file); + st = cloud_env_options.storage_provider->NewCloudReadableFile(GetDestBucketName(), + destname(fname), &file); } if (!st.ok() && HasSrcBucket()) { - st = NewS3ReadableFile(GetSrcBucketName(), srcname(fname), &file); + st = cloud_env_options.storage_provider->NewCloudReadableFile(GetSrcBucketName(), + srcname(fname), &file); } if (st.ok()) { result->reset(dynamic_cast(file.release())); } } Log(InfoLogLevel::DEBUG_LEVEL, info_log_, - "[s3] NewRandomAccessFile file %s %s", fname.c_str(), + "[aws] NewRandomAccessFile file %s %s", fname.c_str(), st.ToString().c_str()); return st; } else if (logfile && !cloud_env_options.keep_local_log_files) { // read from Kinesis - st = cloud_log_controller_->status(); + st = cloud_env_options.log_controller->status(); if (st.ok()) { // map pathname to cache dir - std::string pathname = cloud_log_controller_->GetCachePath(Slice(fname)); + std::string pathname = cloud_env_options.log_controller->GetCachePath(Slice(fname)); Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[kinesis] NewRandomAccessFile logfile %s %s", pathname.c_str(), "ok"); @@ -779,7 +424,7 @@ Status AwsEnv::NewRandomAccessFile(const std::string& logical_fname, auto lambda = [this, pathname, &result, options]() -> Status { return base_env_->NewRandomAccessFile(pathname, result, options); }; - return cloud_log_controller_->Retry(lambda); + return cloud_env_options.log_controller->Retry(lambda); } } @@ -804,20 +449,20 @@ Status AwsEnv::NewWritableFile(const std::string& logical_fname, Status s; if (HasDestBucket() && (sstfile || identity || manifest)) { - std::unique_ptr f( - new S3WritableFile(this, fname, GetDestBucketName(), destname(fname), - options, cloud_env_options)); + std::unique_ptr f; + cloud_env_options.storage_provider->NewCloudWritableFile(fname, GetDestBucketName(), + destname(fname), &f, options); s = f->status(); if (!s.ok()) { Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] NewWritableFile src %s %s", fname.c_str(), + "[aws] NewWritableFile src %s %s", fname.c_str(), s.ToString().c_str()); return s; } result->reset(dynamic_cast(f.release())); } else if (logfile && !cloud_env_options.keep_local_log_files) { std::unique_ptr f( - cloud_log_controller_->CreateWritableFile(fname, options)); + cloud_env_options.log_controller->CreateWritableFile(fname, options)); if (!f || !f->status().ok()) { s = Status::IOError("[aws] NewWritableFile", fname.c_str()); Log(InfoLogLevel::ERROR_LEVEL, info_log_, @@ -904,24 +549,24 @@ Status AwsEnv::FileExists(const std::string& logical_fname) { // We read first from local storage and then from cloud storage. st = base_env_->FileExists(fname); if (st.IsNotFound() && HasDestBucket()) { - st = ExistsObject(GetDestBucketName(), destname(fname)); + st = cloud_env_options.storage_provider->ExistsObject(GetDestBucketName(), destname(fname)); } if (!st.ok() && HasSrcBucket()) { - st = ExistsObject(GetSrcBucketName(), srcname(fname)); + st = cloud_env_options.storage_provider->ExistsObject(GetSrcBucketName(), srcname(fname)); } } else if (logfile && !cloud_env_options.keep_local_log_files) { // read from Kinesis - st = cloud_log_controller_->status(); + st = cloud_env_options.log_controller->status(); if (st.ok()) { // map pathname to cache dir - std::string pathname = cloud_log_controller_->GetCachePath(Slice(fname)); + std::string pathname = cloud_env_options.log_controller->GetCachePath(Slice(fname)); Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[kinesis] FileExists logfile %s %s", pathname.c_str(), "ok"); auto lambda = [this, pathname]() -> Status { return base_env_->FileExists(pathname); }; - st = cloud_log_controller_->Retry(lambda); + st = cloud_env_options.log_controller->Retry(lambda); } } else { st = base_env_->FileExists(fname); @@ -931,195 +576,46 @@ Status AwsEnv::FileExists(const std::string& logical_fname) { return st; } -// -// Appends the names of all children of the specified path from S3 -// into the result set. -// -Status AwsEnv::GetChildrenFromS3(const std::string& path, - const std::string& bucket, - std::vector* result) { - assert(status().ok()); - - // S3 paths don't start with '/' - auto prefix = ltrim_if(path, '/'); - // S3 paths better end with '/', otherwise we might also get a list of files - // in a directory for which our path is a prefix - prefix = ensure_ends_with_pathsep(std::move(prefix)); - // the starting object marker - Aws::String marker; - bool loop = true; - - // get info of bucket+object - while (loop) { - Aws::S3::Model::ListObjectsRequest request; - request.SetBucket(ToAwsString(bucket)); - request.SetMaxKeys(50); - request.SetPrefix(ToAwsString(prefix)); - request.SetMarker(marker); - - Aws::S3::Model::ListObjectsOutcome outcome = - s3client_->ListObjects(request); - bool isSuccess = outcome.IsSuccess(); - if (!isSuccess) { - const Aws::Client::AWSError& error = - outcome.GetError(); - std::string errmsg(error.GetMessage().c_str()); - Aws::S3::S3Errors s3err = error.GetErrorType(); - if (s3err == Aws::S3::S3Errors::NO_SUCH_BUCKET || - s3err == Aws::S3::S3Errors::NO_SUCH_KEY || - s3err == Aws::S3::S3Errors::RESOURCE_NOT_FOUND) { - Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] GetChildren dir %s does not exist: %s", path.c_str(), - errmsg.c_str()); - return Status::NotFound(path, errmsg.c_str()); - } - return Status::IOError(path, errmsg.c_str()); - } - const Aws::S3::Model::ListObjectsResult& res = outcome.GetResult(); - const Aws::Vector& objs = res.GetContents(); - for (auto o : objs) { - const Aws::String& key = o.GetKey(); - // Our path should be a prefix of the fetched value - std::string keystr(key.c_str(), key.size()); - assert(keystr.find(prefix) == 0); - if (keystr.find(prefix) != 0) { - return Status::IOError("Unexpected result from AWS S3: " + keystr); - } - auto fname = keystr.substr(prefix.size()); - result->push_back(fname); - } - - // If there are no more entries, then we are done. - if (!res.GetIsTruncated()) { - break; - } - // The new starting point - marker = res.GetNextMarker(); - if (marker.empty()) { - // If response does not include the NextMaker and it is - // truncated, you can use the value of the last Key in the response - // as the marker in the subsequent request because all objects - // are returned in alphabetical order - marker = objs.back().GetKey(); - } - } - return Status::OK(); -} - -Status AwsEnv::HeadObject(const std::string& bucket, - const std::string& path, - Aws::Map* metadata, - uint64_t* size, uint64_t* modtime) { - Aws::S3::Model::HeadObjectRequest request; - request.SetBucket(ToAwsString(bucket)); - request.SetKey(ToAwsString(path)); - - auto outcome = s3client_->HeadObject(request); - bool isSuccess = outcome.IsSuccess(); - if (!isSuccess) { - const auto& error = outcome.GetError(); - Aws::S3::S3Errors s3err = error.GetErrorType(); - auto errMessage = error.GetMessage(); - if (s3err == Aws::S3::S3Errors::NO_SUCH_BUCKET || - s3err == Aws::S3::S3Errors::NO_SUCH_KEY || - s3err == Aws::S3::S3Errors::RESOURCE_NOT_FOUND) { - return Status::NotFound(path, errMessage.c_str()); - } - return Status::IOError(path, errMessage.c_str()); - } - auto& res = outcome.GetResult(); - if (metadata != nullptr) { - *metadata = res.GetMetadata(); - } - if (size != nullptr) { - *size = res.GetContentLength(); - } - if (modtime != nullptr) { - *modtime = res.GetLastModified().Millis(); - } - return Status::OK(); -} - -Status AwsEnv::NewS3ReadableFile(const std::string& bucket, - const std::string& fname, - std::unique_ptr* result) { - // First, check if the file exists and also find its size. We use size in - // S3ReadableFile to make sure we always read the valid ranges of the file - uint64_t size; - Status st = HeadObject(bucket, fname, nullptr, &size, nullptr); - if (!st.ok()) { - return st; - } - result->reset(new S3ReadableFile(this, bucket, fname, size)); - return Status::OK(); -} - -// -// Deletes all the objects with the specified path prefix in our bucket -// -Status AwsEnv::EmptyBucket(const std::string& bucket, - const std::string& s3_object_prefix) { - std::vector results; - - // Get all the objects in the bucket - Status st = GetChildrenFromS3(s3_object_prefix, bucket, &results); - if (!st.ok()) { - Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] EmptyBucket unable to find objects in bucket %s %s", - bucket.c_str(), st.ToString().c_str()); - return st; - } - Log(InfoLogLevel::DEBUG_LEVEL, info_log_, - "[s3] EmptyBucket going to delete %" ROCKSDB_PRIszt - " objects in bucket %s", - results.size(), bucket.c_str()); - - // Delete all objects from bucket - for (auto path : results) { - st = DeletePathInS3(bucket, path); - if (!st.ok()) { - Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] EmptyBucket Unable to delete %s in bucket %s %s", path.c_str(), - bucket.c_str(), st.ToString().c_str()); - } - } - return st; -} Status AwsEnv::GetChildren(const std::string& path, std::vector* result) { assert(status().ok()); - Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[s3] GetChildren path '%s' ", + Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[aws] GetChildren path '%s' ", path.c_str()); result->clear(); // Fetch the list of children from both buckets in S3 Status st; if (HasSrcBucket()) { - st = GetChildrenFromS3(GetSrcObjectPath(), GetSrcBucketName(), result); + st = cloud_env_options.storage_provider->ListObjects(GetSrcBucketName(), GetSrcObjectPath(), + result); if (!st.ok()) { Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] GetChildren src bucket %s %s error from S3 %s", + "[aws] GetChildren src bucket %s %s error from S3 %s", GetSrcBucketName().c_str(), path.c_str(), st.ToString().c_str()); return st; } } if (HasDestBucket() && !SrcMatchesDest()) { - st = GetChildrenFromS3(GetDestObjectPath(), GetDestBucketName(), result); + std::vector dest_objects; + st = cloud_env_options.storage_provider->ListObjects(GetDestBucketName(), + GetDestObjectPath(), &dest_objects); if (!st.ok()) { Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] GetChildren dest bucket %s %s error from S3 %s", + "[aws] GetChildren dest bucket %s %s error from S3 %s", GetDestBucketName().c_str(), path.c_str(), st.ToString().c_str()); return st; } + for (auto const& value : dest_objects) { + result->push_back(value); + } } - // fetch all files that exist in the local posix directory std::vector local_files; st = base_env_->GetChildren(path, &local_files); if (!st.ok()) { Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] GetChildren %s error on local dir", path.c_str()); + "[aws] GetChildren %s error on local dir", path.c_str()); return st; } @@ -1151,20 +647,11 @@ Status AwsEnv::GetChildren(const std::string& path, result->erase(std::unique(result->begin(), result->end()), result->end()); Log(InfoLogLevel::DEBUG_LEVEL, info_log_, - "[s3] GetChildren %s successfully returned %" ROCKSDB_PRIszt " files", + "[aws] GetChildren %s successfully returned %" ROCKSDB_PRIszt " files", path.c_str(), result->size()); return Status::OK(); } -void AwsEnv::RemoveFileFromDeletionQueue(const std::string& filename) { - std::lock_guard lk(files_to_delete_mutex_); - auto itr = files_to_delete_.find(filename); - if (itr != files_to_delete_.end()) { - GetJobExecutor()->CancelJob(itr->second.get()); - files_to_delete_.erase(itr); - } -} - Status AwsEnv::DeleteFile(const std::string& logical_fname) { assert(status().ok()); @@ -1213,11 +700,11 @@ Status AwsEnv::DeleteFile(const std::string& logical_fname) { base_env_->DeleteFile(fname); } else if (logfile && !cloud_env_options.keep_local_log_files) { // read from Kinesis - st = cloud_log_controller_->status(); + st = cloud_env_options.log_controller->status(); if (st.ok()) { // Log a Delete record to kinesis stream std::unique_ptr f( - cloud_log_controller_->CreateWritableFile(fname, EnvOptions())); + cloud_env_options.log_controller->CreateWritableFile(fname, EnvOptions())); if (!f || !f->status().ok()) { st = Status::IOError("[Kinesis] DeleteFile", fname.c_str()); } else { @@ -1227,101 +714,23 @@ Status AwsEnv::DeleteFile(const std::string& logical_fname) { } else { st = base_env_->DeleteFile(fname); } - Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[s3] DeleteFile file %s %s", + Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[aws] DeleteFile file %s %s", fname.c_str(), st.ToString().c_str()); return st; } -Status AwsEnv::DeleteCloudFileFromDest(const std::string& fname) { - assert(HasDestBucket()); - auto base = basename(fname); - // add the job to delete the file in 1 hour - auto doDeleteFile = [this, base]() { - { - std::lock_guard lk(files_to_delete_mutex_); - auto itr = files_to_delete_.find(base); - if (itr == files_to_delete_.end()) { - // File was removed from files_to_delete_, do not delete! - return; - } - files_to_delete_.erase(itr); - } - auto path = GetDestObjectPath() + "/" + base; - // we are ready to delete the file! - auto st = DeletePathInS3(GetDestBucketName(), path); - if (!st.ok() && !st.IsNotFound()) { - Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] DeleteFile DeletePathInS3 file %s error %s", path.c_str(), - st.ToString().c_str()); - } - }; - { - std::lock_guard lk(files_to_delete_mutex_); - if (files_to_delete_.find(base) != files_to_delete_.end()) { - // already in the queue - return Status::OK(); - } - } - { - std::lock_guard lk(files_to_delete_mutex_); - auto handle = GetJobExecutor()->ScheduleJob( - std::chrono::steady_clock::now() + file_deletion_delay_, - std::move(doDeleteFile)); - files_to_delete_.emplace(base, std::move(handle)); - } - return Status::OK(); -} - -// -// Delete the specified path from S3 -// -Status AwsEnv::DeletePathInS3(const std::string& bucket, - const std::string& fname) { - assert(status().ok()); - Status st; - - // The filename is the same as the object name in the bucket - Aws::String object = ToAwsString(fname); - - // create request - Aws::S3::Model::DeleteObjectRequest request; - request.SetBucket(ToAwsString(bucket)); - request.SetKey(object); - - Aws::S3::Model::DeleteObjectOutcome outcome = - s3client_->DeleteObject(request); - bool isSuccess = outcome.IsSuccess(); - if (!isSuccess) { - const Aws::Client::AWSError& error = outcome.GetError(); - std::string errmsg(error.GetMessage().c_str()); - Aws::S3::S3Errors s3err = error.GetErrorType(); - if (s3err == Aws::S3::S3Errors::NO_SUCH_BUCKET || - s3err == Aws::S3::S3Errors::NO_SUCH_KEY || - s3err == Aws::S3::S3Errors::RESOURCE_NOT_FOUND) { - st = Status::NotFound(fname, errmsg.c_str()); - } else { - st = Status::IOError(fname, errmsg.c_str()); - } - } - - Log(InfoLogLevel::INFO_LEVEL, info_log_, "[s3] DeleteFromS3 %s/%s, status %s", - bucket.c_str(), object.c_str(), st.ToString().c_str()); - - return st; -} - // S3 has no concepts of directories, so we just have to forward the request to // base_env_ Status AwsEnv::CreateDir(const std::string& dirname) { assert(status().ok()); - Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[s3] CreateDir dir '%s'", + Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[aws] CreateDir dir '%s'", dirname.c_str()); Status st; // create local dir st = base_env_->CreateDir(dirname); - Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[s3] CreateDir dir %s %s", + Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[aws] CreateDir dir %s %s", dirname.c_str(), st.ToString().c_str()); return st; }; @@ -1330,7 +739,7 @@ Status AwsEnv::CreateDir(const std::string& dirname) { // base_env_ Status AwsEnv::CreateDirIfMissing(const std::string& dirname) { assert(status().ok()); - Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[s3] CreateDirIfMissing dir '%s'", + Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[aws] CreateDirIfMissing dir '%s'", dirname.c_str()); Status st; @@ -1338,7 +747,7 @@ Status AwsEnv::CreateDirIfMissing(const std::string& dirname) { st = base_env_->CreateDirIfMissing(dirname); Log(InfoLogLevel::DEBUG_LEVEL, info_log_, - "[s3] CreateDirIfMissing created dir %s %s", dirname.c_str(), + "[aws] CreateDirIfMissing created dir %s %s", dirname.c_str(), st.ToString().c_str()); return st; }; @@ -1347,10 +756,10 @@ Status AwsEnv::CreateDirIfMissing(const std::string& dirname) { // base_env_ Status AwsEnv::DeleteDir(const std::string& dirname) { assert(status().ok()); - Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[s3] DeleteDir src '%s'", + Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[aws] DeleteDir src '%s'", dirname.c_str()); Status st = base_env_->DeleteDir(dirname); - Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[s3] DeleteDir dir %s %s", + Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[aws] DeleteDir dir %s %s", dirname.c_str(), st.ToString().c_str()); return st; }; @@ -1372,26 +781,26 @@ Status AwsEnv::GetFileSize(const std::string& logical_fname, uint64_t* size) { st = Status::NotFound(); // Get file length from S3 if (HasDestBucket()) { - st = HeadObject(GetDestBucketName(), destname(fname), nullptr, size, - nullptr); + st = cloud_env_options.storage_provider->GetObjectSize(GetDestBucketName(), + destname(fname), size); } if (st.IsNotFound() && HasSrcBucket()) { - st = HeadObject(GetSrcBucketName(), srcname(fname), nullptr, size, - nullptr); + st = cloud_env_options.storage_provider->GetObjectSize(GetSrcBucketName(), + srcname(fname), size); } } } else if (logfile && !cloud_env_options.keep_local_log_files) { - st = cloud_log_controller_->status(); + st = cloud_env_options.log_controller->status(); if (st.ok()) { // map pathname to cache dir - std::string pathname = cloud_log_controller_->GetCachePath(Slice(fname)); + std::string pathname = cloud_env_options.log_controller->GetCachePath(Slice(fname)); Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[kinesis] GetFileSize logfile %s %s", pathname.c_str(), "ok"); auto lambda = [this, pathname, size]() -> Status { return base_env_->GetFileSize(pathname, size); }; - st = cloud_log_controller_->Retry(lambda); + st = cloud_env_options.log_controller->Retry(lambda); } } else { st = base_env_->GetFileSize(fname, size); @@ -1399,6 +808,7 @@ Status AwsEnv::GetFileSize(const std::string& logical_fname, uint64_t* size) { Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[aws] GetFileSize src '%s' %s %" PRIu64, fname.c_str(), st.ToString().c_str(), *size); + return st; } @@ -1419,19 +829,19 @@ Status AwsEnv::GetFileModificationTime(const std::string& logical_fname, } else { st = Status::NotFound(); if (HasDestBucket()) { - st = HeadObject(GetDestBucketName(), destname(fname), nullptr, - nullptr, time); + st = cloud_env_options.storage_provider->GetObjectModificationTime(GetDestBucketName(), + destname(fname), time); } if (st.IsNotFound() && HasSrcBucket()) { - st = HeadObject(GetSrcBucketName(), srcname(fname), nullptr, nullptr, - time); + st = cloud_env_options.storage_provider->GetObjectModificationTime(GetSrcBucketName(), + srcname(fname), time); } } } else if (logfile && !cloud_env_options.keep_local_log_files) { - st = cloud_log_controller_->status(); + st = cloud_env_options.log_controller->status(); if (st.ok()) { // map pathname to cache dir - std::string pathname = cloud_log_controller_->GetCachePath(Slice(fname)); + std::string pathname = cloud_env_options.log_controller->GetCachePath(Slice(fname)); Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[kinesis] GetFileModificationTime logfile %s %s", pathname.c_str(), "ok"); @@ -1439,7 +849,7 @@ Status AwsEnv::GetFileModificationTime(const std::string& logical_fname, auto lambda = [this, pathname, time]() -> Status { return base_env_->GetFileModificationTime(pathname, time); }; - st = cloud_log_controller_->Retry(lambda); + st = cloud_env_options.log_controller->Retry(lambda); } } else { st = base_env_->GetFileModificationTime(fname, time); @@ -1503,7 +913,7 @@ Status AwsEnv::RenameFile(const std::string& logical_src, st = base_env_->RenameFile(src, target); } Log(InfoLogLevel::DEBUG_LEVEL, info_log_, - "[s3] RenameFile src %s target %s: %s", src.c_str(), target.c_str(), + "[aws] RenameFile src %s target %s: %s", src.c_str(), target.c_str(), st.ToString().c_str()); return st; } @@ -1532,7 +942,7 @@ Status AwsEnv::SaveIdentitytoS3(const std::string& localfile, // Upload ID file to S3 if (st.ok()) { - st = PutObject(localfile, GetDestBucketName(), idfile); + st = cloud_env_options.storage_provider->PutObject(localfile, GetDestBucketName(), idfile); } // Save mapping from ID to cloud pathname @@ -1549,39 +959,26 @@ Status AwsEnv::SaveIdentitytoS3(const std::string& localfile, Status AwsEnv::SaveDbid(const std::string& bucket_name, const std::string& dbid, const std::string& dirname) { assert(status().ok()); - Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[s3] SaveDbid dbid %s dir '%s'", + Log(InfoLogLevel::DEBUG_LEVEL, info_log_, "[aws] SaveDbid dbid %s dir '%s'", dbid.c_str(), dirname.c_str()); std::string dbidkey = dbid_registry_ + dbid; - Aws::String s3_bucket = ToAwsString(bucket_name); - Aws::String key = ToAwsString(dbidkey); - - Aws::Map metadata; - metadata[ToAwsString("dirname")] = ToAwsString(dirname); - - // create request - Aws::S3::Model::PutObjectRequest put_request; - put_request.SetBucket(s3_bucket); - put_request.SetKey(key); - put_request.SetMetadata(metadata); - SetEncryptionParameters(cloud_env_options, put_request); - - Aws::S3::Model::PutObjectOutcome put_outcome = - s3client_->PutObject(put_request); - bool isSuccess = put_outcome.IsSuccess(); - if (!isSuccess) { - const Aws::Client::AWSError& error = - put_outcome.GetError(); - std::string errmsg(error.GetMessage().c_str(), error.GetMessage().size()); + std::unordered_map metadata; + metadata["dirname"] = dirname; + + Status st = + cloud_env_options.storage_provider->PutObjectMetadata(bucket_name, dbidkey, metadata); + if (!st.ok()) { Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] Bucket %s SaveDbid error in saving dbid %s dirname %s %s", - bucket_name.c_str(), dbid.c_str(), dirname.c_str(), errmsg.c_str()); - return Status::IOError(dirname, errmsg.c_str()); + "[aws] Bucket %s SaveDbid error in saving dbid %s dirname %s %s", + bucket_name.c_str(), dbid.c_str(), dirname.c_str(), + st.ToString().c_str()); + } else { + Log(InfoLogLevel::INFO_LEVEL, info_log_, + "[aws] Bucket %s SaveDbid dbid %s dirname %s %s", bucket_name.c_str(), + dbid.c_str(), dirname.c_str(), "ok"); } - Log(InfoLogLevel::INFO_LEVEL, info_log_, - "[s3] Bucket %s SaveDbid dbid %s dirname %s %s", bucket_name.c_str(), - dbid.c_str(), dirname.c_str(), "ok"); - return Status::OK(); + return st; }; // @@ -1592,19 +989,18 @@ Status AwsEnv::GetPathForDbid(const std::string& bucket, std::string dbidkey = dbid_registry_ + dbid; Log(InfoLogLevel::DEBUG_LEVEL, info_log_, - "[s3] Bucket %s GetPathForDbid dbid %s", bucket.c_str(), - dbid.c_str()); + "[aws] Bucket %s GetPathForDbid dbid %s", bucket.c_str(), dbid.c_str()); - Aws::Map metadata; - Status st = HeadObject(bucket, dbidkey, &metadata); + std::unordered_map metadata; + Status st = cloud_env_options.storage_provider->GetObjectMetadata(bucket, dbidkey, &metadata); if (!st.ok()) { if (st.IsNotFound()) { Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] %s GetPathForDbid error non-existent dbid %s %s", + "[aws] %s GetPathForDbid error non-existent dbid %s %s", bucket.c_str(), dbid.c_str(), st.ToString().c_str()); } else { Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] %s GetPathForDbid error dbid %s %s", bucket.c_str(), + "[aws] %s GetPathForDbid error dbid %s %s", bucket.c_str(), dbid.c_str(), st.ToString().c_str()); } return st; @@ -1612,14 +1008,13 @@ Status AwsEnv::GetPathForDbid(const std::string& bucket, // Find "dirname" metadata that stores the pathname of the db const char* kDirnameTag = "dirname"; - auto it = metadata.find(Aws::String(kDirnameTag)); + auto it = metadata.find(kDirnameTag); if (it != metadata.end()) { - Aws::String as = it->second; - dirname->assign(as.c_str(), as.size()); + *dirname = it->second; } else { st = Status::NotFound("GetPathForDbid"); } - Log(InfoLogLevel::INFO_LEVEL, info_log_, "[s3] %s GetPathForDbid dbid %s %s", + Log(InfoLogLevel::INFO_LEVEL, info_log_, "[aws] %s GetPathForDbid dbid %s %s", bucket.c_str(), dbid.c_str(), st.ToString().c_str()); return st; } @@ -1631,10 +1026,11 @@ Status AwsEnv::GetDbidList(const std::string& bucket, DbidList* dblist) { // fetch the list all all dbids std::vector dbid_list; - Status st = GetChildrenFromS3(dbid_registry_, bucket, &dbid_list); + Status st = + cloud_env_options.storage_provider->ListObjects(bucket, dbid_registry_, &dbid_list); if (!st.ok()) { Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] %s GetDbidList error in GetChildrenFromS3 %s", bucket.c_str(), + "[aws] %s GetDbidList error in GetChildrenFromS3 %s", bucket.c_str(), st.ToString().c_str()); return st; } @@ -1644,7 +1040,7 @@ Status AwsEnv::GetDbidList(const std::string& bucket, DbidList* dblist) { st = GetPathForDbid(bucket, dbid, &dirname); if (!st.ok()) { Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] %s GetDbidList error in GetPathForDbid(%s) %s", bucket.c_str(), + "[aws] %s GetDbidList error in GetPathForDbid(%s) %s", bucket.c_str(), dbid.c_str(), st.ToString().c_str()); return st; } @@ -1662,273 +1058,13 @@ Status AwsEnv::DeleteDbid(const std::string& bucket, // fetch the list all all dbids std::string dbidkey = dbid_registry_ + dbid; - Status st = DeletePathInS3(bucket, dbidkey); + Status st = cloud_env_options.storage_provider->DeleteObject(bucket, dbidkey); Log(InfoLogLevel::DEBUG_LEVEL, info_log_, - "[s3] %s DeleteDbid DeleteDbid(%s) %s", bucket.c_str(), dbid.c_str(), + "[aws] %s DeleteDbid DeleteDbid(%s) %s", bucket.c_str(), dbid.c_str(), st.ToString().c_str()); return st; } -// Returns a list of all objects that start with the specified -// prefix and are stored in the bucket. -Status AwsEnv::ListObjects(const std::string& bucket_name, - const std::string& object_path, - BucketObjectMetadata* meta) { - return GetChildrenFromS3(object_path, bucket_name, - &meta->pathnames); -} - -// Deletes the specified object from cloud storage -Status AwsEnv::DeleteObject(const std::string& bucket_name, - const std::string& object_path) { - return DeletePathInS3(bucket_name, object_path); -} - -// Delete the specified object from the specified cloud bucket -Status AwsEnv::ExistsObject(const std::string& bucket_name, - const std::string& object_path) { - return HeadObject(bucket_name, object_path); -} - -// Return size of cloud object -Status AwsEnv::GetObjectSize(const std::string& bucket_name, - const std::string& object_path, - uint64_t* filesize) { - return HeadObject(bucket_name, object_path, nullptr, filesize, - nullptr); -} - -// Copy the specified cloud object from one location in the cloud -// storage to another location in cloud storage -Status AwsEnv::CopyObject(const std::string& bucket_name_src, - const std::string& object_path_src, - const std::string& bucket_name_dest, - const std::string& object_path_dest) { - Status st; - Aws::String src_bucket = ToAwsString(bucket_name_src); - Aws::String dest_bucket = ToAwsString(bucket_name_dest); - - // The filename is the same as the object name in the bucket - Aws::String src_object = ToAwsString(object_path_src); - Aws::String dest_object = ToAwsString(object_path_dest); - - Aws::String src_url = src_bucket + src_object; - - // create copy request - Aws::S3::Model::CopyObjectRequest request; - request.SetCopySource(src_url); - request.SetBucket(dest_bucket); - request.SetKey(dest_object); - - // execute request - Aws::S3::Model::CopyObjectOutcome outcome = s3client_->CopyObject(request); - bool isSuccess = outcome.IsSuccess(); - if (!isSuccess) { - const Aws::Client::AWSError& error = outcome.GetError(); - std::string errmsg(error.GetMessage().c_str()); - Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[aws] S3WritableFile src path %s error in copying to %s %s", - src_url.c_str(), dest_object.c_str(), errmsg.c_str()); - return Status::IOError(dest_object.c_str(), errmsg.c_str()); - } - Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[aws] S3WritableFile src path %s copied to %s %s", src_url.c_str(), - dest_object.c_str(), st.ToString().c_str()); - return st; -} - -AwsEnv::GetObjectResult AwsEnv::DoGetObject(const Aws::String& bucket, - const Aws::String& key, - const std::string& destination) { - Aws::S3::Model::GetObjectRequest getObjectRequest; - getObjectRequest.SetBucket(bucket); - getObjectRequest.SetKey(key); - - getObjectRequest.SetResponseStreamFactory([destination]() { - return Aws::New(Aws::Utils::ARRAY_ALLOCATION_TAG, destination, - std::ios_base::out); - }); - auto getOutcome = s3client_->GetObject(getObjectRequest); - GetObjectResult result; - result.success = getOutcome.IsSuccess(); - if (!result.success) { - result.error = getOutcome.GetError(); - } else { - result.objectSize = getOutcome.GetResult().GetContentLength(); - } - return result; -} - -AwsEnv::GetObjectResult AwsEnv::DoGetObjectWithTransferManager( - const Aws::String& bucket, const Aws::String& key, - const std::string& destination) { - CloudRequestCallbackGuard guard( - cloud_env_options.cloud_request_callback.get(), - CloudRequestOpType::kReadOp); - - auto transferHandle = - awsTransferManager_->DownloadFile(bucket, key, ToAwsString(destination)); - - transferHandle->WaitUntilFinished(); - - GetObjectResult result; - result.success = - transferHandle->GetStatus() == Aws::Transfer::TransferStatus::COMPLETED; - if (!result.success) { - result.error = transferHandle->GetLastError(); - } else { - result.objectSize = transferHandle->GetBytesTotalSize(); - } - - guard.SetSize(result.objectSize); - guard.SetSuccess(result.success); - return result; -} - -Status AwsEnv::GetObject(const std::string& bucket_name, - const std::string& object_path, - const std::string& local_destination) { - Env* localenv = GetBaseEnv(); - std::string tmp_destination = local_destination + ".tmp"; - - GetObjectResult result; - if (cloud_env_options.use_aws_transfer_manager) { - result = DoGetObjectWithTransferManager(ToAwsString(bucket_name), ToAwsString(object_path), tmp_destination); - } else { - result = DoGetObject(ToAwsString(bucket_name), ToAwsString(object_path), tmp_destination); - } - - if (!result.success) { - localenv->DeleteFile(tmp_destination); - const auto& error = result.error; - std::string errmsg(error.GetMessage().c_str(), error.GetMessage().size()); - Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] GetObject %s/%s error %s.", bucket_name.c_str(), - object_path.c_str(), errmsg.c_str()); - auto errorType = error.GetErrorType(); - if (errorType == Aws::S3::S3Errors::NO_SUCH_BUCKET || - errorType == Aws::S3::S3Errors::NO_SUCH_KEY || - errorType == Aws::S3::S3Errors::RESOURCE_NOT_FOUND) { - return Status::NotFound(std::move(errmsg)); - } - return Status::IOError(std::move(errmsg)); - } - - // Check if our local file is the same as S3 promised - uint64_t file_size{0}; - auto s = localenv->GetFileSize(tmp_destination, &file_size); - if (!s.ok()) { - return s; - } - if (file_size != result.objectSize) { - localenv->DeleteFile(tmp_destination); - s = Status::IOError("Partial download of a file " + local_destination); - Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] GetObject %s/%s local size %" PRIu64 - " != cloud size " - "%ld. %s", - bucket_name.c_str(), object_path.c_str(), file_size, result.objectSize, - s.ToString().c_str()); - } - - if (s.ok()) { - s = localenv->RenameFile(tmp_destination, local_destination); - } - Log(InfoLogLevel::INFO_LEVEL, info_log_, - "[s3] GetObject %s/%s size %" PRIu64 ". %s", bucket_name.c_str(), - object_path.c_str(), file_size, s.ToString().c_str()); - return s; -} - -AwsEnv::PutObjectResult AwsEnv::DoPutObject(const std::string& filename, - const Aws::String& bucket, - const Aws::String& key, - uint64_t sizeHint) { - auto inputData = Aws::MakeShared( - key.c_str(), filename.c_str(), std::ios_base::in | std::ios_base::out); - - Aws::S3::Model::PutObjectRequest putRequest; - putRequest.SetBucket(bucket); - putRequest.SetKey(key); - putRequest.SetBody(inputData); - SetEncryptionParameters(cloud_env_options, putRequest); - - auto putOutcome = s3client_->PutObject(putRequest, sizeHint); - PutObjectResult result; - result.success = putOutcome.IsSuccess(); - if (!result.success) { - result.error = putOutcome.GetError(); - } - return result; -} - -AwsEnv::PutObjectResult AwsEnv::DoPutObjectWithTransferManager( - const std::string& filename, const Aws::String& bucket, - const Aws::String& key, uint64_t sizeHint) { - CloudRequestCallbackGuard guard( - cloud_env_options.cloud_request_callback.get(), - CloudRequestOpType::kWriteOp, sizeHint); - - auto transferHandle = awsTransferManager_->UploadFile( - ToAwsString(filename), bucket, key, Aws::DEFAULT_CONTENT_TYPE, - Aws::Map()); - - transferHandle->WaitUntilFinished(); - - PutObjectResult result; - result.success = - transferHandle->GetStatus() == Aws::Transfer::TransferStatus::COMPLETED; - if (!result.success) { - result.error = transferHandle->GetLastError(); - } - - guard.SetSuccess(result.success); - return result; -} - -Status AwsEnv::PutObject(const std::string& local_file, - const std::string& bucket_name, - const std::string& object_path) { - uint64_t fsize = 0; - // debugging paranoia. Files uploaded to S3 can never be zero size. - auto st = GetBaseEnv()->GetFileSize(local_file, &fsize); - if (!st.ok()) { - Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] PutObject localpath %s error getting size %s", local_file.c_str(), - st.ToString().c_str()); - return st; - } - if (fsize == 0) { - Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] PutObject localpath %s error zero size", local_file.c_str()); - return Status::IOError(local_file + " Zero size."); - } - - auto s3_bucket = ToAwsString(bucket_name); - PutObjectResult result; - if (cloud_env_options.use_aws_transfer_manager) { - result = DoPutObjectWithTransferManager( - local_file, s3_bucket, ToAwsString(object_path), fsize); - } else { - result = DoPutObject(local_file, s3_bucket, ToAwsString(object_path), - fsize); - } - if (!result.success) { - auto error = result.error; - std::string errmsg(error.GetMessage().c_str(), error.GetMessage().size()); - st = Status::IOError(local_file, errmsg); - Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[s3] PutObject %s/%s, size %" PRIu64 ", ERROR %s", s3_bucket.c_str(), - object_path.c_str(), fsize, errmsg.c_str()); - } else { - Log(InfoLogLevel::INFO_LEVEL, info_log_, - "[s3] PutObject %s/%s, size %" PRIu64 ", OK", s3_bucket.c_str(), - object_path.c_str(), fsize); - } - - return st; -} - // // prepends the configured src object path name // @@ -1945,17 +1081,16 @@ std::string AwsEnv::destname(const std::string& localname) { return cloud_env_options.dest_bucket.GetObjectPath() + "/" + basename(localname); } -Status AwsEnv::LockFile(const std::string& fname, FileLock** lock) { +Status AwsEnv::LockFile(const std::string& /*fname*/, FileLock** lock) { // there isn's a very good way to atomically check and create // a file via libs3 *lock = nullptr; return Status::OK(); } -Status AwsEnv::UnlockFile(FileLock* lock) { return Status::OK(); } +Status AwsEnv::UnlockFile(FileLock* /*lock*/) { return Status::OK(); } -Status AwsEnv::NewLogger(const std::string& fname, - std::shared_ptr* result) { +Status AwsEnv::NewLogger(const std::string& fname, std::shared_ptr* result) { return base_env_->NewLogger(fname, result); } @@ -1969,20 +1104,46 @@ Status AwsEnv::NewAwsEnv(Env* base_env, if (!base_env) { base_env = Env::Default(); } - std::unique_ptr aenv(new AwsEnv(base_env, cloud_options, info_log)); - if (!aenv->status().ok()) { - status = aenv->status(); - } else { + // These lines of code are likely temporary until the new configuration stuff + // comes into play. + std::unique_ptr provider; + std::unique_ptr controller; + + status = CloudStorageProvider::CreateS3Provider(&provider); + // create cloud log client for storing/reading logs + if (status.ok() && !cloud_options.keep_local_log_files) { + if (cloud_options.log_type == kLogKinesis) { + status = CloudLogController::CreateKinesisController(&controller); + } else if (cloud_options.log_type == kLogKafka) { + status = CloudLogController::CreateKafkaController(&controller); + } else { + status = Status::NotSupported("We currently only support Kinesis and Kafka"); + Log(InfoLogLevel::ERROR_LEVEL, info_log, + "[aws] NewAwsEnv Unknown log type %d. %s", cloud_options.log_type, + status.ToString().c_str()); + } + } + if (!status.ok()) { + Log(InfoLogLevel::ERROR_LEVEL, info_log, + "[aws] NewAwsEnv Unable to create environment %s", + status.ToString().c_str()); + return status; + } + CloudEnvOptions options = cloud_options; // Make a copy + options.storage_provider.reset(provider.release()); + options.log_controller.reset(controller.release()); + std::unique_ptr aenv(new AwsEnv(base_env, options, info_log)); + status = aenv->Prepare(); + if (status.ok()) { *cenv = aenv.release(); } return status; } std::string AwsEnv::GetWALCacheDir() { - return cloud_log_controller_->GetCacheDir(); + return cloud_env_options.log_controller->GetCacheDir(); } - -#pragma GCC diagnostic pop #endif // USE_AWS } // namespace rocksdb + diff --git a/cloud/aws/aws_env.h b/cloud/aws/aws_env.h index 9248318cbf3..8af8b2bbea8 100644 --- a/cloud/aws/aws_env.h +++ b/cloud/aws/aws_env.h @@ -9,66 +9,15 @@ #include "cloud/cloud_env_impl.h" #include "port/sys_time.h" -#ifdef USE_AWS - -#include -#include -#include -#include -#include -#include -#include #include #include #include namespace rocksdb { +class ObjectLibrary; -class S3ReadableFile; - -class AwsS3ClientWrapper { - public: - AwsS3ClientWrapper( - std::shared_ptr client, - std::shared_ptr cloud_request_callback); - - Aws::S3::Model::ListObjectsOutcome ListObjects( - const Aws::S3::Model::ListObjectsRequest& request); - - Aws::S3::Model::CreateBucketOutcome CreateBucket( - const Aws::S3::Model::CreateBucketRequest& request); - - Aws::S3::Model::HeadBucketOutcome HeadBucket( - const Aws::S3::Model::HeadBucketRequest& request); - - Aws::S3::Model::DeleteObjectOutcome DeleteObject( - const Aws::S3::Model::DeleteObjectRequest& request); - - Aws::S3::Model::CopyObjectOutcome CopyObject( - const Aws::S3::Model::CopyObjectRequest& request); - - Aws::S3::Model::GetObjectOutcome GetObject( - const Aws::S3::Model::GetObjectRequest& request); - - Aws::S3::Model::PutObjectOutcome PutObject( - const Aws::S3::Model::PutObjectRequest& request, uint64_t size_hint = 0); - - Aws::S3::Model::HeadObjectOutcome HeadObject( - const Aws::S3::Model::HeadObjectRequest& request); - - const std::shared_ptr& GetClient() const { - return client_; - } - - private: - std::shared_ptr client_; - std::shared_ptr cloud_request_callback_; -}; - -namespace detail { -struct JobHandle; -} // namespace detail +#ifdef USE_AWS // // The S3 environment for rocksdb. This class overrides all the @@ -99,14 +48,19 @@ class AwsEnv : public CloudEnvImpl { static Status NewAwsEnv(Env* env, const CloudEnvOptions& env_options, const std::shared_ptr & info_log, CloudEnv** cenv); - + static Status CreateS3StorageProvider(std::unique_ptr* provider); + static Status CreateKinesisController(std::unique_ptr* controller); + explicit AwsEnv(Env* underlying_env, + const CloudEnvOptions& cloud_options, + const std::shared_ptr& logger); virtual ~AwsEnv(); + const char *Name() const override { return "AWS"; } // We cannot invoke Aws::ShutdownAPI from the destructor because there could // be // multiple AwsEnv's ceated by a process and Aws::ShutdownAPI should be called // only once by the entire process when all AwsEnvs are destroyed. - static void Shutdown() { Aws::ShutdownAPI(Aws::SDKOptions()); } + static void Shutdown(); // If you do not specify a region, then S3 buckets are created in the // standard-region which might not satisfy read-your-own-writes. So, @@ -230,16 +184,8 @@ class AwsEnv : public CloudEnvImpl { virtual uint64_t GetThreadID() const override { return AwsEnv::gettid(); } - virtual Status EmptyBucket(const std::string& bucket, - const std::string& path) override; - std::string GetWALCacheDir(); - // The S3 client - std::shared_ptr s3client_; - - // AWS's utility to help out with uploading and downloading S3 file - std::shared_ptr awsTransferManager_; // Saves and retrieves the dbid->dirname mapping in S3 Status SaveDbid(const std::string& bucket_name, const std::string& dbid, @@ -250,105 +196,18 @@ class AwsEnv : public CloudEnvImpl { DbidList* dblist) override; Status DeleteDbid(const std::string& bucket, const std::string& dbid) override; - Status ListObjects(const std::string& bucket_name, - const std::string& bucket_object, - BucketObjectMetadata* meta) override; - Status DeleteObject(const std::string& bucket_name, - const std::string& bucket_object_path) override; - Status ExistsObject(const std::string& bucket_name, - const std::string& bucket_object_path) override; - Status GetObjectSize(const std::string& bucket_name, - const std::string& bucket_object_path, - uint64_t* filesize) override; - Status CopyObject(const std::string& bucket_name_src, - const std::string& bucket_object_path_src, - const std::string& bucket_name_dest, - const std::string& bucket_object_path_dest) override; - Status GetObject(const std::string& bucket_name, - const std::string& bucket_object_path, - const std::string& local_path) override; - Status PutObject(const std::string& local_path, - const std::string& bucket_name, - const std::string& bucket_object_path) override; - Status DeleteCloudFileFromDest(const std::string& fname) override; - - void RemoveFileFromDeletionQueue(const std::string& filename); - - void TEST_SetFileDeletionDelay(std::chrono::seconds delay) { - std::lock_guard lk(files_to_delete_mutex_); - file_deletion_delay_ = delay; - } - + Status Prepare() override; +protected: private: - // - // The AWS credentials are specified to the constructor via - // access_key_id and secret_key. - // - explicit AwsEnv(Env* underlying_env, - const CloudEnvOptions& cloud_options, - const std::shared_ptr & info_log = nullptr); - - struct GetObjectResult { - bool success{false}; - Aws::Client::AWSError error; // if success == false - size_t objectSize{0}; - }; - - GetObjectResult DoGetObject(const Aws::String& bucket, const Aws::String& key, - const std::string& destination); - GetObjectResult DoGetObjectWithTransferManager( - const Aws::String& bucket, const Aws::String& key, - const std::string& destination); - - - struct PutObjectResult { - bool success{false}; - Aws::Client::AWSError error; // if success == false - }; - - PutObjectResult DoPutObject(const std::string& filename, - const Aws::String& bucket, const Aws::String& key, - uint64_t sizeHint); - - PutObjectResult DoPutObjectWithTransferManager(const std::string& filename, - const Aws::String& bucket, - const Aws::String& key, - uint64_t sizeHint); - // The pathname that contains a list of all db's inside a bucket. static constexpr const char* dbid_registry_ = "/.rockset/dbid/"; Status create_bucket_status_; - - std::mutex files_to_delete_mutex_; - std::chrono::seconds file_deletion_delay_ = std::chrono::hours(1); - std::unordered_map> - files_to_delete_; - - Aws::S3::Model::BucketLocationConstraint bucket_location_; - Status status(); - // Delete the specified path from S3 - Status DeletePathInS3(const std::string& bucket, - const std::string& fname); - // Validate options Status CheckOption(const EnvOptions& options); - // Return the list of children of the specified path - Status GetChildrenFromS3(const std::string& path, - const std::string& bucket, - std::vector* result); - - // If metadata, size or modtime is non-nullptr, returns requested data - Status HeadObject(const std::string& bucket, const std::string& path, - Aws::Map* metadata = nullptr, - uint64_t* size = nullptr, uint64_t* modtime = nullptr); - - Status NewS3ReadableFile(const std::string& bucket, const std::string& fname, - std::unique_ptr* result); - // Save IDENTITY file to S3. Update dbid registry. Status SaveIdentitytoS3(const std::string& localfile, const std::string& target_idfile); @@ -359,7 +218,9 @@ class AwsEnv : public CloudEnvImpl { // Converts a local pathname to an object name in the dest bucket std::string destname(const std::string& localname); }; +#endif // USE_AWS +extern "C" { +void RegisterAwsObjects(ObjectLibrary& library, const std::string& arg); +} // extern "C" } // namespace rocksdb - -#endif // USE_AWS diff --git a/cloud/aws/aws_file.h b/cloud/aws/aws_file.h index 33e0a59fdae..ebc4a69916c 100644 --- a/cloud/aws/aws_file.h +++ b/cloud/aws/aws_file.h @@ -3,153 +3,13 @@ #pragma once #ifdef USE_AWS -#include -#include -#include -#include "cloud/aws/aws_env.h" -#include "cloud/filename.h" -#include "file/filename.h" -#include "rocksdb/env.h" -#include "rocksdb/status.h" - #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include namespace rocksdb { inline Aws::String ToAwsString(const std::string& s) { return Aws::String(s.data(), s.size()); } -class S3ReadableFile : virtual public SequentialFile, - virtual public RandomAccessFile { - public: - S3ReadableFile(AwsEnv* env, const std::string& bucket_prefix, - const std::string& fname, uint64_t size); - - // sequential access, read data at current offset in file - virtual Status Read(size_t n, Slice* result, char* scratch) override; - - // random access, read data from specified offset in file - virtual Status Read(uint64_t offset, size_t n, Slice* result, - char* scratch) const override; - - virtual Status Skip(uint64_t n) override; - - virtual size_t GetUniqueId(char* id, size_t max_size) const override; - - private: - AwsEnv* env_; - std::string fname_; - Aws::String s3_bucket_; - Aws::String s3_object_; - uint64_t offset_; - uint64_t file_size_; -}; - -// Appends to a file in S3. -class S3WritableFile : public WritableFile { - private: - AwsEnv* env_; - std::string fname_; - std::string tmp_file_; - Status status_; - std::unique_ptr local_file_; - std::string bucket_prefix_; - std::string cloud_fname_; - bool is_manifest_; - - public: - // create S3 bucket - static Status CreateBucketInS3( - std::shared_ptr client, - const std::string& bucket_prefix, - const Aws::S3::Model::BucketLocationConstraint& location); - - // bucket exists and we can access it - static Status BucketExistsInS3( - std::shared_ptr client, - const std::string& bucket_prefix, - const Aws::S3::Model::BucketLocationConstraint& location); - - S3WritableFile(AwsEnv* env, const std::string& local_fname, - const std::string& bucket_prefix, - const std::string& cloud_fname, const EnvOptions& options, - const CloudEnvOptions cloud_env_options); - - virtual ~S3WritableFile(); - - virtual Status Append(const Slice& data) override { - assert(status_.ok()); - // write to temporary file - return local_file_->Append(data); - } - - Status PositionedAppend(const Slice& data, uint64_t offset) override { - return local_file_->PositionedAppend(data, offset); - } - Status Truncate(uint64_t size) override { - return local_file_->Truncate(size); - } - Status Fsync() override { return local_file_->Fsync(); } - bool IsSyncThreadSafe() const override { - return local_file_->IsSyncThreadSafe(); - } - bool use_direct_io() const override { return local_file_->use_direct_io(); } - size_t GetRequiredBufferAlignment() const override { - return local_file_->GetRequiredBufferAlignment(); - } - uint64_t GetFileSize() override { return local_file_->GetFileSize(); } - size_t GetUniqueId(char* id, size_t max_size) const override { - return local_file_->GetUniqueId(id, max_size); - } - Status InvalidateCache(size_t offset, size_t length) override { - return local_file_->InvalidateCache(offset, length); - } - Status RangeSync(uint64_t offset, uint64_t nbytes) override { - return local_file_->RangeSync(offset, nbytes); - } - Status Allocate(uint64_t offset, uint64_t len) override { - return local_file_->Allocate(offset, len); - } - - virtual Status Flush() override { - assert(status_.ok()); - return local_file_->Flush(); - } - - virtual Status Sync() override; - - virtual Status status() { return status_; } - - virtual Status Close() override; -}; - } // namepace rocksdb #endif /* USE_AWS */ diff --git a/cloud/aws/aws_kafka.cc b/cloud/aws/aws_kafka.cc index 45ce4973746..056cc8e2436 100644 --- a/cloud/aws/aws_kafka.cc +++ b/cloud/aws/aws_kafka.cc @@ -8,8 +8,8 @@ #include #include -#include "cloud/cloud_log_controller.h" #include "rocksdb/cloud/cloud_env_options.h" +#include "rocksdb/cloud/cloud_log_controller.h" #include "rocksdb/status.h" #include "util/coding.h" #include "util/stderr_logger.h" @@ -20,7 +20,7 @@ namespace rocksdb { namespace cloud { namespace kafka { - + /***************************************************/ /* KafkaWritableFile */ /***************************************************/ @@ -28,14 +28,15 @@ class KafkaWritableFile : public CloudLogWritableFile { public: static const std::chrono::microseconds kFlushTimeout; - KafkaWritableFile(CloudEnv* env, const std::string& fname, const EnvOptions& options, + KafkaWritableFile(CloudEnv* env, const std::string& fname, + const EnvOptions& options, std::shared_ptr producer, std::shared_ptr topic) - : CloudLogWritableFile(env, fname, options), - producer_(producer), - topic_(topic), - current_offset_(0) { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + : CloudLogWritableFile(env, fname, options), + producer_(producer), + topic_(topic), + current_offset_(0) { + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kafka] WritableFile opened file %s", fname_.c_str()); } @@ -58,7 +59,6 @@ class KafkaWritableFile : public CloudLogWritableFile { const std::chrono::microseconds KafkaWritableFile::kFlushTimeout = std::chrono::seconds(10); - Status KafkaWritableFile::ProduceRaw(const std::string& operation_name, const Slice& message) { if (!status_.ok()){ @@ -72,20 +72,20 @@ Status KafkaWritableFile::ProduceRaw(const std::string& operation_name, message.size(), &fname_ /* Partitioning key */, nullptr); if (resp == RdKafka::ERR_NO_ERROR) { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kafka] WritableFile %s file %s %ld", fname_.c_str(), operation_name.c_str(), message.size()); return Status::OK(); } else if (resp == RdKafka::ERR__QUEUE_FULL) { const std::string formatted_err = RdKafka::err2str(resp); - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kafka] WritableFile src %s %s error %s", fname_.c_str(), operation_name.c_str(), formatted_err.c_str()); return Status::Busy(topic_->name().c_str(), RdKafka::err2str(resp).c_str()); } else { const std::string formatted_err = RdKafka::err2str(resp); - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kafka] WritableFile src %s %s error %s", fname_.c_str(), operation_name.c_str(), formatted_err.c_str()); @@ -106,7 +106,7 @@ Status KafkaWritableFile::Append(const Slice& data) { } Status KafkaWritableFile::Close() { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kafka] S3WritableFile closing %s", fname_.c_str()); std::string serialized_data; @@ -132,7 +132,7 @@ Status KafkaWritableFile::Flush() { while (status_.ok() && !(done = (producer_->outq_len() == 0)) && !(timeout = (std::chrono::microseconds(env_->NowMicros()) - start > kFlushTimeout))) { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kafka] WritableFile src %s " "Waiting on flush: Output queue length: %d", fname_.c_str(), producer_->outq_len()); @@ -141,15 +141,15 @@ Status KafkaWritableFile::Flush() { } if (done) { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kafka] WritableFile src %s Flushed", fname_.c_str()); } else if (timeout) { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kafka] WritableFile src %s Flushing timed out after %" PRId64 "us", fname_.c_str(), kFlushTimeout.count()); status_ = Status::TimedOut(); } else { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kafka] WritableFile src %s Flush interrupted", fname_.c_str()); } @@ -157,7 +157,7 @@ Status KafkaWritableFile::Flush() { } Status KafkaWritableFile::LogDelete() { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, "[kafka] LogDelete %s", + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kafka] LogDelete %s", fname_.c_str()); std::string serialized_data; @@ -175,49 +175,18 @@ Status KafkaWritableFile::LogDelete() { // class KafkaController : public CloudLogController { public: - KafkaController(CloudEnv* env, - std::unique_ptr producer, - std::unique_ptr consumer) - : CloudLogController(env), - producer_(std::move(producer)), - consumer_(std::move(consumer)) { - const std::string topic_name = env_->GetSrcBucketName(); - - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[%s] KafkaController opening stream %s using cachedir '%s'", - Name(), topic_name.c_str(), cache_dir_.c_str()); - - std::string pt_errstr, ct_errstr; - - // Initialize stream name. - RdKafka::Topic* producer_topic = - RdKafka::Topic::create(producer_.get(), topic_name, NULL, pt_errstr); - - RdKafka::Topic* consumer_topic = - RdKafka::Topic::create(consumer_.get(), topic_name, NULL, ct_errstr); - - RdKafka::Queue* consuming_queue = RdKafka::Queue::create(consumer_.get()); - - assert(producer_topic != nullptr); - assert(consumer_topic != nullptr); - assert(consuming_queue != nullptr); - - consuming_queue_.reset(consuming_queue); - - producer_topic_.reset(producer_topic); - consumer_topic_.reset(consumer_topic); - } ~KafkaController() { - for (size_t i = 0; i < partitions_.size(); i++) { - consumer_->stop(consumer_topic_.get(), partitions_[i]->partition()); + if (consumer_) { + for (size_t i = 0; i < partitions_.size(); i++) { + consumer_->stop(consumer_topic_.get(), partitions_[i]->partition()); + } } - - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[%s] KafkaController closed.", Name()); } - - const char *Name() const override { return "kafka"; } + + const char* Name() const override { return "kafka"; } virtual Status CreateStream(const std::string& /* bucket_prefix */) override { // Kafka client cannot create a topic. Topics are either manually created @@ -225,16 +194,19 @@ class KafkaController : public CloudLogController { // true. return status_; } - virtual Status WaitForStreamReady(const std::string& /* bucket_prefix */) override { + virtual Status WaitForStreamReady( + const std::string& /* bucket_prefix */) override { // Kafka topics don't need to be waited on. return status_; } virtual Status TailStream() override; - virtual CloudLogWritableFile* CreateWritableFile(const std::string& fname, - const EnvOptions& options) override; - + virtual CloudLogWritableFile* CreateWritableFile( + const std::string& fname, const EnvOptions& options) override; + Status Verify() const override; +protected: + Status Initialize(CloudEnv *env) override; private: Status InitializePartitions(); @@ -249,6 +221,87 @@ class KafkaController : public CloudLogController { std::vector> partitions_; }; +Status KafkaController::Initialize(CloudEnv *env) { + Status status = CloudLogController::Initialize(env); + if (!status.ok()) { + return status; + } + std::string conf_errstr, producer_errstr, consumer_errstr; + const auto& kconf = + env->GetCloudEnvOptions().kafka_log_options.client_config_params; + if (kconf.empty()) { + return Status::InvalidArgument("No configs specified to kafka client"); + } + + std::unique_ptr conf(RdKafka::Conf::create(RdKafka::Conf::CONF_GLOBAL)); + + + for (auto const& item : kconf) { + if (conf->set(item.first, item.second, conf_errstr) != + RdKafka::Conf::CONF_OK) { + status = Status::InvalidArgument("Failed adding specified conf to Kafka conf", + conf_errstr.c_str()); + + Log(InfoLogLevel::ERROR_LEVEL, env->GetLogger(), + "[aws] NewAwsEnv Kafka conf set error: %s", status.ToString().c_str()); + return status; + } + } + + producer_.reset(RdKafka::Producer::create(conf.get(), producer_errstr)); + consumer_.reset(RdKafka::Consumer::create(conf.get(), consumer_errstr)); + + if (!producer_) { + status = Status::InvalidArgument("Failed creating Kafka producer", + producer_errstr.c_str()); + + Log(InfoLogLevel::ERROR_LEVEL, env->GetLogger(), + "[aws] NewAwsEnv Kafka producer error: %s", status.ToString().c_str()); + } else if (!consumer_) { + status = Status::InvalidArgument("Failed creating Kafka consumer", + consumer_errstr.c_str()); + + Log(InfoLogLevel::ERROR_LEVEL, env->GetLogger(), + "[aws] NewAwsEnv Kafka consumer error: %s", status.ToString().c_str()); + } else { + const std::string topic_name = env->GetSrcBucketName(); + + Log(InfoLogLevel::DEBUG_LEVEL, env->GetLogger(), + "[%s] KafkaController opening stream %s using cachedir '%s'", Name(), + topic_name.c_str(), cache_dir_.c_str()); + + std::string pt_errstr, ct_errstr; + + // Initialize stream name. + consuming_queue_.reset(RdKafka::Queue::create(consumer_.get())); + producer_topic_.reset(RdKafka::Topic::create(producer_.get(), topic_name, NULL, pt_errstr)); + consumer_topic_.reset(RdKafka::Topic::create(consumer_.get(), topic_name, NULL, ct_errstr)); + + assert(producer_topic_ != nullptr); + assert(consumer_topic_ != nullptr); + assert(consuming_queue_ != nullptr); + } + return status; +} + +Status KafkaController::Verify() const { + Status s = CloudLogController::Verify(); + if (s.ok()) { + if (!producer_) { + s = Status::InvalidArgument("Failed creating Kafka producer"); + } else if (!consumer_) { + s = Status::InvalidArgument("Failed creating Kafka consumer"); + } else if (producer_topic_ == nullptr) { + s = Status::InvalidArgument("Failed to initialize Kafa Producer Topic"); + } else if (consumer_topic_ == nullptr) { + s = Status::InvalidArgument("Failed to initialize Kafa Consumer Topic"); + } else if (consuming_queue_ == nullptr) { + s = Status::InvalidArgument("Failed to initialize Kafa Consuming Queue"); + } + } + return s; +} + Status KafkaController::TailStream() { InitializePartitions(); @@ -256,9 +309,8 @@ Status KafkaController::TailStream() { return status_; } - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, "[%s] TailStream topic %s %s", - Name(), consumer_topic_->name().c_str(), - status_.ToString().c_str()); + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[%s] TailStream topic %s %s", + Name(), consumer_topic_->name().c_str(), status_.ToString().c_str()); Status lastErrorStatus; int retryAttempt = 0; @@ -280,17 +332,17 @@ Status KafkaController::TailStream() { // Apply the payload to local filesystem status_ = Apply(sl); if (!status_.ok()) { - Log(InfoLogLevel::ERROR_LEVEL, env_->info_log_, + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), "[%s] error processing message size %ld " "extracted from stream %s %s", - Name(), message->len(), - consumer_topic_->name().c_str(), status_.ToString().c_str()); + Name(), message->len(), consumer_topic_->name().c_str(), + status_.ToString().c_str()); } else { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[%s] successfully processed message size %ld " "extracted from stream %s %s", - Name(), message->len(), - consumer_topic_->name().c_str(), status_.ToString().c_str()); + Name(), message->len(), consumer_topic_->name().c_str(), + status_.ToString().c_str()); } // Remember last read offset from topic (currently unused). @@ -307,9 +359,8 @@ Status KafkaController::TailStream() { Status::IOError(consumer_topic_->name().c_str(), RdKafka::err2str(message->err()).c_str()); - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[%s] error reading %s %s", Name(), - consumer_topic_->name().c_str(), + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), + "[%s] error reading %s %s", Name(), consumer_topic_->name().c_str(), RdKafka::err2str(message->err()).c_str()); ++retryAttempt; @@ -317,7 +368,7 @@ Status KafkaController::TailStream() { } } } - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[%s] TailStream topic %s finished: %s", Name(), consumer_topic_->name().c_str(), status_.ToString().c_str()); @@ -339,10 +390,9 @@ Status KafkaController::InitializePartitions() { status_ = Status::IOError(consumer_topic_->name().c_str(), RdKafka::err2str(err).c_str()); - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[%s] S3ReadableFile file %s Unable to find shards %s", - Name(), consumer_topic_->name().c_str(), - status_.ToString().c_str()); + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), + "[%s] S3ReadableFile file %s Unable to find shards %s", Name(), + consumer_topic_->name().c_str(), status_.ToString().c_str()); return status_; } @@ -392,71 +442,15 @@ CloudLogWritableFile* KafkaController::CreateWritableFile( #endif /* USE_KAFKA */ namespace rocksdb { +Status CloudLogController::CreateKafkaController(std::unique_ptr* output) { #ifndef USE_KAFKA -Status CreateKafkaController(CloudEnv *, - std::unique_ptr *) { - return Status::NotSupported("In order to use Kafka, make sure you're compiling with " - "USE_KAFKA=1"); -} + output->reset(); + return Status::NotSupported( + "In order to use Kafka, make sure you're compiling with " + "USE_KAFKA=1"); #else -Status CreateKafkaController(CloudEnv *env, - std::unique_ptr *output) { - Status st = Status::OK(); - std::string conf_errstr, producer_errstr, consumer_errstr; - const auto& kconf = env->GetCloudEnvOptions().kafka_log_options.client_config_params; - - std::unique_ptr conf( - RdKafka::Conf::create(RdKafka::Conf::CONF_GLOBAL)); - - if (kconf.empty()) { - st = Status::InvalidArgument("No configs specified to kafka client"); - - return st; - } - - for (auto const& item : kconf) { - if (conf->set(item.first, - item.second, - conf_errstr) != RdKafka::Conf::CONF_OK) { - st = Status::InvalidArgument("Failed adding specified conf to Kafka conf", - conf_errstr.c_str()); - - Log(InfoLogLevel::ERROR_LEVEL, env->info_log_, - "[aws] NewAwsEnv Kafka conf set error: %s", st.ToString().c_str()); - return st; - } - } - - { - std::unique_ptr producer( - RdKafka::Producer::create(conf.get(), producer_errstr)); - std::unique_ptr consumer( - RdKafka::Consumer::create(conf.get(), consumer_errstr)); - - if (!producer) { - st = Status::InvalidArgument("Failed creating Kafka producer", - producer_errstr.c_str()); - - Log(InfoLogLevel::ERROR_LEVEL, env->info_log_, - "[aws] NewAwsEnv Kafka producer error: %s", st.ToString().c_str()); - } else if (!consumer) { - st = Status::InvalidArgument("Failed creating Kafka consumer", - consumer_errstr.c_str()); - - Log(InfoLogLevel::ERROR_LEVEL, env->info_log_, - "[aws] NewAwsEnv Kafka consumer error: %s", st.ToString().c_str()); - } else { - output->reset(new rocksdb::cloud::kafka::KafkaController(env, - std::move(producer), - std::move(consumer))); - - if (output->get() == nullptr) { - st = Status::IOError("Error in creating Kafka controller"); - } - } - } - return st; + output->reset(new rocksdb::cloud::kafka::KafkaController()); + return Status::OK(); +#endif // USE_KAFKA } -#endif // USE_KAFKA } // namespace rocksdb - diff --git a/cloud/aws/aws_kinesis.cc b/cloud/aws/aws_kinesis.cc index 5ed843d4660..36be2eb5cb2 100644 --- a/cloud/aws/aws_kinesis.cc +++ b/cloud/aws/aws_kinesis.cc @@ -7,8 +7,9 @@ #include #include -#include "cloud/cloud_log_controller.h" +#include "cloud/aws/aws_env.h" #include "rocksdb/cloud/cloud_env_options.h" +#include "rocksdb/cloud/cloud_log_controller.h" #include "rocksdb/status.h" #include "util/coding.h" #include "util/stderr_logger.h" @@ -35,19 +36,19 @@ namespace rocksdb { namespace cloud { namespace kinesis { - + /***************************************************/ /* KinesisWritableFile */ /***************************************************/ class KinesisWritableFile : public CloudLogWritableFile { public: - KinesisWritableFile(CloudEnv* env, const std::string& fname, - const EnvOptions& options, - const std::shared_ptr & kinesis_client) - : CloudLogWritableFile(env, fname, options), - kinesis_client_(kinesis_client), current_offset_(0) { - - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + KinesisWritableFile( + CloudEnv* env, const std::string& fname, const EnvOptions& options, + const std::shared_ptr& kinesis_client) + : CloudLogWritableFile(env, fname, options), + kinesis_client_(kinesis_client), + current_offset_(0) { + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kinesis] WritableFile opened file %s", fname_.c_str()); std::string bucket = env_->GetSrcBucketName(); topic_ = Aws::String(bucket.c_str(), bucket.size()); @@ -80,25 +81,26 @@ Status KinesisWritableFile::Append(const Slice& data) { buffer.size())); // write to stream - const auto & outcome = kinesis_client_->PutRecord(request); + const auto& outcome = kinesis_client_->PutRecord(request); bool isSuccess = outcome.IsSuccess(); if (!isSuccess) { - const Aws::Client::AWSError& error = outcome.GetError(); - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + const Aws::Client::AWSError& error = + outcome.GetError(); + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kinesis] NewWritableFile src %s Append error %s", fname_.c_str(), error.GetMessage().c_str()); return Status::IOError(fname_, error.GetMessage().c_str()); } current_offset_ += data.size(); - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kinesis] WritableFile Append file %s %ld %ld", fname_.c_str(), data.size(), buffer.size()); return Status::OK(); } Status KinesisWritableFile::Close() { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kinesis] S3WritableFile closing %s", fname_.c_str()); assert(status_.ok()); @@ -114,18 +116,19 @@ Status KinesisWritableFile::Close() { buffer.size())); // write to stream - const auto & outcome = kinesis_client_->PutRecord(request); + const auto& outcome = kinesis_client_->PutRecord(request); bool isSuccess = outcome.IsSuccess(); if (!isSuccess) { - const Aws::Client::AWSError& error = outcome.GetError(); - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + const Aws::Client::AWSError& error = + outcome.GetError(); + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kinesis] NewWritableFile src %s Close error %s", fname_.c_str(), error.GetMessage().c_str()); return Status::IOError(fname_, error.GetMessage().c_str()); } current_offset_ += buffer.size(); - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kinesis] WritableFile Close file %s %ld", fname_.c_str(), buffer.size()); return Status::OK(); @@ -135,7 +138,7 @@ Status KinesisWritableFile::Close() { // Log a delete record to stream // Status KinesisWritableFile::LogDelete() { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, "[kinesis] LogDelete %s", + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kinesis] LogDelete %s", fname_.c_str()); assert(status_.ok()); @@ -151,17 +154,19 @@ Status KinesisWritableFile::LogDelete() { buffer.size())); // write to stream - const Aws::Kinesis::Model::PutRecordOutcome& outcome = kinesis_client_->PutRecord(request); + const Aws::Kinesis::Model::PutRecordOutcome& outcome = + kinesis_client_->PutRecord(request); bool isSuccess = outcome.IsSuccess(); if (!isSuccess) { - const Aws::Client::AWSError& error = outcome.GetError(); - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + const Aws::Client::AWSError& error = + outcome.GetError(); + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kinesis] LogDelete src %s error %s", fname_.c_str(), error.GetMessage().c_str()); return Status::IOError(fname_, error.GetMessage().c_str()); } - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[kinesis] LogDelete file %s %ld", fname_.c_str(), buffer.size()); return Status::OK(); } @@ -169,34 +174,16 @@ Status KinesisWritableFile::LogDelete() { /***************************************************/ /* KinesisController */ /***************************************************/ - + // // Intricacies of reading a Kinesis stream // class KinesisController : public CloudLogController { public: - KinesisController(CloudEnv* env, - const std::shared_ptr & provider, - const Aws::Client::ClientConfiguration & config) - : CloudLogController(env) { - kinesis_client_.reset(provider - ? new Aws::Kinesis::KinesisClient(provider, config) - : new Aws::Kinesis::KinesisClient(config)); - // Initialize stream name. - std::string bucket = env_->GetSrcBucketName(); - topic_ = Aws::String(bucket.c_str(), bucket.size()); - - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[%s] KinesisController opening stream %s using cachedir '%s'", - Name(), topic_.c_str(), cache_dir_.c_str()); - } - virtual ~KinesisController() { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[%s] KinesisController closed", Name()); } - const char *Name() const override { return "kinesis"; } + const char* Name() const override { return "kinesis"; } Status CreateStream(const std::string& bucket) override; Status WaitForStreamReady(const std::string& bucket) override; @@ -204,9 +191,12 @@ class KinesisController : public CloudLogController { CloudLogWritableFile* CreateWritableFile(const std::string& fname, const EnvOptions& options) override; + Status Verify() const override; +protected: + Status Initialize(CloudEnv *env) override; private: std::shared_ptr kinesis_client_; - + Aws::String topic_; // list of shards and their positions @@ -221,11 +211,46 @@ class KinesisController : public CloudLogController { void SeekShards(); }; +Status KinesisController::Initialize(CloudEnv *env) { + Status status = CloudLogController::Initialize(env); + if (status.ok()) { + Aws::Client::ClientConfiguration config; + const auto &options = env->GetCloudEnvOptions(); + std::shared_ptr provider; + status = options.credentials.GetCredentialsProvider(&provider); + if (status.ok()) { + status = AwsCloudOptions::GetClientConfiguration(env, options.src_bucket.GetRegion(), &config); + } + if (status.ok()) { + kinesis_client_.reset(provider ? new Aws::Kinesis::KinesisClient(provider, config) : + new Aws::Kinesis::KinesisClient(config)); + // Initialize stream name. + std::string bucket = env->GetSrcBucketName(); + topic_ = Aws::String(bucket.c_str(), bucket.size()); + + Log(InfoLogLevel::DEBUG_LEVEL, env->GetLogger(), + "[%s] KinesisController opening stream %s using cachedir '%s'", Name(), + topic_.c_str(), cache_dir_.c_str()); + } + } + return status; + +} + +Status KinesisController::Verify() const { + Status s = CloudLogController::Verify(); + if (s.ok()) { + if (!kinesis_client_) { + s = Status::InvalidArgument("Failed to initialize kinesis client"); + } + } + return s; +} + Status KinesisController::TailStream() { status_ = InitializeShards(); - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[%s] TailStream topic %s %s", + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[%s] TailStream topic %s %s", Name(), topic_.c_str(), status_.ToString().c_str()); Status lastErrorStatus; @@ -242,21 +267,23 @@ Status KinesisController::TailStream() { // Issue a read from Kinesis stream Aws::Kinesis::Model::GetRecordsRequest request; request.SetShardIterator(shards_iterator_[0]); - Aws::Kinesis::Model::GetRecordsOutcome outcome = kinesis_client_->GetRecords(request); + Aws::Kinesis::Model::GetRecordsOutcome outcome = + kinesis_client_->GetRecords(request); bool isSuccess = outcome.IsSuccess(); if (!isSuccess) { - const Aws::Client::AWSError& error = outcome.GetError(); + const Aws::Client::AWSError& error = + outcome.GetError(); Aws::Kinesis::KinesisErrors err = error.GetErrorType(); if (err == Aws::Kinesis::KinesisErrors::EXPIRED_ITERATOR) { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[%s] expired shard iterator for %s. Reseeking...", Name(), topic_.c_str()); shards_iterator_[0] = ""; SeekShards(); // read position at last seqno } else { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[%s] error reading %s %s", - Name(), topic_.c_str(), error.GetMessage().c_str()); + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), + "[%s] error reading %s %s", Name(), topic_.c_str(), + error.GetMessage().c_str()); lastErrorStatus = Status::IOError(topic_.c_str(), error.GetMessage().c_str()); ++retryAttempt; @@ -283,15 +310,15 @@ Status KinesisController::TailStream() { // apply the payload to local filesystem status_ = Apply(sl); if (!status_.ok()) { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[%s] error processing message size %ld " - "extracted from stream %s %s", Name(), - b.GetLength(), topic_.c_str(), status_.ToString().c_str()); + "extracted from stream %s %s", + Name(), b.GetLength(), topic_.c_str(), status_.ToString().c_str()); } else { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[%s] successfully processed message size %ld " - "extracted from stream %s %s", Name(), - b.GetLength(), topic_.c_str(), status_.ToString().c_str()); + "extracted from stream %s %s", + Name(), b.GetLength(), topic_.c_str(), status_.ToString().c_str()); } // remember last read seqno from stream @@ -304,7 +331,7 @@ Status KinesisController::TailStream() { } return status_; } - + Status KinesisController::CreateStream(const std::string& bucket) { Aws::String topic = Aws::String(bucket.c_str(), bucket.size()); @@ -312,11 +339,13 @@ Status KinesisController::CreateStream(const std::string& bucket) { Aws::Kinesis::Model::CreateStreamRequest create_request; create_request.SetStreamName(topic); create_request.SetShardCount(1); - Aws::Kinesis::Model::CreateStreamOutcome outcome = kinesis_client_->CreateStream(create_request); + Aws::Kinesis::Model::CreateStreamOutcome outcome = + kinesis_client_->CreateStream(create_request); bool isSuccess = outcome.IsSuccess(); Status st; if (!isSuccess) { - const Aws::Client::AWSError& error = outcome.GetError(); + const Aws::Client::AWSError& error = + outcome.GetError(); std::string errmsg(error.GetMessage().c_str()); if (errmsg.find("already exists") == std::string::npos) { st = Status::IOError(topic.c_str(), errmsg); @@ -341,16 +370,20 @@ Status KinesisController::WaitForStreamReady(const std::string& bucket) { st = Status::OK(); Aws::Kinesis::Model::DescribeStreamRequest request; request.SetStreamName(topic); - Aws::Kinesis::Model::DescribeStreamOutcome outcome = kinesis_client_->DescribeStream(request); + Aws::Kinesis::Model::DescribeStreamOutcome outcome = + kinesis_client_->DescribeStream(request); isSuccess = outcome.IsSuccess(); if (!isSuccess) { - const Aws::Client::AWSError& error = outcome.GetError(); + const Aws::Client::AWSError& error = + outcome.GetError(); st = Status::IOError(topic.c_str(), error.GetMessage().c_str()); - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, " Waiting for stream ready %s", - st.ToString().c_str()); + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), + " Waiting for stream ready %s", st.ToString().c_str()); } else { - const Aws::Kinesis::Model::DescribeStreamResult& result = outcome.GetResult(); - const Aws::Kinesis::Model::StreamDescription& description = result.GetStreamDescription(); + const Aws::Kinesis::Model::DescribeStreamResult& result = + outcome.GetResult(); + const Aws::Kinesis::Model::StreamDescription& description = + result.GetStreamDescription(); auto& shards = description.GetShards(); if (shards.size() != 1) { isSuccess = false; @@ -383,14 +416,17 @@ Status KinesisController::InitializeShards() { auto outcome = kinesis_client_->DescribeStream(request); bool isSuccess = outcome.IsSuccess(); if (!isSuccess) { - const Aws::Client::AWSError& error = outcome.GetError(); + const Aws::Client::AWSError& error = + outcome.GetError(); st = Status::IOError(topic_.c_str(), error.GetMessage().c_str()), - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[%s] S3ReadableFile file %s Unable to find shards %s", - Name(), topic_.c_str(), st.ToString().c_str()); + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), + "[%s] S3ReadableFile file %s Unable to find shards %s", Name(), + topic_.c_str(), st.ToString().c_str()); } else { - const Aws::Kinesis::Model::DescribeStreamResult& result = outcome.GetResult(); - const Aws::Kinesis::Model::StreamDescription& description = result.GetStreamDescription(); + const Aws::Kinesis::Model::DescribeStreamResult& result = + outcome.GetResult(); + const Aws::Kinesis::Model::StreamDescription& description = + result.GetStreamDescription(); // append all shards to global list auto& shards = description.GetShards(); @@ -418,22 +454,26 @@ void KinesisController::SeekShards() { request.SetStreamName(topic_); request.SetShardId(shards_[i].GetShardId()); if (shards_position_[i].size() == 0) { - request.SetShardIteratorType(Aws::Kinesis::Model::ShardIteratorType::TRIM_HORIZON); + request.SetShardIteratorType( + Aws::Kinesis::Model::ShardIteratorType::TRIM_HORIZON); } else { - request.SetShardIteratorType(Aws::Kinesis::Model::ShardIteratorType::AFTER_SEQUENCE_NUMBER); + request.SetShardIteratorType( + Aws::Kinesis::Model::ShardIteratorType::AFTER_SEQUENCE_NUMBER); request.SetStartingSequenceNumber(shards_position_[i]); } Aws::Kinesis::Model::GetShardIteratorOutcome outcome = - kinesis_client_->GetShardIterator(request); + kinesis_client_->GetShardIterator(request); bool isSuccess = outcome.IsSuccess(); if (!isSuccess) { - const Aws::Client::AWSError& error = outcome.GetError(); + const Aws::Client::AWSError& error = + outcome.GetError(); status_ = Status::IOError(topic_.c_str(), error.GetMessage().c_str()); - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[%s] S3ReadableFile file %s Unable to find shards %s", - Name(), topic_.c_str(), status_.ToString().c_str()); + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), + "[%s] S3ReadableFile file %s Unable to find shards %s", Name(), + topic_.c_str(), status_.ToString().c_str()); } else { - const Aws::Kinesis::Model::GetShardIteratorResult& result = outcome.GetResult(); + const Aws::Kinesis::Model::GetShardIteratorResult& result = + outcome.GetResult(); shards_iterator_[i] = result.GetShardIterator(); } } @@ -445,33 +485,19 @@ CloudLogWritableFile* KinesisController::CreateWritableFile( new KinesisWritableFile(env_, fname, options, kinesis_client_)); } -} // namespace aws +} // namespace kinesis } // namespace cloud } // namespace rocksdb #endif /* USE_AWS */ namespace rocksdb { +Status CloudLogController::CreateKinesisController(std::unique_ptr *output) { #ifndef USE_AWS -Status CreateKinesisController(CloudEnv *, - std::unique_ptr *) { - return Status::NotSupported("In order to use Kinesis, make sure you're compiling with " - "USE_AWS=1"); -} + output->reset(); + return Status::NotSupported("In order to use Kinesis, make sure you're compiling with USE_AWS=1"); #else -Status CreateKinesisController(CloudEnv *env, - std::unique_ptr *output) { - Aws::Client::ClientConfiguration config; - const auto & options = env->GetCloudEnvOptions(); - std::shared_ptr provider; - Status st = options.credentials.GetCredentialsProvider(&provider); - if (st.ok()) { - st = AwsCloudOptions::GetClientConfiguration(env, - options.src_bucket.GetRegion(), &config); - } - if (st.ok()) { - output->reset(new rocksdb::cloud::kinesis::KinesisController(env, provider, config)); - } - return st; -} + output->reset(new rocksdb::cloud::kinesis::KinesisController()); + return Status::OK(); #endif /* USE_AWS */ -} // namespace rocksdb +} +} // namespace rocksdb diff --git a/cloud/aws/aws_retry.cc b/cloud/aws/aws_retry.cc index 20a239c5151..b57bcbda09b 100644 --- a/cloud/aws/aws_retry.cc +++ b/cloud/aws/aws_retry.cc @@ -11,7 +11,7 @@ #include #include #include -#endif // USE_AWS +#endif // USE_AWS namespace rocksdb { #ifdef USE_AWS @@ -20,28 +20,28 @@ namespace rocksdb { // class AwsRetryStrategy : public Aws::Client::RetryStrategy { public: - AwsRetryStrategy(CloudEnv *env) : env_(env) { + AwsRetryStrategy(CloudEnv* env) : env_(env) { default_strategy_ = std::make_shared(); - Log(InfoLogLevel::INFO_LEVEL, env_->info_log_, + Log(InfoLogLevel::INFO_LEVEL, env_->GetLogger(), "[aws] Configured custom retry policy"); } - - ~AwsRetryStrategy() override { } - + + ~AwsRetryStrategy() override {} // Returns true if the error can be retried given the error and the number of // times already tried. bool ShouldRetry(const Aws::Client::AWSError& error, long attemptedRetries) const override; - + // Calculates the time in milliseconds the client should sleep before // attempting another request based on the error and attemptedRetries count. - long CalculateDelayBeforeNextRetry(const Aws::Client::AWSError& error, - long attemptedRetries) const override; + long CalculateDelayBeforeNextRetry( + const Aws::Client::AWSError& error, + long attemptedRetries) const override; private: // rocksdb retries, etc - CloudEnv *env_; + CloudEnv* env_; // The default strategy implemented by AWS client std::shared_ptr default_strategy_; @@ -54,8 +54,9 @@ class AwsRetryStrategy : public Aws::Client::RetryStrategy { // Returns true if the error can be retried given the error and the number of // times already tried. // -bool AwsRetryStrategy::ShouldRetry(const Aws::Client::AWSError& error, - long attemptedRetries) const { +bool AwsRetryStrategy::ShouldRetry( + const Aws::Client::AWSError& error, + long attemptedRetries) const { auto ce = error.GetErrorType(); const Aws::String errmsg = error.GetMessage(); const Aws::String exceptionMsg = error.GetExceptionName(); @@ -68,7 +69,7 @@ bool AwsRetryStrategy::ShouldRetry(const Aws::Client::AWSErrorinfo_log_, + Log(InfoLogLevel::INFO_LEVEL, env_->GetLogger(), "[aws] Encountered retriable failure: %s (code %d, http %d). " "Exception %s. retry attempt %ld is lesser than max retries %d. " "Retrying...", @@ -77,7 +78,7 @@ bool AwsRetryStrategy::ShouldRetry(const Aws::Client::AWSErrorinfo_log_, + Log(InfoLogLevel::INFO_LEVEL, env_->GetLogger(), "[aws] Encountered retriable failure: %s (code %d, http %d). Exception " "%s. retry attempt %ld exceeds max retries %d. Aborting...", err.c_str(), static_cast(ce), @@ -85,7 +86,7 @@ bool AwsRetryStrategy::ShouldRetry(const Aws::Client::AWSErrorinfo_log_, + Log(InfoLogLevel::WARN_LEVEL, env_->GetLogger(), "[aws] Encountered S3 failure %s (code %d, http %d). Exception %s." " retry attempt %ld max retries %d. Using default retry policy...", err.c_str(), static_cast(ce), @@ -99,18 +100,19 @@ bool AwsRetryStrategy::ShouldRetry(const Aws::Client::AWSError& error, long attemptedRetries) const { + const Aws::Client::AWSError& error, + long attemptedRetries) const { return default_strategy_->CalculateDelayBeforeNextRetry(error, attemptedRetries); } -Status AwsCloudOptions::GetClientConfiguration(CloudEnv *env, - const std::string& region, - Aws::Client::ClientConfiguration* config) { +Status AwsCloudOptions::GetClientConfiguration( + CloudEnv* env, const std::string& region, + Aws::Client::ClientConfiguration* config) { config->connectTimeoutMs = 30000; config->requestTimeoutMs = 600000; - - const auto & cloud_env_options = env->GetCloudEnvOptions(); + + const auto& cloud_env_options = env->GetCloudEnvOptions(); // Setup how retries need to be done config->retryStrategy = std::make_shared(env); if (cloud_env_options.request_timeout_ms != 0) { @@ -121,12 +123,10 @@ Status AwsCloudOptions::GetClientConfiguration(CloudEnv *env, return Status::OK(); } #else -Status AwsCloudOptions::GetClientConfiguration(CloudEnv *, - const std::string& , - Aws::Client::ClientConfiguration*) { +Status AwsCloudOptions::GetClientConfiguration( + CloudEnv*, const std::string&, Aws::Client::ClientConfiguration*) { return Status::NotSupported("Not configured for AWS support"); } #endif /* USE_AWS */ - -} // namespace +} // namespace rocksdb diff --git a/cloud/aws/aws_s3.cc b/cloud/aws/aws_s3.cc index 0bfc5d8b3d6..81ed572db5a 100644 --- a/cloud/aws/aws_s3.cc +++ b/cloud/aws/aws_s3.cc @@ -5,196 +5,479 @@ // A sst file maps to an object in that S3 bucket. // #ifdef USE_AWS - #include #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + #include "cloud/aws/aws_env.h" #include "cloud/aws/aws_file.h" +#include "cloud/cloud_storage_provider_impl.h" +#include "cloud/filename.h" +#include "rocksdb/cloud/cloud_env_options.h" +#include "rocksdb/cloud/cloud_storage_provider.h" +#include "rocksdb/options.h" #include "util/coding.h" #include "util/stderr_logger.h" #include "util/string_util.h" namespace rocksdb { -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" - -/******************** Readablefile ******************/ - -S3ReadableFile::S3ReadableFile(AwsEnv* env, const std::string& bucket, - const std::string& fname, uint64_t file_size) - : env_(env), fname_(fname), offset_(0), file_size_(file_size) { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[s3] S3ReadableFile opening file %s", fname_.c_str()); - s3_bucket_ = ToAwsString(bucket); - s3_object_ = ToAwsString(fname_); -} +class CloudRequestCallbackGuard { + public: + CloudRequestCallbackGuard(CloudRequestCallback* callback, + CloudRequestOpType type, uint64_t size = 0) + : callback_(callback), type_(type), size_(size), start_(now()) {} + + ~CloudRequestCallbackGuard() { + if (callback_) { + (*callback_)(type_, size_, now() - start_, success_); + } + } -// sequential access, read data at current offset in file -Status S3ReadableFile::Read(size_t n, Slice* result, char* scratch) { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[s3] S3ReadableFile reading %s %ld", fname_.c_str(), n); - Status s = Read(offset_, n, result, scratch); + void SetSize(uint64_t size) { size_ = size; } + void SetSuccess(bool success) { success_ = success; } - // If the read successfully returned some data, then update - // offset_ - if (s.ok()) { - offset_ += result->size(); + private: + uint64_t now() { + return std::chrono::duration_cast( + std::chrono::system_clock::now() - + std::chrono::system_clock::from_time_t(0)) + .count(); + } + CloudRequestCallback* callback_; + CloudRequestOpType type_; + uint64_t size_; + bool success_{false}; + uint64_t start_; +}; + +template +void SetEncryptionParameters(const CloudEnvOptions& cloud_env_options, + T& put_request) { + if (cloud_env_options.server_side_encryption) { + if (cloud_env_options.encryption_key_id.empty()) { + put_request.SetServerSideEncryption( + Aws::S3::Model::ServerSideEncryption::AES256); + } else { + put_request.SetServerSideEncryption( + Aws::S3::Model::ServerSideEncryption::aws_kms); + put_request.SetSSEKMSKeyId(cloud_env_options.encryption_key_id.c_str()); + } } - return s; } -// random access, read data from specified offset in file -Status S3ReadableFile::Read(uint64_t offset, size_t n, Slice* result, - char* scratch) const { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[s3] S3ReadableFile reading %s at offset %" PRIu64 - " size %" ROCKSDB_PRIszt, - fname_.c_str(), offset, n); - - *result = Slice(); - - if (offset >= file_size_) { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[s3] S3ReadableFile reading %s at offset %" PRIu64 " filesize %" PRIu64 - "." - " Nothing to do", - fname_.c_str(), offset, file_size_); - return Status::OK(); +/******************** S3ClientWrapper ******************/ + +class AwsS3ClientWrapper { + public: + AwsS3ClientWrapper( + const std::shared_ptr& creds, + const Aws::Client::ClientConfiguration& config, + const CloudEnvOptions& cloud_options) + : cloud_request_callback_(cloud_options.cloud_request_callback) { + if (creds) { + client_ = std::make_shared(creds, config); + } else { + client_ = std::make_shared(config); + } + if (cloud_options.use_aws_transfer_manager) { + Aws::Transfer::TransferManagerConfiguration transferManagerConfig( + GetAwsTransferManagerExecutor()); + transferManagerConfig.s3Client = client_; + SetEncryptionParameters(cloud_options, + transferManagerConfig.putObjectTemplate); + SetEncryptionParameters( + cloud_options, transferManagerConfig.createMultipartUploadTemplate); + transfer_manager_ = + Aws::Transfer::TransferManager::Create(transferManagerConfig); + } } - // trim size if needed - if (offset + n > file_size_) { - n = file_size_ - offset; - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[s3] S3ReadableFile reading %s at offset %" PRIu64 - " trimmed size %" ROCKSDB_PRIszt, - fname_.c_str(), offset, n); - } - - // create a range read request - // Ranges are inclusive, so we can't read 0 bytes; read 1 instead and - // drop it later. - size_t rangeLen = (n != 0 ? n : 1); - char buffer[512]; - int ret = snprintf(buffer, sizeof(buffer), "bytes=%" PRIu64 "-%" PRIu64, - offset, offset + rangeLen - 1); - if (ret < 0) { - Log(InfoLogLevel::ERROR_LEVEL, env_->info_log_, - "[s3] S3ReadableFile vsnprintf error %s offset %" PRIu64 - " rangelen %" ROCKSDB_PRIszt "\n", - fname_.c_str(), offset, rangeLen); - return Status::IOError("S3ReadableFile vsnprintf ", fname_.c_str()); - } - Aws::String range(buffer); - - // set up S3 request to read this range - Aws::S3::Model::GetObjectRequest request; - request.SetBucket(s3_bucket_); - request.SetKey(s3_object_); - request.SetRange(range); - - Aws::S3::Model::GetObjectOutcome outcome = - env_->s3client_->GetObject(request); - bool isSuccess = outcome.IsSuccess(); - if (!isSuccess) { - const Aws::Client::AWSError& error = outcome.GetError(); - std::string errmsg(error.GetMessage().c_str(), error.GetMessage().size()); - Aws::S3::S3Errors s3err = error.GetErrorType(); - if (s3err == Aws::S3::S3Errors::NO_SUCH_BUCKET || - s3err == Aws::S3::S3Errors::NO_SUCH_KEY || - s3err == Aws::S3::S3Errors::RESOURCE_NOT_FOUND || - errmsg.find("Response code: 404") != std::string::npos) { - Log(InfoLogLevel::ERROR_LEVEL, env_->info_log_, - "[s3] S3ReadableFile error in reading not-existent %s %s", - fname_.c_str(), errmsg.c_str()); - return Status::NotFound(fname_, errmsg.c_str()); + Aws::S3::Model::ListObjectsOutcome ListObjects( + const Aws::S3::Model::ListObjectsRequest& request) { + CloudRequestCallbackGuard t(cloud_request_callback_.get(), + CloudRequestOpType::kListOp); + auto outcome = client_->ListObjects(request); + t.SetSuccess(outcome.IsSuccess()); + return outcome; + } + + Aws::S3::Model::CreateBucketOutcome CreateBucket( + const Aws::S3::Model::CreateBucketRequest& request) { + CloudRequestCallbackGuard t(cloud_request_callback_.get(), + CloudRequestOpType::kCreateOp); + auto outcome = client_->CreateBucket(request); + t.SetSuccess(outcome.IsSuccess()); + return outcome; + } + + Aws::S3::Model::HeadBucketOutcome HeadBucket( + const Aws::S3::Model::HeadBucketRequest& request) { + CloudRequestCallbackGuard t(cloud_request_callback_.get(), + CloudRequestOpType::kInfoOp); + auto outcome = client_->HeadBucket(request); + t.SetSuccess(outcome.IsSuccess()); + return outcome; + } + Aws::S3::Model::DeleteObjectOutcome DeleteObject( + const Aws::S3::Model::DeleteObjectRequest& request) { + CloudRequestCallbackGuard t(cloud_request_callback_.get(), + CloudRequestOpType::kDeleteOp); + auto outcome = client_->DeleteObject(request); + t.SetSuccess(outcome.IsSuccess()); + return outcome; + } + + Aws::S3::Model::CopyObjectOutcome CopyObject( + const Aws::S3::Model::CopyObjectRequest& request) { + CloudRequestCallbackGuard t(cloud_request_callback_.get(), + CloudRequestOpType::kCopyOp); + auto outcome = client_->CopyObject(request); + t.SetSuccess(outcome.IsSuccess()); + return outcome; + } + + Aws::S3::Model::GetObjectOutcome GetObject( + const Aws::S3::Model::GetObjectRequest& request) { + CloudRequestCallbackGuard t(cloud_request_callback_.get(), + CloudRequestOpType::kReadOp); + auto outcome = client_->GetObject(request); + if (outcome.IsSuccess()) { + t.SetSize(outcome.GetResult().GetContentLength()); + t.SetSuccess(true); } - Log(InfoLogLevel::ERROR_LEVEL, env_->info_log_, - "[s3] S3ReadableFile error in reading %s %" PRIu64 " %s %s", - fname_.c_str(), offset, buffer, error.GetMessage().c_str()); - return Status::IOError(fname_, errmsg.c_str()); - } - std::stringstream ss; - // const Aws::S3::Model::GetObjectResult& res = outcome.GetResult(); - - // extract data payload - Aws::IOStream& body = outcome.GetResult().GetBody(); - uint64_t size = 0; - if (n != 0) { - body.read(scratch, n); - size = body.gcount(); - assert(size <= n); - } - *result = Slice(scratch, size); - - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[s3] S3ReadableFile file %s filesize %" PRIu64 " read %" PRIu64 " bytes", - fname_.c_str(), file_size_, size); - return Status::OK(); -} + return outcome; + } + std::shared_ptr DownloadFile( + const Aws::String& bucket_name, const Aws::String& object_path, + const Aws::String& destination) { + CloudRequestCallbackGuard guard(cloud_request_callback_.get(), + CloudRequestOpType::kReadOp); + auto handle = + transfer_manager_->DownloadFile(bucket_name, object_path, destination); + + handle->WaitUntilFinished(); + bool success = + handle->GetStatus() == Aws::Transfer::TransferStatus::COMPLETED; + guard.SetSuccess(success); + if (success) { + guard.SetSize(handle->GetBytesTotalSize()); + } + return handle; + } -Status S3ReadableFile::Skip(uint64_t n) { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[s3] S3ReadableFile file %s skip %" PRIu64, fname_.c_str(), n); - // Update offset_ so that it does not go beyond filesize - offset_ += n; - if (offset_ > file_size_) { - offset_ = file_size_; + Aws::S3::Model::PutObjectOutcome PutObject( + const Aws::S3::Model::PutObjectRequest& request, uint64_t size_hint = 0) { + CloudRequestCallbackGuard t(cloud_request_callback_.get(), + CloudRequestOpType::kWriteOp, size_hint); + auto outcome = client_->PutObject(request); + t.SetSuccess(outcome.IsSuccess()); + return outcome; } - return Status::OK(); -} -size_t S3ReadableFile::GetUniqueId(char* id, size_t max_size) const { - // If this is an SST file name, then it can part of the persistent cache. - // We need to generate a unique id for the cache. - // If it is not a sst file, then nobody should be using this id. - uint64_t file_number; - FileType file_type; - WalFileType log_type; - ParseFileName(RemoveEpoch(basename(fname_)), &file_number, &file_type, - &log_type); - if (max_size < kMaxVarint64Length && file_number > 0) { - char* rid = id; - rid = EncodeVarint64(rid, file_number); - return static_cast(rid - id); - } - return 0; + std::shared_ptr UploadFile( + const Aws::String& bucket_name, const Aws::String& object_path, + const Aws::String& destination, uint64_t file_size) { + CloudRequestCallbackGuard guard(cloud_request_callback_.get(), + CloudRequestOpType::kWriteOp, file_size); + + auto handle = transfer_manager_->UploadFile( + destination, bucket_name, object_path, Aws::DEFAULT_CONTENT_TYPE, + Aws::Map()); + + handle->WaitUntilFinished(); + guard.SetSuccess(handle->GetStatus() == + Aws::Transfer::TransferStatus::COMPLETED); + return handle; + } + + Aws::S3::Model::HeadObjectOutcome HeadObject( + const Aws::S3::Model::HeadObjectRequest& request) { + CloudRequestCallbackGuard t(cloud_request_callback_.get(), + CloudRequestOpType::kInfoOp); + auto outcome = client_->HeadObject(request); + t.SetSuccess(outcome.IsSuccess()); + return outcome; + } + CloudRequestCallback* GetRequestCallback() { + return cloud_request_callback_.get(); + } + bool HasTransferManager() const { return transfer_manager_.get() != nullptr; } + + private: + static Aws::Utils::Threading::Executor* GetAwsTransferManagerExecutor() { + static Aws::Utils::Threading::PooledThreadExecutor executor(8); + return &executor; + } + + std::shared_ptr client_; + std::shared_ptr transfer_manager_; + std::shared_ptr cloud_request_callback_; +}; + +static bool IsNotFound(const Aws::S3::S3Errors& s3err) { + return (s3err == Aws::S3::S3Errors::NO_SUCH_BUCKET || + s3err == Aws::S3::S3Errors::NO_SUCH_KEY || + s3err == Aws::S3::S3Errors::RESOURCE_NOT_FOUND); } +/******************** S3ReadableFile ******************/ +class S3ReadableFile : public CloudStorageReadableFile { + public: + S3ReadableFile(const std::shared_ptr& s3client, + const std::shared_ptr& info_log, + const std::string& bucket, const std::string& fname, + uint64_t size) + : CloudStorageReadableFile(info_log, bucket, fname, size), + s3client_(s3client) {} + + virtual const char* Type() const { return "s3"; } + + // random access, read data from specified offset in file + Status DoCloudRead(uint64_t offset, size_t n, char* scratch, + uint64_t* bytes_read) const override { + // create a range read request + // Ranges are inclusive, so we can't read 0 bytes; read 1 instead and + // drop it later. + size_t rangeLen = (n != 0 ? n : 1); + char buffer[512]; + int ret = snprintf(buffer, sizeof(buffer), "bytes=%" PRIu64 "-%" PRIu64, + offset, offset + rangeLen - 1); + if (ret < 0) { + Log(InfoLogLevel::ERROR_LEVEL, info_log_, + "[s3] S3ReadableFile vsnprintf error %s offset %" PRIu64 + " rangelen %" ROCKSDB_PRIszt "\n", + fname_.c_str(), offset, rangeLen); + return Status::IOError("S3ReadableFile vsnprintf ", fname_.c_str()); + } + Aws::String range(buffer); + + // set up S3 request to read this range + Aws::S3::Model::GetObjectRequest request; + request.SetBucket(ToAwsString(bucket_)); + request.SetKey(ToAwsString(fname_)); + request.SetRange(range); + + Aws::S3::Model::GetObjectOutcome outcome = s3client_->GetObject(request); + bool isSuccess = outcome.IsSuccess(); + if (!isSuccess) { + const Aws::Client::AWSError& error = + outcome.GetError(); + std::string errmsg(error.GetMessage().c_str(), error.GetMessage().size()); + if (IsNotFound(error.GetErrorType()) || + errmsg.find("Response code: 404") != std::string::npos) { + Log(InfoLogLevel::ERROR_LEVEL, info_log_, + "[s3] S3ReadableFile error in reading not-existent %s %s", + fname_.c_str(), errmsg.c_str()); + return Status::NotFound(fname_, errmsg.c_str()); + } + Log(InfoLogLevel::ERROR_LEVEL, info_log_, + "[s3] S3ReadableFile error in reading %s %" PRIu64 " %s %s", + fname_.c_str(), offset, buffer, error.GetMessage().c_str()); + return Status::IOError(fname_, errmsg.c_str()); + } + std::stringstream ss; + // const Aws::S3::Model::GetObjectResult& res = outcome.GetResult(); + + // extract data payload + Aws::IOStream& body = outcome.GetResult().GetBody(); + *bytes_read = 0; + if (n != 0) { + body.read(scratch, n); + *bytes_read = body.gcount(); + assert(*bytes_read <= n); + } + Log(InfoLogLevel::DEBUG_LEVEL, info_log_, + "[s3] S3ReadableFile file %s filesize %" PRIu64 " read %" PRIu64 " bytes", + fname_.c_str(), file_size_, *bytes_read); + return Status::OK(); + } + + private: + std::shared_ptr s3client_; +}; // End class S3ReadableFile + /******************** Writablefile ******************/ -Status S3WritableFile::BucketExistsInS3( - std::shared_ptr client, - const std::string& bucket, - const Aws::S3::Model::BucketLocationConstraint& location) { - Aws::S3::Model::HeadBucketRequest request; - request.SetBucket(Aws::String(bucket.c_str(), bucket.size())); - Aws::S3::Model::HeadBucketOutcome outcome = client->HeadBucket(request); - return outcome.IsSuccess() ? Status::OK() : Status::NotFound(); +class S3WritableFile : public CloudStorageWritableFile { + public: + S3WritableFile(CloudEnv* env, const std::string& local_fname, + const std::string& bucket, const std::string& cloud_fname, + const EnvOptions& options) + : CloudStorageWritableFile(env, local_fname, bucket, cloud_fname, + options) {} + virtual const char* Name() const override { return "s3"; } +}; + + +/******************** S3StorageProvider ******************/ +class S3StorageProvider : public CloudStorageProviderImpl { + public: + ~S3StorageProvider() override {} + virtual const char* Name() const override { return "s3"; } + Status CreateBucket(const std::string& bucket) override; + Status ExistsBucket(const std::string& bucket) override; + Status EmptyBucket(const std::string& bucket_name, + const std::string& object_path) override; + // Empties all contents of the associated cloud storage bucket. + // Status EmptyBucket(const std::string& bucket_name, + // const std::string& object_path) override; + // Delete the specified object from the specified cloud bucket + Status DeleteObject(const std::string& bucket_name, + const std::string& object_path) override; + Status ListObjects(const std::string& bucket_name, + const std::string& object_path, + std::vector* result) override; + Status ExistsObject(const std::string& bucket_name, + const std::string& object_path) override; + Status GetObjectSize(const std::string& bucket_name, + const std::string& object_path, + uint64_t* filesize) override; + // Get the modification time of the object in cloud storage + Status GetObjectModificationTime(const std::string& bucket_name, + const std::string& object_path, + uint64_t* time) override; + + // Get the metadata of the object in cloud storage + Status GetObjectMetadata( + const std::string& bucket_name, const std::string& object_path, + std::unordered_map* metadata) override; + Status PutObjectMetadata( + const std::string& bucket_name, const std::string& object_path, + const std::unordered_map& metadata) override; + Status CopyObject(const std::string& bucket_name_src, + const std::string& object_path_src, + const std::string& bucket_name_dest, + const std::string& object_path_dest) override; + Status DoNewCloudReadableFile( + const std::string& bucket, const std::string& fname, uint64_t fsize, + std::unique_ptr* result) override; + Status NewCloudWritableFile(const std::string& local_path, + const std::string& bucket_name, + const std::string& object_path, + std::unique_ptr* result, + const EnvOptions& options) override; + Status Verify() const override; + protected: + Status Initialize(CloudEnv *env) override; + Status DoGetObject(const std::string& bucket_name, + const std::string& object_path, + const std::string& destination, + uint64_t* remote_size) override; + Status DoPutObject(const std::string& local_file, + const std::string& bucket_name, + const std::string& object_path, + uint64_t file_size) override; + + private: + // If metadata, size or modtime is non-nullptr, returns requested data + Status HeadObject(const std::string& bucket, const std::string& path, + std::unordered_map* metadata = nullptr, + uint64_t* size = nullptr, uint64_t* modtime = nullptr); + // The S3 client + std::shared_ptr s3client_; +}; + +Status S3StorageProvider::Initialize(CloudEnv* env) { + Status status = CloudStorageProviderImpl::Initialize(env); + if (! status.ok()) { + return status; + } + const CloudEnvOptions& cloud_opts = env->GetCloudEnvOptions(); + // TODO: support buckets being in different regions + if (!env->SrcMatchesDest() && env->HasSrcBucket() && + env->HasDestBucket()) { + if (cloud_opts.src_bucket.GetRegion() != + cloud_opts.dest_bucket.GetRegion()) { + Log(InfoLogLevel::ERROR_LEVEL, env->GetLogger(), + "[aws] NewAwsEnv Buckets %s, %s in two different regions %s, %s " + "is not supported", + cloud_opts.src_bucket.GetBucketName().c_str(), + cloud_opts.dest_bucket.GetBucketName().c_str(), + cloud_opts.src_bucket.GetRegion().c_str(), + cloud_opts.dest_bucket.GetRegion().c_str()); + return Status::InvalidArgument("Two different regions not supported"); + } + } + Aws::Client::ClientConfiguration config; + status = AwsCloudOptions::GetClientConfiguration(env, cloud_opts.src_bucket.GetRegion(), + &config); + if (status.ok()) { + std::shared_ptr creds; + status = cloud_opts.credentials.GetCredentialsProvider(&creds); + if (!status.ok()) { + Log(InfoLogLevel::INFO_LEVEL, env->GetLogger(), + "[aws] NewAwsEnv - Bad AWS credentials"); + } else { + Header(env->GetLogger(), "S3 connection to endpoint in region: %s", + config.region.c_str()); + s3client_ = + std::make_shared(creds, config, cloud_opts); + } + } + return status; } +Status S3StorageProvider::Verify() const { + Status s = CloudStorageProviderImpl::Verify(); + if (s.ok()) { + if (! s3client_) { + s = Status::InvalidArgument("S3Client Failed to initialize"); + } + } + return s; +} + // // Create bucket in S3 if it does not already exist. // -Status S3WritableFile::CreateBucketInS3( - std::shared_ptr client, - const std::string& bucket, - const Aws::S3::Model::BucketLocationConstraint& location) { +Status S3StorageProvider::CreateBucket(const std::string& bucket) { // specify region for the bucket Aws::S3::Model::CreateBucketConfiguration conf; - if (location != Aws::S3::Model::BucketLocationConstraint::NOT_SET) { + // AWS's utility to help out with uploading and downloading S3 file + Aws::S3::Model::BucketLocationConstraint bucket_location = + Aws::S3::Model::BucketLocationConstraintMapper:: + GetBucketLocationConstraintForName(ToAwsString(env_->GetCloudEnvOptions().dest_bucket.GetRegion())); + if (bucket_location != Aws::S3::Model::BucketLocationConstraint::NOT_SET) { // only set the location constraint if it's not not set - conf.SetLocationConstraint(location); + conf.SetLocationConstraint(bucket_location); } // create bucket Aws::S3::Model::CreateBucketRequest request; - request.SetBucket(Aws::String(bucket.c_str(), bucket.size())); + request.SetBucket(ToAwsString(bucket)); request.SetCreateBucketConfiguration(conf); - Aws::S3::Model::CreateBucketOutcome outcome = client->CreateBucket(request); + Aws::S3::Model::CreateBucketOutcome outcome = + s3client_->CreateBucket(request); bool isSuccess = outcome.IsSuccess(); if (!isSuccess) { const Aws::Client::AWSError& error = outcome.GetError(); @@ -208,153 +491,390 @@ Status S3WritableFile::CreateBucketInS3( return Status::OK(); } -S3WritableFile::S3WritableFile(AwsEnv* env, const std::string& local_fname, - const std::string& bucket_prefix, - const std::string& cloud_fname, - const EnvOptions& options, - const CloudEnvOptions cloud_env_options) - : env_(env), - fname_(local_fname), - bucket_prefix_(bucket_prefix), - cloud_fname_(cloud_fname) { - auto fname_no_epoch = RemoveEpoch(fname_); - // Is this a manifest file? - is_manifest_ = IsManifestFile(fname_no_epoch); - assert(IsSstFile(fname_no_epoch) || is_manifest_); - - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[s3] S3WritableFile bucket %s opened local file %s " - "cloud file %s manifest %d", - bucket_prefix.c_str(), fname_.c_str(), cloud_fname.c_str(), is_manifest_); - - auto* file_to_open = &fname_; - auto local_env = env_->GetBaseEnv(); - Status s; - if (is_manifest_) { - s = local_env->FileExists(fname_); - if (!s.ok() && !s.IsNotFound()) { - status_ = s; - return; - } - if (s.ok()) { - // Manifest exists. Instead of overwriting the MANIFEST (which could be - // bad if we crash mid-write), write to the temporary file and do an - // atomic rename on Sync() (Sync means we have a valid data in the - // MANIFEST, so we can crash after it) - tmp_file_ = fname_ + ".tmp"; - file_to_open = &tmp_file_; +Status S3StorageProvider::ExistsBucket(const std::string& bucket) { + Aws::S3::Model::HeadBucketRequest request; + request.SetBucket(ToAwsString(bucket)); + Aws::S3::Model::HeadBucketOutcome outcome = s3client_->HeadBucket(request); + return outcome.IsSuccess() ? Status::OK() : Status::NotFound(); +} + +// +// Deletes all the objects with the specified path prefix in our bucket +// +Status S3StorageProvider::EmptyBucket(const std::string& bucket_name, + const std::string& object_path) { + std::vector results; + + // Get all the objects in the bucket + Status st = ListObjects(bucket_name, object_path, &results); + if (!st.ok()) { + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[s3] EmptyBucket unable to find objects in bucket %s %s", + bucket_name.c_str(), st.ToString().c_str()); + return st; + } + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), + "[s3] EmptyBucket going to delete %" ROCKSDB_PRIszt " objects in bucket %s", + results.size(), bucket_name.c_str()); + + // Delete all objects from bucket + for (auto path : results) { + st = DeleteObject(bucket_name, path); + if (!st.ok()) { + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[s3] EmptyBucket Unable to delete %s in bucket %s %s", path.c_str(), + bucket_name.c_str(), st.ToString().c_str()); } } + return st; +} + +Status S3StorageProvider::DeleteObject(const std::string& bucket_name, + const std::string& object_path) { + Status st; - s = local_env->NewWritableFile(*file_to_open, &local_file_, options); - if (!s.ok()) { - Log(InfoLogLevel::ERROR_LEVEL, env_->info_log_, - "[s3] NewWritableFile src %s %s", fname_.c_str(), s.ToString().c_str()); - status_ = s; + // create request + Aws::S3::Model::DeleteObjectRequest request; + request.SetBucket(ToAwsString(bucket_name)); + request.SetKey(ToAwsString( + object_path)); // The filename is the object name in the bucket + + Aws::S3::Model::DeleteObjectOutcome outcome = + s3client_->DeleteObject(request); + bool isSuccess = outcome.IsSuccess(); + if (!isSuccess) { + const Aws::Client::AWSError& error = outcome.GetError(); + std::string errmsg(error.GetMessage().c_str()); + if (IsNotFound(error.GetErrorType())) { + st = Status::NotFound(object_path, errmsg.c_str()); + } else { + st = Status::IOError(object_path, errmsg.c_str()); + } } + + Log(InfoLogLevel::INFO_LEVEL, env_->GetLogger(), + "[s3] DeleteFromS3 %s/%s, status %s", bucket_name.c_str(), + object_path.c_str(), st.ToString().c_str()); + + return st; } -S3WritableFile::~S3WritableFile() { - if (local_file_ != nullptr) { - Close(); +// +// Appends the names of all children of the specified path from S3 +// into the result set. +// +Status S3StorageProvider::ListObjects(const std::string& bucket_name, + const std::string& object_path, + std::vector* result) { + // S3 paths don't start with '/' + auto prefix = ltrim_if(object_path, '/'); + // S3 paths better end with '/', otherwise we might also get a list of files + // in a directory for which our path is a prefix + prefix = ensure_ends_with_pathsep(std::move(prefix)); + // the starting object marker + Aws::String marker; + bool loop = true; + + // get info of bucket+object + while (loop) { + Aws::S3::Model::ListObjectsRequest request; + request.SetBucket(ToAwsString(bucket_name)); + request.SetMaxKeys(50); + request.SetPrefix(ToAwsString(prefix)); + request.SetMarker(marker); + + Aws::S3::Model::ListObjectsOutcome outcome = + s3client_->ListObjects(request); + bool isSuccess = outcome.IsSuccess(); + if (!isSuccess) { + const Aws::Client::AWSError& error = + outcome.GetError(); + std::string errmsg(error.GetMessage().c_str()); + if (IsNotFound(error.GetErrorType())) { + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[s3] GetChildren dir %s does not exist: %s", object_path.c_str(), + errmsg.c_str()); + return Status::NotFound(object_path, errmsg.c_str()); + } + return Status::IOError(object_path, errmsg.c_str()); + } + const Aws::S3::Model::ListObjectsResult& res = outcome.GetResult(); + const Aws::Vector& objs = res.GetContents(); + for (auto o : objs) { + const Aws::String& key = o.GetKey(); + // Our path should be a prefix of the fetched value + std::string keystr(key.c_str(), key.size()); + assert(keystr.find(prefix) == 0); + if (keystr.find(prefix) != 0) { + return Status::IOError("Unexpected result from AWS S3: " + keystr); + } + auto fname = keystr.substr(prefix.size()); + result->push_back(fname); + } + + // If there are no more entries, then we are done. + if (!res.GetIsTruncated()) { + break; + } + // The new starting point + marker = res.GetNextMarker(); + if (marker.empty()) { + // If response does not include the NextMaker and it is + // truncated, you can use the value of the last Key in the response + // as the marker in the subsequent request because all objects + // are returned in alphabetical order + marker = objs.back().GetKey(); + } } + return Status::OK(); +} +// Delete the specified object from the specified cloud bucket +Status S3StorageProvider::ExistsObject(const std::string& bucket_name, + const std::string& object_path) { + Status s = HeadObject(bucket_name, object_path); + return s; } -Status S3WritableFile::Close() { - if (local_file_ == nullptr) { // already closed - return status_; - } - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[s3] S3WritableFile closing %s", fname_.c_str()); - assert(status_.ok()); +// Return size of cloud object +Status S3StorageProvider::GetObjectSize(const std::string& bucket_name, + const std::string& object_path, + uint64_t* filesize) { + Status s = HeadObject(bucket_name, object_path, nullptr, filesize, nullptr); + return s; +} - // close local file - Status st = local_file_->Close(); - if (!st.ok()) { - Log(InfoLogLevel::ERROR_LEVEL, env_->info_log_, - "[s3] S3WritableFile closing error on local %s\n", fname_.c_str()); - return st; - } - local_file_.reset(); +Status S3StorageProvider::GetObjectModificationTime( + const std::string& bucket_name, const std::string& object_path, + uint64_t* time) { + return HeadObject(bucket_name, object_path, nullptr, nullptr, time); +} - if (is_manifest_) { - // MANIFEST is made durable in each Sync() call, no need to re-upload it - return Status::OK(); +Status S3StorageProvider::GetObjectMetadata( + const std::string& bucket_name, const std::string& object_path, + std::unordered_map* metadata) { + return HeadObject(bucket_name, object_path, metadata, nullptr, nullptr); +} + +Status S3StorageProvider::PutObjectMetadata( + const std::string& bucket_name, const std::string& object_path, + const std::unordered_map& metadata) { + Aws::S3::Model::PutObjectRequest request; + Aws::Map aws_metadata; + for (const auto m : metadata) { + aws_metadata[ToAwsString(m.first)] = ToAwsString(m.second); } + request.SetBucket(ToAwsString(bucket_name)); + request.SetKey(ToAwsString(object_path)); + request.SetMetadata(aws_metadata); + SetEncryptionParameters(env_->GetCloudEnvOptions(), request); - // upload sst file to S3, but first remove from deletion queue if it's in - // there - uint64_t fileNumber; - FileType type; - WalFileType walType; - bool ok __attribute__((unused)) = ParseFileName(RemoveEpoch(basename(fname_)), - &fileNumber, &type, &walType); - assert(ok && type == kTableFile); - env_->RemoveFileFromDeletionQueue(basename(fname_)); - status_ = env_->PutObject(fname_, bucket_prefix_, cloud_fname_); - if (!status_.ok()) { - Log(InfoLogLevel::ERROR_LEVEL, env_->info_log_, - "[s3] S3WritableFile closing PutObject failed on local file %s", - fname_.c_str()); - return status_; - } - - // delete local file - if (!env_->GetCloudEnvOptions().keep_local_sst_files) { - status_ = env_->GetBaseEnv()->DeleteFile(fname_); - if (!status_.ok()) { - Log(InfoLogLevel::ERROR_LEVEL, env_->info_log_, - "[s3] S3WritableFile closing delete failed on local file %s", - fname_.c_str()); - return status_; - } + auto outcome = s3client_->PutObject(request); + bool isSuccess = outcome.IsSuccess(); + if (!isSuccess) { + const auto& error = outcome.GetError(); + std::string errmsg(error.GetMessage().c_str(), error.GetMessage().size()); + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[s3] Bucket %s error in saving metadata %s", bucket_name.c_str(), + errmsg.c_str()); + return Status::IOError(object_path, errmsg.c_str()); } - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[s3] S3WritableFile closed file %s", fname_.c_str()); return Status::OK(); } -// Sync a file to stable storage -Status S3WritableFile::Sync() { - if (local_file_ == nullptr) { - return status_; - } - assert(status_.ok()); +Status S3StorageProvider::DoNewCloudReadableFile( + const std::string& bucket, const std::string& fname, uint64_t fsize, + std::unique_ptr* result) { + result->reset( + new S3ReadableFile(s3client_, env_->GetLogger(), bucket, fname, fsize)); + return Status::OK(); +} - // sync local file - Status stat = local_file_->Sync(); +Status S3StorageProvider::NewCloudWritableFile( + const std::string& local_path, const std::string& bucket_name, + const std::string& object_path, + std::unique_ptr* result, + const EnvOptions& options) { + result->reset( + new S3WritableFile(env_, local_path, bucket_name, object_path, options)); + return (*result)->status(); +} + +Status S3StorageProvider::HeadObject( + const std::string& bucket_name, const std::string& object_path, + std::unordered_map* metadata, uint64_t* size, + uint64_t* modtime) { + Aws::S3::Model::HeadObjectRequest request; + request.SetBucket(ToAwsString(bucket_name)); + request.SetKey(ToAwsString(object_path)); - if (stat.ok() && !tmp_file_.empty()) { - assert(is_manifest_); - // We are writing to the temporary file. On a first sync we need to rename - // the file to the real filename. - stat = env_->GetBaseEnv()->RenameFile(tmp_file_, fname_); - // Note: this is not thread safe, but we know that manifest writes happen - // from the same thread, so we are fine. - tmp_file_.clear(); + auto outcome = s3client_->HeadObject(request); + bool isSuccess = outcome.IsSuccess(); + if (!isSuccess) { + const auto& error = outcome.GetError(); + auto errMessage = error.GetMessage(); + if (IsNotFound(error.GetErrorType())) { + return Status::NotFound(object_path, errMessage.c_str()); + } + return Status::IOError(object_path, errMessage.c_str()); + } + auto& res = outcome.GetResult(); + if (metadata != nullptr) { + for (const auto m : res.GetMetadata()) { + (*metadata)[m.first.c_str()] = m.second.c_str(); + } + } + if (size != nullptr) { + *size = res.GetContentLength(); + } + if (modtime != nullptr) { + *modtime = res.GetLastModified().Millis(); } + return Status::OK(); +} - // We copy MANIFEST to S3 on every Sync() - if (is_manifest_ && stat.ok()) { - stat = env_->PutObject(fname_, bucket_prefix_, cloud_fname_); +// Copy the specified cloud object from one location in the cloud +// storage to another location in cloud storage +Status S3StorageProvider::CopyObject(const std::string& bucket_name_src, + const std::string& object_path_src, + const std::string& bucket_name_dest, + const std::string& object_path_dest) { + Status st; + Aws::String src_bucket = ToAwsString(bucket_name_src); + Aws::String dest_bucket = ToAwsString(bucket_name_dest); + + // The filename is the same as the object name in the bucket + Aws::String src_object = ToAwsString(object_path_src); + Aws::String dest_object = ToAwsString(object_path_dest); + + Aws::String src_url = src_bucket + src_object; + + // create copy request + Aws::S3::Model::CopyObjectRequest request; + request.SetCopySource(src_url); + request.SetBucket(dest_bucket); + request.SetKey(dest_object); + + // execute request + Aws::S3::Model::CopyObjectOutcome outcome = s3client_->CopyObject(request); + bool isSuccess = outcome.IsSuccess(); + if (!isSuccess) { + const Aws::Client::AWSError& error = outcome.GetError(); + std::string errmsg(error.GetMessage().c_str()); + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[s3] S3WritableFile src path %s error in copying to %s %s", + src_url.c_str(), dest_object.c_str(), errmsg.c_str()); + return Status::IOError(dest_object.c_str(), errmsg.c_str()); + } + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[s3] S3WritableFile src path %s copied to %s %s", src_url.c_str(), + dest_object.c_str(), st.ToString().c_str()); + return st; +} - if (stat.ok()) { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[s3] S3WritableFile made manifest %s durable to " - "bucket %s bucketpath %s.", - fname_.c_str(), bucket_prefix_.c_str(), cloud_fname_.c_str()); +Status S3StorageProvider::DoGetObject(const std::string& bucket_name, + const std::string& object_path, + const std::string& destination, + uint64_t* remote_size) { + if (s3client_->HasTransferManager()) { + auto handle = s3client_->DownloadFile(ToAwsString(bucket_name), + ToAwsString(object_path), + ToAwsString(destination)); + bool success = + handle->GetStatus() == Aws::Transfer::TransferStatus::COMPLETED; + if (success) { + *remote_size = handle->GetBytesTotalSize(); } else { - Log(InfoLogLevel::ERROR_LEVEL, env_->info_log_, - "[s3] S3WritableFile failed to make manifest %s durable to " - "bucket %s bucketpath %s. %s", - fname_.c_str(), bucket_prefix_.c_str(), cloud_fname_.c_str(), - stat.ToString().c_str()); + const auto& error = handle->GetLastError(); + std::string errmsg(error.GetMessage().c_str(), error.GetMessage().size()); + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[s3] DownloadFile %s/%s error %s.", bucket_name.c_str(), + object_path.c_str(), errmsg.c_str()); + if (IsNotFound(error.GetErrorType())) { + return Status::NotFound(std::move(errmsg)); + } + return Status::IOError(std::move(errmsg)); + } + } else { + Aws::S3::Model::GetObjectRequest request; + request.SetBucket(ToAwsString(bucket_name)); + request.SetKey(ToAwsString(object_path)); + + request.SetResponseStreamFactory([destination]() { + return Aws::New(Aws::Utils::ARRAY_ALLOCATION_TAG, + destination, std::ios_base::out); + }); + auto outcome = s3client_->GetObject(request); + if (outcome.IsSuccess()) { + *remote_size = outcome.GetResult().GetContentLength(); + } else { + const auto& error = outcome.GetError(); + std::string errmsg(error.GetMessage().c_str(), error.GetMessage().size()); + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[s3] GetObject %s/%s error %s.", bucket_name.c_str(), + object_path.c_str(), errmsg.c_str()); + if (IsNotFound(error.GetErrorType())) { + return Status::NotFound(std::move(errmsg)); + } + return Status::IOError(std::move(errmsg)); } } - return stat; + return Status::OK(); } -#pragma GCC diagnostic pop +Status S3StorageProvider::DoPutObject(const std::string& local_file, + const std::string& bucket_name, + const std::string& object_path, + uint64_t file_size) { + if (s3client_->HasTransferManager()) { + auto handle = + s3client_->UploadFile(ToAwsString(local_file), ToAwsString(bucket_name), + ToAwsString(object_path), file_size); + if (handle->GetStatus() != Aws::Transfer::TransferStatus::COMPLETED) { + auto error = handle->GetLastError(); + std::string errmsg(error.GetMessage().c_str(), error.GetMessage().size()); + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[s3] UploadFile %s/%s, size %" PRIu64 ", ERROR %s", bucket_name.c_str(), + object_path.c_str(), file_size, errmsg.c_str()); + return Status::IOError(local_file, errmsg); + } + } else { + auto inputData = + Aws::MakeShared(object_path.c_str(), local_file.c_str(), + std::ios_base::in | std::ios_base::out); + + Aws::S3::Model::PutObjectRequest putRequest; + putRequest.SetBucket(ToAwsString(bucket_name)); + putRequest.SetKey(ToAwsString(object_path)); + putRequest.SetBody(inputData); + SetEncryptionParameters(env_->GetCloudEnvOptions(), putRequest); + + auto outcome = s3client_->PutObject(putRequest, file_size); + if (!outcome.IsSuccess()) { + auto error = outcome.GetError(); + std::string errmsg(error.GetMessage().c_str(), error.GetMessage().size()); + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[s3] PutObject %s/%s, size %" PRIu64 ", ERROR %s", bucket_name.c_str(), + object_path.c_str(), file_size, errmsg.c_str()); + return Status::IOError(local_file, errmsg); + } + } + Log(InfoLogLevel::INFO_LEVEL, env_->GetLogger(), + "[s3] PutObject %s/%s, size %" PRIu64 ", OK", bucket_name.c_str(), + object_path.c_str(), file_size); + return Status::OK(); +} } // namespace #endif /* USE_AWS */ + +namespace rocksdb { +Status CloudStorageProvider::CreateS3Provider(std::unique_ptr* provider) { +#ifndef USE_AWS + provider->reset(); + return Status::NotSupported("In order to use S3, make sure you're compiling with USE_AWS=1"); +#else + provider->reset(new S3StorageProvider()); + return Status::OK(); +#endif /* USE_AWS */ +} +} diff --git a/cloud/cloud_env.cc b/cloud/cloud_env.cc index db6c7723b08..f6d8aece155 100644 --- a/cloud/cloud_env.cc +++ b/cloud/cloud_env.cc @@ -5,11 +5,11 @@ #include "cloud/aws/aws_env.h" #include "cloud/cloud_env_impl.h" -#include "cloud/cloud_env_wrapper.h" -#include "cloud/cloud_log_controller.h" #include "cloud/db_cloud_impl.h" #include "cloud/filename.h" #include "port/likely.h" +#include "rocksdb/cloud/cloud_log_controller.h" +#include "rocksdb/cloud/cloud_storage_provider.h" #include "rocksdb/db.h" #include "rocksdb/env.h" #include "rocksdb/options.h" @@ -17,6 +17,9 @@ namespace rocksdb { +const std::string CloudEnv::kCloudEnvOpts ="CloudEnvOptions"; +const std::string CloudEnv::kCloudEnvName = "CloudEnv"; + bool CloudEnvOptions::GetNameFromEnvironment(const char *name, const char *alt, std::string * result) { char *value = getenv(name); // See if name is set in the environment @@ -38,6 +41,7 @@ void CloudEnvOptions::TEST_Initialize(const std::string& bucket, credentials.TEST_Initialize(); } + BucketOptions::BucketOptions() { prefix_ = "rockset."; } @@ -86,15 +90,75 @@ void BucketOptions::TEST_Initialize(const std::string& bucket, } } -CloudEnv::CloudEnv(const CloudEnvOptions& options, Env *base, const std::shared_ptr& logger) - : cloud_env_options(options), - base_env_(base), - info_log_(logger) { +CloudEnv::CloudEnv(const CloudEnvOptions& options, Env* base, const std::shared_ptr& logger) + : cloud_env_options(options), base_env_(base), info_log_(logger) { } + +CloudEnv::~CloudEnv() { + cloud_env_options.storage_provider.reset(); + cloud_env_options.log_controller.reset(); +} + -CloudEnv::~CloudEnv() {} +Status CloudEnv::Verify() const { + Status s; + if (!cloud_env_options.storage_provider) { + s = Status::InvalidArgument("Cloud environment requires a storage provider"); + } else { + s = cloud_env_options.storage_provider->Verify(); + } + if (s.ok()) { + if (cloud_env_options.log_controller) { + s = cloud_env_options.log_controller->Verify(); + } else if (!cloud_env_options.keep_local_log_files) { + s = Status::InvalidArgument("Log controller required for remote log files"); + } + } + return s; +} + +Status CloudEnv::Prepare() { + Header(info_log_, " %s.src_bucket_name: %s", + Name(), cloud_env_options.src_bucket.GetBucketName().c_str()); + Header(info_log_, " %s.src_object_path: %s", + Name(), cloud_env_options.src_bucket.GetObjectPath().c_str()); + Header(info_log_, " %s.src_bucket_region: %s", + Name(), cloud_env_options.src_bucket.GetRegion().c_str()); + Header(info_log_, " %s.dest_bucket_name: %s", + Name(), cloud_env_options.dest_bucket.GetBucketName().c_str()); + Header(info_log_, " %s.dest_object_path: %s", + Name(), cloud_env_options.dest_bucket.GetObjectPath().c_str()); + Header(info_log_, " %s.dest_bucket_region: %s", + Name(), cloud_env_options.dest_bucket.GetRegion().c_str()); + + Status s; + if (cloud_env_options.src_bucket.GetBucketName().empty() != + cloud_env_options.src_bucket.GetObjectPath().empty()) { + s = Status::InvalidArgument("Must specify both src bucket name and path"); + } else if (cloud_env_options.dest_bucket.GetBucketName().empty() != + cloud_env_options.dest_bucket.GetObjectPath().empty()) { + s = Status::InvalidArgument("Must specify both dest bucket name and path"); + } else { + if (!cloud_env_options.storage_provider) { + s = Status::InvalidArgument("Cloud environment requires a storage provider"); + } else { + Header(info_log_, " %s.storage_provider: %s", + Name(), cloud_env_options.storage_provider->Name()); + s = cloud_env_options.storage_provider->Prepare(this); + } + if (s.ok()) { + if (cloud_env_options.log_controller) { + Header(info_log_, " %s.log controller: %s", + Name(), cloud_env_options.log_controller->Name()); + s = cloud_env_options.log_controller->Prepare(this); + } else if (!cloud_env_options.keep_local_log_files) { + s = Status::InvalidArgument("Log controller required for remote log files"); + } + } + } + return s; +} -CloudEnvWrapper::~CloudEnvWrapper() {} Status CloudEnv::NewAwsEnv( Env* base_env, const std::string& src_cloud_bucket, @@ -113,30 +177,18 @@ Status CloudEnv::NewAwsEnv( } #ifndef USE_AWS -Status CloudEnv::NewAwsEnv(Env* /*base_env*/, - const CloudEnvOptions& /*options*/, - const std::shared_ptr& /*logger*/, - CloudEnv** /*cenv*/) { +Status CloudEnv::NewAwsEnv(Env*, const CloudEnvOptions&, + const std::shared_ptr&, CloudEnv**) { return Status::NotSupported("RocksDB Cloud not compiled with AWS support"); } #else -Status CloudEnv::NewAwsEnv(Env* base_env, - const CloudEnvOptions& options, - const std::shared_ptr & logger, CloudEnv** cenv) { +Status CloudEnv::NewAwsEnv(Env* base_env, const CloudEnvOptions& options, + const std::shared_ptr& logger, + CloudEnv** cenv) { // Dump out cloud env options options.Dump(logger.get()); Status st = AwsEnv::NewAwsEnv(base_env, options, logger, cenv); - if (st.ok()) { - // store a copy of the logger - CloudEnvImpl* cloud = static_cast(*cenv); - cloud->info_log_ = logger; - - // start the purge thread only if there is a destination bucket - if (options.dest_bucket.IsValid() && options.run_purger) { - cloud->purge_thread_ = std::thread([cloud] { cloud->Purger(); }); - } - } return st; } #endif diff --git a/cloud/cloud_env_impl.cc b/cloud/cloud_env_impl.cc index 5f0a5320b9a..fe403ee8309 100644 --- a/cloud/cloud_env_impl.cc +++ b/cloud/cloud_env_impl.cc @@ -4,13 +4,13 @@ #include #include "cloud/cloud_env_impl.h" -#include "cloud/cloud_env_wrapper.h" -#include "cloud/cloud_log_controller.h" #include "cloud/filename.h" #include "cloud/manifest_reader.h" #include "file/filename.h" #include "file/file_util.h" #include "port/likely.h" +#include "rocksdb/cloud/cloud_log_controller.h" +#include "rocksdb/cloud/cloud_storage_provider.h" #include "rocksdb/db.h" #include "rocksdb/env.h" #include "rocksdb/options.h" @@ -19,17 +19,125 @@ #include "util/xxhash.h" namespace rocksdb { +namespace detail { - CloudEnvImpl::CloudEnvImpl(const CloudEnvOptions& opts, Env* base, const std::shared_ptr& l) - : CloudEnv(opts, base, l), purger_is_running_(true) {} +using ScheduledJob = + std::pair>; +struct Comp { + bool operator()(const ScheduledJob& a, const ScheduledJob& b) const { + return a.first < b.first; + } +}; +struct JobHandle { + std::multiset::iterator itr; + JobHandle(std::multiset::iterator i) + : itr(std::move(i)) {} +}; + +class JobExecutor { + public: + std::shared_ptr ScheduleJob(std::chrono::steady_clock::time_point time, + std::function callback); + void CancelJob(JobHandle* handle); + + JobExecutor(); + ~JobExecutor(); + + private: + void DoWork(); + + std::mutex mutex_; + // Notified when the earliest job to be scheduled has changed. + std::condition_variable jobs_changed_cv_; + std::multiset scheduled_jobs_; + bool shutting_down_{false}; + + std::thread thread_; +}; + +JobExecutor::JobExecutor() { + thread_ = std::thread([this]() { DoWork(); }); +} + +JobExecutor::~JobExecutor() { + { + std::lock_guard lk(mutex_); + shutting_down_ = true; + jobs_changed_cv_.notify_all(); + } + if (thread_.joinable()) { + thread_.join(); + } +} + +std::shared_ptr JobExecutor::ScheduleJob( + std::chrono::steady_clock::time_point time, + std::function callback) { + std::lock_guard lk(mutex_); + auto itr = scheduled_jobs_.emplace(time, std::move(callback)); + if (itr == scheduled_jobs_.begin()) { + jobs_changed_cv_.notify_all(); + } + return std::make_shared(itr); +} + +void JobExecutor::CancelJob(JobHandle* handle) { + std::lock_guard lk(mutex_); + if (scheduled_jobs_.begin() == handle->itr) { + jobs_changed_cv_.notify_all(); + } + scheduled_jobs_.erase(handle->itr); +} + +void JobExecutor::DoWork() { + while (true) { + std::unique_lock lk(mutex_); + if (shutting_down_) { + break; + } + if (scheduled_jobs_.empty()) { + jobs_changed_cv_.wait(lk); + continue; + } + auto earliest_job = scheduled_jobs_.begin(); + auto earliest_job_time = earliest_job->first; + if (earliest_job_time >= std::chrono::steady_clock::now()) { + jobs_changed_cv_.wait_until(lk, earliest_job_time); + continue; + } + // invoke the function + lk.unlock(); + earliest_job->second(); + lk.lock(); + scheduled_jobs_.erase(earliest_job); + } +} + +} // namespace detail + +detail::JobExecutor* GetJobExecutor() { + static detail::JobExecutor executor; + return &executor; +} + +CloudEnvImpl::CloudEnvImpl(const CloudEnvOptions& opts, Env* base, const std::shared_ptr & logger) + : CloudEnv(opts, base, logger), purger_is_running_(true) {} CloudEnvImpl::~CloudEnvImpl() { - if (cloud_log_controller_) { - cloud_log_controller_->StopTailingStream(); + if (cloud_env_options.log_controller) { + cloud_env_options.log_controller->StopTailingStream(); + } + { + std::lock_guard lk(files_to_delete_mutex_); + using std::swap; + for (auto& e : files_to_delete_) { + GetJobExecutor()->CancelJob(e.second.get()); + } + files_to_delete_.clear(); } StopPurger(); } - + void CloudEnvImpl::StopPurger() { { std::lock_guard lk(purger_lock_); @@ -61,8 +169,7 @@ Status CloudEnvImpl::LoadLocalCloudManifest(const std::string& dbname) { } std::string CloudEnvImpl::RemapFilename(const std::string& logical_path) const { - if (UNLIKELY(GetCloudType() == CloudType::kCloudNone) || - UNLIKELY(test_disable_cloud_manifest_)) { + if (UNLIKELY(test_disable_cloud_manifest_)) { return logical_path; } auto file_name = basename(logical_path); @@ -104,13 +211,14 @@ std::string CloudEnvImpl::RemapFilename(const std::string& logical_path) const { Status CloudEnvImpl::DeleteInvisibleFiles(const std::string& dbname) { Status s; if (HasDestBucket()) { - BucketObjectMetadata metadata; - s = ListObjects(GetDestBucketName(), GetDestObjectPath(), &metadata); + std::vector pathnames; + s = cloud_env_options.storage_provider->ListObjects(GetDestBucketName(), GetDestObjectPath(), + &pathnames); if (!s.ok()) { return s; } - for (auto& fname : metadata.pathnames) { + for (auto& fname : pathnames) { auto noepoch = RemoveEpoch(fname); if (IsSstFile(noepoch) || IsManifestFile(noepoch)) { if (RemapFilename(noepoch) != fname) { @@ -507,8 +615,8 @@ Status CloudEnvImpl::GetCloudDbid(const std::string& local_dir, // Read dbid from src bucket if it exists if (HasSrcBucket()) { - Status st = GetObject(GetSrcBucketName(), GetSrcObjectPath() + "/IDENTITY", - tmpfile); + Status st = cloud_env_options.storage_provider->GetObject( + GetSrcBucketName(), GetSrcObjectPath() + "/IDENTITY", tmpfile); if (!st.ok() && !st.IsNotFound()) { return st; } @@ -530,8 +638,8 @@ Status CloudEnvImpl::GetCloudDbid(const std::string& local_dir, // Read dbid from dest bucket if it exists if (HasDestBucket()) { - Status st = GetObject(GetDestBucketName(), - GetDestObjectPath() + "/IDENTITY", tmpfile); + Status st = cloud_env_options.storage_provider->GetObject( + GetDestBucketName(), GetDestObjectPath() + "/IDENTITY", tmpfile); if (!st.ok() && !st.IsNotFound()) { return st; } @@ -590,55 +698,49 @@ Status CloudEnvImpl::MaybeMigrateManifestFile(const std::string& local_dbname) { } Status CloudEnvImpl::PreloadCloudManifest(const std::string& local_dbname) { - Status st; Env* local_env = GetBaseEnv(); local_env->CreateDirIfMissing(local_dbname); - if (GetCloudType() != CloudType::kCloudNone) { - st = MaybeMigrateManifestFile(local_dbname); - if (st.ok()) { - // Init cloud manifest - st = FetchCloudManifest(local_dbname, false); - } - if (st.ok()) { - // Inits CloudEnvImpl::cloud_manifest_, which will enable us to read files - // from the cloud - st = LoadLocalCloudManifest(local_dbname); - } + Status st = MaybeMigrateManifestFile(local_dbname); + if (st.ok()) { + // Init cloud manifest + st = FetchCloudManifest(local_dbname, false); + } + if (st.ok()) { + // Inits CloudEnvImpl::cloud_manifest_, which will enable us to read files + // from the cloud + st = LoadLocalCloudManifest(local_dbname); } return st; } Status CloudEnvImpl::LoadCloudManifest(const std::string& local_dbname, bool read_only) { - Status st; - if (GetCloudType() != CloudType::kCloudNone) { - st = MaybeMigrateManifestFile(local_dbname); - if (st.ok()) { - // Init cloud manifest - st = FetchCloudManifest(local_dbname, false); - } - if (st.ok()) { - // Inits CloudEnvImpl::cloud_manifest_, which will enable us to read files - // from the cloud - st = LoadLocalCloudManifest(local_dbname); - } - if (st.ok()) { - // Rolls the new epoch in CLOUDMANIFEST - st = RollNewEpoch(local_dbname); - } + Status st = MaybeMigrateManifestFile(local_dbname); + if (st.ok()) { + // Init cloud manifest + st = FetchCloudManifest(local_dbname, false); + } + if (st.ok()) { + // Inits CloudEnvImpl::cloud_manifest_, which will enable us to read files + // from the cloud + st = LoadLocalCloudManifest(local_dbname); + } + if (st.ok()) { + // Rolls the new epoch in CLOUDMANIFEST + st = RollNewEpoch(local_dbname); + } + if (!st.ok()) { + return st; + } + + // Do the cleanup, but don't fail if the cleanup fails. + if (!read_only) { + st = DeleteInvisibleFiles(local_dbname); if (!st.ok()) { - return st; - } - - // Do the cleanup, but don't fail if the cleanup fails. - if (!read_only) { - st = DeleteInvisibleFiles(local_dbname); - if (!st.ok()) { - Log(InfoLogLevel::INFO_LEVEL, info_log_, - "Failed to delete invisible files: %s", st.ToString().c_str()); - // Ignore the fail - st = Status::OK(); - } + Log(InfoLogLevel::INFO_LEVEL, info_log_, + "Failed to delete invisible files: %s", st.ToString().c_str()); + // Ignore the fail + st = Status::OK(); } } return st; @@ -658,20 +760,6 @@ Status CloudEnvImpl::SanitizeDirectory(const DBOptions& options, env->CreateDirIfMissing(local_name); } - if (GetCloudType() == CloudType::kCloudNone) { - // We don't need to SanitizeDirectory() - Log(InfoLogLevel::INFO_LEVEL, info_log_, - "[cloud_env_impl] SanitizeDirectory skipping dir %s for non-cloud env", - local_name.c_str()); - return Status::OK(); - } - if (GetCloudType() != CloudType::kCloudAws) { - Log(InfoLogLevel::ERROR_LEVEL, info_log_, - "[cloud_env_impl] SanitizeDirectory dir %s found non aws env", - local_name.c_str()); - return Status::NotSupported("We only support AWS for now."); - } - // Shall we reinitialize the clone dir? bool do_reinit = true; Status st = NeedsReinitialization(local_name, &do_reinit); @@ -758,8 +846,9 @@ Status CloudEnvImpl::SanitizeDirectory(const DBOptions& options, // Download IDENTITY, first try destination, then source if (HasDestBucket()) { // download IDENTITY from dest - st = GetObject(GetDestBucketName(), IdentityFileName(GetDestObjectPath()), - IdentityFileName(local_name)); + st = cloud_env_options.storage_provider->GetObject(GetDestBucketName(), + IdentityFileName(GetDestObjectPath()), + IdentityFileName(local_name)); if (!st.ok() && !st.IsNotFound()) { // If there was an error and it's not IsNotFound() we need to bail return st; @@ -768,8 +857,9 @@ Status CloudEnvImpl::SanitizeDirectory(const DBOptions& options, } if (!got_identity_from_dest && HasSrcBucket() && !SrcMatchesDest()) { // download IDENTITY from src - st = GetObject(GetSrcBucketName(), IdentityFileName(GetSrcObjectPath()), - IdentityFileName(local_name)); + st = cloud_env_options.storage_provider->GetObject(GetSrcBucketName(), + IdentityFileName(GetSrcObjectPath()), + IdentityFileName(local_name)); if (!st.ok() && !st.IsNotFound()) { // If there was an error and it's not IsNotFound() we need to bail return st; @@ -849,9 +939,9 @@ Status CloudEnvImpl::FetchCloudManifest(const std::string& local_dbname, } // first try to get cloudmanifest from dest if (HasDestBucket()) { - Status st = - GetObject(GetDestBucketName(), CloudManifestFile(GetDestObjectPath()), - cloudmanifest); + Status st = cloud_env_options.storage_provider->GetObject( + GetDestBucketName(), CloudManifestFile(GetDestObjectPath()), + cloudmanifest); if (!st.ok() && !st.IsNotFound()) { // something went wrong, bail out Log(InfoLogLevel::INFO_LEVEL, info_log_, @@ -871,8 +961,9 @@ Status CloudEnvImpl::FetchCloudManifest(const std::string& local_dbname, } // we couldn't get cloud manifest from dest, need to try from src? if (HasSrcBucket() && !SrcMatchesDest()) { - Status st = GetObject(GetSrcBucketName(), - CloudManifestFile(GetSrcObjectPath()), cloudmanifest); + Status st = cloud_env_options.storage_provider->GetObject( + GetSrcBucketName(), CloudManifestFile(GetSrcObjectPath()), + cloudmanifest); if (!st.ok() && !st.IsNotFound()) { // something went wrong, bail out Log(InfoLogLevel::INFO_LEVEL, info_log_, @@ -946,21 +1037,90 @@ Status CloudEnvImpl::RollNewEpoch(const std::string& local_dbname) { // upload new manifest, only if we have it (i.e. this is not a new // database, indicated by maxFileNumber) if (maxFileNumber > 0) { - st = PutObject(ManifestFileWithEpoch(local_dbname, newEpoch), - GetDestBucketName(), - ManifestFileWithEpoch(GetDestObjectPath(), newEpoch)); + st = cloud_env_options.storage_provider->PutObject( + ManifestFileWithEpoch(local_dbname, newEpoch), GetDestBucketName(), + ManifestFileWithEpoch(GetDestObjectPath(), newEpoch)); if (!st.ok()) { return st; } } // upload new cloud manifest - st = PutObject(CloudManifestFile(local_dbname), GetDestBucketName(), - CloudManifestFile(GetDestObjectPath())); + st = cloud_env_options.storage_provider->PutObject(CloudManifestFile(local_dbname), + GetDestBucketName(), + CloudManifestFile(GetDestObjectPath())); if (!st.ok()) { return st; } } return Status::OK(); } + +Status CloudEnvImpl::CopyLocalFileToDest(const std::string& local_name, + const std::string& dest_name) { + RemoveFileFromDeletionQueue(basename(local_name)); + return cloud_env_options.storage_provider->PutObject(local_name, GetDestBucketName(), + dest_name); +} + +void CloudEnvImpl::RemoveFileFromDeletionQueue(const std::string& filename) { + std::lock_guard lk(files_to_delete_mutex_); + auto itr = files_to_delete_.find(filename); + if (itr != files_to_delete_.end()) { + GetJobExecutor()->CancelJob(itr->second.get()); + files_to_delete_.erase(itr); + } +} + +Status CloudEnvImpl::DeleteCloudFileFromDest(const std::string& fname) { + assert(HasDestBucket()); + auto base = basename(fname); + // add the job to delete the file in 1 hour + auto doDeleteFile = [this, base]() { + { + std::lock_guard lk(files_to_delete_mutex_); + auto itr = files_to_delete_.find(base); + if (itr == files_to_delete_.end()) { + // File was removed from files_to_delete_, do not delete! + return; + } + files_to_delete_.erase(itr); + } + auto path = GetDestObjectPath() + "/" + base; + // we are ready to delete the file! + auto st = cloud_env_options.storage_provider->DeleteObject(GetDestBucketName(), path); + if (!st.ok() && !st.IsNotFound()) { + Log(InfoLogLevel::ERROR_LEVEL, info_log_, + "[s3] DeleteFile DeletePathInS3 file %s error %s", path.c_str(), + st.ToString().c_str()); + } + }; + { + std::lock_guard lk(files_to_delete_mutex_); + if (files_to_delete_.find(base) != files_to_delete_.end()) { + // already in the queue + return Status::OK(); + } + } + { + std::lock_guard lk(files_to_delete_mutex_); + auto handle = GetJobExecutor()->ScheduleJob( + std::chrono::steady_clock::now() + file_deletion_delay_, + std::move(doDeleteFile)); + files_to_delete_.emplace(base, std::move(handle)); + } + return Status::OK(); +} + +Status CloudEnvImpl::Prepare() { + Status s = CloudEnv::Prepare(); + if (s.ok()) { + if (HasDestBucket() && cloud_env_options.run_purger) { + // start the purge thread only if there is a destination bucket + purge_thread_ = std::thread([this] { Purger(); }); + } + } + return s; +} + } // namespace rocksdb #endif // ROCKSDB_LITE diff --git a/cloud/cloud_env_impl.h b/cloud/cloud_env_impl.h index 78b27941b36..a2158a7a16f 100644 --- a/cloud/cloud_env_impl.h +++ b/cloud/cloud_env_impl.h @@ -11,6 +11,9 @@ #include "rocksdb/status.h" namespace rocksdb { +namespace detail { +struct JobHandle; +} // namespace detail // // The Cloud environment @@ -20,15 +23,16 @@ class CloudEnvImpl : public CloudEnv { public: // Constructor - CloudEnvImpl(const CloudEnvOptions & options, Env* base_env, const std::shared_ptr& logger); + CloudEnvImpl(const CloudEnvOptions & options, Env* base_env, const std::shared_ptr & logger); virtual ~CloudEnvImpl(); - const CloudType& GetCloudType() const { return cloud_env_options.cloud_type; } - Status SanitizeDirectory(const DBOptions& options, const std::string& clone_name, bool read_only); Status LoadCloudManifest(const std::string& local_dbname, bool read_only); + // Deletes file from a destination bucket. + Status DeleteCloudFileFromDest(const std::string& fname) override; + // The separator used to separate dbids while creating the dbid of a clone static constexpr const char* DBID_SEPARATOR = "rockset"; @@ -97,6 +101,14 @@ class CloudEnvImpl : public CloudEnv { return base_env_->GetThreadList(thread_list); } + void TEST_SetFileDeletionDelay(std::chrono::seconds delay) { + std::lock_guard lk(files_to_delete_mutex_); + file_deletion_delay_ = delay; + } + Status CopyLocalFileToDest(const std::string& local_name, + const std::string& cloud_name) override; + void RemoveFileFromDeletionQueue(const std::string& filename); + virtual Status Prepare() override; protected: // Does the dir need to be re-initialized? Status NeedsReinitialization(const std::string& clone_dir, bool* do_reinit); @@ -131,6 +143,11 @@ class CloudEnvImpl : public CloudEnv { void Purger(); void StopPurger(); + std::mutex files_to_delete_mutex_; + std::chrono::seconds file_deletion_delay_ = std::chrono::hours(1); + std::unordered_map> + files_to_delete_; + private: Status writeCloudManifest(CloudManifest* manifest, const std::string& fname); std::string generateNewEpochId(); diff --git a/cloud/cloud_env_options.cc b/cloud/cloud_env_options.cc index a4d20f6c0d8..8ef23815db8 100644 --- a/cloud/cloud_env_options.cc +++ b/cloud/cloud_env_options.cc @@ -1,17 +1,20 @@ // Copyright (c) 2017 Rockset. #ifndef ROCKSDB_LITE +#include "rocksdb/cloud/cloud_env_options.h" #include -#include "cloud/cloud_env_impl.h" -#include "cloud/cloud_env_wrapper.h" -#include "cloud/db_cloud_impl.h" -#include "rocksdb/env.h" +#include "rocksdb/cloud/cloud_log_controller.h" +#include "rocksdb/cloud/cloud_storage_provider.h" namespace rocksdb { void CloudEnvOptions::Dump(Logger* log) const { - Header(log, " COptions.cloud_type: %u", cloud_type); - Header(log, " COptions.log_type: %u", log_type); + Header(log, " COptions.cloud_type: %s", + ((storage_provider != nullptr) ? + storage_provider->Name() : "None")); + Header(log, " COptions.log_type: %s", + ((log_controller != nullptr) ? + log_controller->Name() : "None")); Header(log, " COptions.keep_local_sst_files: %d", keep_local_sst_files); Header(log, " COptions.keep_local_log_files: %d", diff --git a/cloud/cloud_env_wrapper.h b/cloud/cloud_env_wrapper.h index da9f6e754e4..bf2d1fec8ce 100644 --- a/cloud/cloud_env_wrapper.h +++ b/cloud/cloud_env_wrapper.h @@ -3,64 +3,156 @@ #pragma once #include #include + #include "cloud/cloud_env_impl.h" #include "rocksdb/cloud/cloud_env_options.h" +#include "rocksdb/cloud/cloud_storage_provider.h" #include "rocksdb/env.h" #include "rocksdb/status.h" namespace rocksdb { -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" +class MockStorageProvider : public CloudStorageProvider { + public: + MockStorageProvider() { + notsup_ = Status::NotSupported(); + } + virtual const char *Name() const override { return "Mock"; } + virtual Status CreateBucket(const std::string& /*bucket_name*/) override { + return notsup_; + } + + virtual Status ExistsBucket(const std::string& /*bucket_name*/) override { + return notsup_; + } + virtual Status EmptyBucket(const std::string& /*bucket_name*/, + const std::string& /*path_prefix*/) override { + return notsup_; + } + Status ListObjects(const std::string& /*bucket_name*/, + const std::string& /*object_path*/, + std::vector* /*result*/) override { + return notsup_; + } + Status DeleteObject(const std::string& /*bucket_name*/, + const std::string& /*object_path*/) override { + return notsup_; + } + Status ExistsObject(const std::string& /*bucket_name*/, + const std::string& /*object_path*/) override { + return notsup_; + } + Status GetObjectSize(const std::string& /*bucket_name*/, + const std::string& /*object_path*/, + uint64_t* /*size*/) override { + return notsup_; + } + Status GetObjectModificationTime(const std::string& /*bucket_name*/, + const std::string& /*object_path*/, + uint64_t* /*time*/) override { + return notsup_; + } + Status GetObjectMetadata(const std::string& /*bucket_name*/, + const std::string& /*object_path*/, + std::unordered_map* /*metadata*/) override { + return notsup_; + } + Status CopyObject(const std::string& /*bucket_name_src*/, + const std::string& /*object_path_src*/, + const std::string& /*bucket_name_dest*/, + const std::string& /*object_path_dest*/) override { + return notsup_; + } + Status PutObjectMetadata(const std::string& /*bucket_name*/, + const std::string& /*object_path*/, + const std::unordered_map& /*metadata*/) override { + return notsup_; + } + Status NewCloudWritableFile( + const std::string& /*local_path*/, + const std::string& /*bucket_name*/, + const std::string& /*object_path*/, + std::unique_ptr* /*result*/, + const EnvOptions& /*options*/) override { + return notsup_; + } + +protected: + Status DoNewCloudReadableFile(const std::string& /*bucket*/, + const std::string& /*fname*/, + uint64_t /*fsize*/, + std::unique_ptr* /*result*/) override { + return notsup_; + } + Status DoGetObject(const std::string& /*bucket_name*/, + const std::string& /*object_path*/, + const std::string& /*local_path*/, + uint64_t* /*remote_size*/) override { + return notsup_; + } + Status DoPutObject(const std::string& /*local_path*/, + const std::string& /*bucket_name*/, + const std::string& /*object_path*/, + uint64_t /*file_size*/) override { + return notsup_; + } + + Status notsup_; +}; // An implementation of Env that forwards all calls to another Env. // May be useful to clients who wish to override just part of the // functionality of another Env. -class CloudEnvWrapper : public CloudEnvImpl { + +class MockCloudEnv : public CloudEnv { public: // Initialize an EnvWrapper that delegates all calls to *t - explicit CloudEnvWrapper(const CloudEnvOptions& options, Env* t, const std::shared_ptr& l) : - CloudEnvImpl(options, t, l) { - cloud_env_options.log_type = LogType::kLogNone; - cloud_env_options.cloud_type = CloudType::kCloudNone; + explicit MockCloudEnv(const CloudEnvOptions & opts = CloudEnvOptions()) + : CloudEnv(opts, Env::Default()) { notsup_ = Status::NotSupported(); } - virtual ~CloudEnvWrapper(); + virtual ~MockCloudEnv() {} - virtual Status EmptyBucket(const std::string& /*bucket*/, - const std::string& /*path_prefix*/) override { + const char *Name() const override { return "MockCloudEnv"; } + + Status CopyLocalFileToDest(const std::string& /*local_name*/, + const std::string& /*dest_name*/) override { return notsup_; } - virtual Status NewSequentialFileCloud(const std::string& bucket_prefix, - const std::string& fname, - std::unique_ptr* result, - const EnvOptions& options) override { + Status PreloadCloudManifest(const std::string& /*local_dbname*/) override { return notsup_; } - virtual Status SaveDbid(const std::string& bucket_name, - const std::string& dbid, - const std::string& dirname) override { + + virtual Status NewSequentialFileCloud( + const std::string& /*bucket_name*/, const std::string& /*fname*/, + std::unique_ptr* /*result*/, + const EnvOptions& /*options*/) override { return notsup_; } - virtual Status GetPathForDbid(const std::string& bucket_prefix, - const std::string& dbid, - std::string* dirname) override { + virtual Status SaveDbid(const std::string& /*bucket_name*/, + const std::string& /*dbid */, + const std::string& /*dirname*/) override { return notsup_; } - virtual Status GetDbidList(const std::string& bucket_prefix, - DbidList* dblist) override { + virtual Status GetPathForDbid(const std::string& /*bucket_name*/, + const std::string& /*dbid*/, + std::string* /*dirname*/) override { return notsup_; } - virtual Status DeleteDbid(const std::string& bucket_prefix, - const std::string& dbid) override { + virtual Status GetDbidList(const std::string& /*bucket_name*/, + DbidList* /*dblist*/) override { + return notsup_; + } + virtual Status DeleteDbid(const std::string& /*bucket_name*/, + const std::string& /*dbid*/) override { return notsup_; } // Ability to read a file directly from cloud storage - virtual Status NewSequentialFileCloud(const std::string& fname, - std::unique_ptr* result, - const EnvOptions& options) { + virtual Status NewSequentialFileCloud(const std::string& /*fname*/, + std::unique_ptr* /*result*/, + const EnvOptions& /*options*/) { return notsup_; } @@ -207,42 +299,7 @@ class CloudEnvWrapper : public CloudEnvImpl { uint64_t GetThreadID() const override { return base_env_->GetThreadID(); } - - Status ListObjects(const std::string& bucket_name_prefix, - const std::string& bucket_object_prefix, - BucketObjectMetadata* meta) override { - return notsup_; - } - Status DeleteObject(const std::string& bucket_name_prefix, - const std::string& bucket_object_path) override { - return notsup_; - } - Status ExistsObject(const std::string& bucket_name_prefix, - const std::string& bucket_object_path) override { - return notsup_; - } - Status GetObjectSize(const std::string& bucket_name_prefix, - const std::string& bucket_object_path, - uint64_t* size) override { - return notsup_; - } - Status CopyObject(const std::string& bucket_name_prefix_src, - const std::string& bucket_object_path_src, - const std::string& bucket_name_prefix_dest, - const std::string& bucket_object_path_dest) override { - return notsup_; - } - Status GetObject(const std::string& bucket_name_prefix, - const std::string& bucket_object_path, - const std::string& local_path) override { - return notsup_; - } - Status PutObject(const std::string& local_path, - const std::string& bucket_name_prefix, - const std::string& bucket_object_path) override { - return notsup_; - } - Status DeleteCloudFileFromDest(const std::string& path) override { + Status DeleteCloudFileFromDest(const std::string& /*path*/) override { return notsup_; } @@ -250,6 +307,4 @@ class CloudEnvWrapper : public CloudEnvImpl { Status notsup_; std::string empty_; }; - -#pragma GCC diagnostic pop } // namespace rocksdb diff --git a/cloud/cloud_log_controller.cc b/cloud/cloud_log_controller.cc index 7ffbf404292..dab9fe6baac 100644 --- a/cloud/cloud_log_controller.cc +++ b/cloud/cloud_log_controller.cc @@ -4,7 +4,7 @@ // A log file maps to a stream in Kinesis. // -#include "cloud/cloud_log_controller.h" +#include "rocksdb/cloud/cloud_log_controller.h" #include #include @@ -18,49 +18,27 @@ #include "util/string_util.h" namespace rocksdb { -CloudLogWritableFile::CloudLogWritableFile( - CloudEnv* env, const std::string& fname, const EnvOptions& /*options*/) - : env_(env), fname_(fname) {} +CloudLogWritableFile::CloudLogWritableFile(CloudEnv* env, + const std::string& fname, + const EnvOptions& /*options*/) + : env_(env), fname_(fname) {} CloudLogWritableFile::~CloudLogWritableFile() {} const std::chrono::microseconds CloudLogController::kRetryPeriod = std::chrono::seconds(30); -CloudLogController::CloudLogController(CloudEnv* env) - : env_(env), running_(false) { - - // Create a random number for the cache directory. - const std::string uid = trim(env_->GetBaseEnv()->GenerateUniqueId()); - - // Temporary directory for cache. - const std::string bucket_dir = kCacheDir + pathsep + env_->GetSrcBucketName(); - cache_dir_ = bucket_dir + pathsep + uid; - - // Create temporary directories. - status_ = env_->GetBaseEnv()->CreateDirIfMissing(kCacheDir); - if (status_.ok()) { - status_ = env_->GetBaseEnv()->CreateDirIfMissing(bucket_dir); - } - if (status_.ok()) { - status_ = env_->GetBaseEnv()->CreateDirIfMissing(cache_dir_); - } -} - CloudLogController::~CloudLogController() { if (running_) { - // This is probably not a good situation as the derived class is partially destroyed - // but the tailer might still be active. - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[%s] CloudLogController closing. Stopping stream.", Name()); + // This is probably not a good situation as the derived class is partially + // destroyed but the tailer might still be active. StopTailingStream(); } - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[%s] CloudLogController closed.", Name()); } -std::string CloudLogController::GetCachePath(const Slice& original_pathname) const { - const std::string & cache_dir = GetCacheDir(); +std::string CloudLogController::GetCachePath( + const Slice& original_pathname) const { + const std::string& cache_dir = GetCacheDir(); return cache_dir + pathsep + basename(original_pathname.ToString()); } @@ -82,7 +60,7 @@ Status CloudLogController::Apply(const Slice& in) { // Apply operation on cache file. if (operation == kAppend) { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[%s] Tailer: Appending %ld bytes to %s at offset %" PRIu64, Name(), payload.size(), pathname.c_str(), offset_in_file); @@ -107,9 +85,9 @@ Status CloudLogController::Apply(const Slice& in) { if (st.ok()) { cache_fds_[pathname] = std::move(result); - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[%s] Tailer: Successfully opened file %s and cached", - Name(), pathname.c_str()); + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), + "[%s] Tailer: Successfully opened file %s and cached", Name(), + pathname.c_str()); } else { return st; } @@ -118,7 +96,7 @@ Status CloudLogController::Apply(const Slice& in) { RandomRWFile* fd = cache_fds_[pathname].get(); st = fd->Write(offset_in_file, payload); if (!st.ok()) { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[%s] Tailer: Error writing to cached file %s: %s", pathname.c_str(), Name(), st.ToString().c_str()); } @@ -126,18 +104,19 @@ Status CloudLogController::Apply(const Slice& in) { // Delete file from cache directory. auto iter = cache_fds_.find(pathname); if (iter != cache_fds_.end()) { - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), "[%s] Tailer: Delete file %s, but it is still open." - " Closing it now..", Name(), pathname.c_str()); + " Closing it now..", + Name(), pathname.c_str()); RandomRWFile* fd = iter->second.get(); fd->Close(); cache_fds_.erase(iter); } st = env_->GetBaseEnv()->DeleteFile(pathname); - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[%s] Tailer: Deleted file: %s %s", - Name(), pathname.c_str(), st.ToString().c_str()); + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), + "[%s] Tailer: Deleted file: %s %s", Name(), pathname.c_str(), + st.ToString().c_str()); if (st.IsNotFound()) { st = Status::OK(); @@ -149,15 +128,14 @@ Status CloudLogController::Apply(const Slice& in) { st = fd->Close(); cache_fds_.erase(iter); } - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[%s] Tailer: Closed file %s %s", - Name(), pathname.c_str(), st.ToString().c_str()); + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), + "[%s] Tailer: Closed file %s %s", Name(), pathname.c_str(), + st.ToString().c_str()); } else { st = Status::IOError("Unknown operation"); - Log(InfoLogLevel::DEBUG_LEVEL, env_->info_log_, - "[%s] Tailer: Unknown operation '%x': File %s %s", - Name(), operation, pathname.c_str(), - st.ToString().c_str()); + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), + "[%s] Tailer: Unknown operation '%x': File %s %s", Name(), operation, + pathname.c_str(), st.ToString().c_str()); } return st; @@ -238,7 +216,7 @@ bool CloudLogController::ExtractLogRecord(const Slice& input, return true; } -Status CloudLogController::StartTailingStream(const std::string & topic) { +Status CloudLogController::StartTailingStream(const std::string& topic) { if (tid_) { return Status::Busy("Tailer already started"); } @@ -266,7 +244,7 @@ void CloudLogController::StopTailingStream() { Status CloudLogController::Retry(RetryType func) { Status stat; std::chrono::microseconds start(env_->NowMicros()); - + while (true) { // If command is successful, return immediately stat = func(); @@ -285,5 +263,46 @@ Status CloudLogController::Retry(RetryType func) { } return stat; } + +Status CloudLogController::Initialize(CloudEnv *env) { + env_ = env; + // Create a random number for the cache directory. + const std::string uid = trim(env_->GetBaseEnv()->GenerateUniqueId()); + + // Temporary directory for cache. + const std::string bucket_dir = kCacheDir + pathsep + env_->GetSrcBucketName(); + cache_dir_ = bucket_dir + pathsep + uid; + + // Create temporary directories. + Status status = env_->GetBaseEnv()->CreateDirIfMissing(kCacheDir); + if (status.ok()) { + status = env_->GetBaseEnv()->CreateDirIfMissing(bucket_dir); + } + if (status.ok()) { + status = env_->GetBaseEnv()->CreateDirIfMissing(cache_dir_); + } + return status; +} + +Status CloudLogController::Prepare(CloudEnv *env) { + if (env != nullptr) { + status_ = Initialize(env); + } else { + status_ = Status::NotSupported("LogController requires a CloudEnv"); + } + if (status_.ok()) { + status_ = StartTailingStream(env->GetSrcBucketName()); + } + return status_; +} + +Status CloudLogController::Verify() const { + if (!status_.ok()) { + return status_; + } else if (!env_) { + return Status::InvalidArgument("Log Controller not initialized: ", Name()); + } + return status_; +} } // namespace rocksdb diff --git a/cloud/cloud_storage_provider.cc b/cloud/cloud_storage_provider.cc new file mode 100644 index 00000000000..02ae507e3ae --- /dev/null +++ b/cloud/cloud_storage_provider.cc @@ -0,0 +1,379 @@ +// Copyright (c) 2016-present, Rockset, Inc. All rights reserved. +// + +#include "rocksdb/cloud/cloud_storage_provider.h" + +#include +#include +#include + +#include "cloud/cloud_env_impl.h" +#include "cloud/filename.h" +#include "cloud/cloud_storage_provider_impl.h" +#include "file/filename.h" +#include "rocksdb/cloud/cloud_env_options.h" +#include "rocksdb/env.h" +#include "rocksdb/options.h" +#include "rocksdb/status.h" +#include "util/coding.h" +#include "util/stderr_logger.h" +#include "util/string_util.h" + +namespace rocksdb { + +/******************** Readablefile ******************/ +CloudStorageReadableFile::CloudStorageReadableFile( + const std::shared_ptr& info_log, const std::string& bucket, + const std::string& fname, uint64_t file_size) + : info_log_(info_log), + bucket_(bucket), + fname_(fname), + offset_(0), + file_size_(file_size) { + Log(InfoLogLevel::DEBUG_LEVEL, info_log_, + "[%s] CloudReadableFile opening file %s", Name(), fname_.c_str()); +} + +// sequential access, read data at current offset in file +Status CloudStorageReadableFile::Read(size_t n, Slice* result, char* scratch) { + Log(InfoLogLevel::DEBUG_LEVEL, info_log_, + "[%s] CloudReadableFile reading %s %ld", Name(), fname_.c_str(), n); + Status s = Read(offset_, n, result, scratch); + + // If the read successfully returned some data, then update + // offset_ + if (s.ok()) { + offset_ += result->size(); + } + return s; +} + +// random access, read data from specified offset in file +Status CloudStorageReadableFile::Read(uint64_t offset, size_t n, Slice* result, + char* scratch) const { + Log(InfoLogLevel::DEBUG_LEVEL, info_log_, + "[%s] CloudReadableFile reading %s at offset %" PRIu64 " size %" ROCKSDB_PRIszt, + Name(), fname_.c_str(), offset, n); + + *result = Slice(); + + if (offset >= file_size_) { + Log(InfoLogLevel::DEBUG_LEVEL, info_log_, + "[%s] CloudReadableFile reading %s at offset %" PRIu64 + " filesize %" PRIu64 ". Nothing to do", + Name(), fname_.c_str(), offset, file_size_); + return Status::OK(); + } + + // trim size if needed + if (offset + n > file_size_) { + n = file_size_ - offset; + Log(InfoLogLevel::DEBUG_LEVEL, info_log_, + "[%s] CloudReadableFile reading %s at offset %" PRIu64 + " trimmed size %ld", + Name(), fname_.c_str(), offset, n); + } + uint64_t bytes_read; + Status st = DoCloudRead(offset, n, scratch, &bytes_read); + if (st.ok()) { + *result = Slice(scratch, bytes_read); + Log(InfoLogLevel::DEBUG_LEVEL, info_log_, + "[%s] CloudReadableFile file %s filesize %" PRIu64 " read %" PRIu64 " bytes", + Name(), fname_.c_str(), file_size_, bytes_read); + } + return st; +} + +Status CloudStorageReadableFile::Skip(uint64_t n) { + Log(InfoLogLevel::DEBUG_LEVEL, info_log_, + "[%s] CloudReadableFile file %s skip %" PRIu64, Name(), fname_.c_str(), n); + // Update offset_ so that it does not go beyond filesize + offset_ += n; + if (offset_ > file_size_) { + offset_ = file_size_; + } + return Status::OK(); +} + +size_t CloudStorageReadableFile::GetUniqueId(char* id, size_t max_size) const { + // If this is an SST file name, then it can part of the persistent cache. + // We need to generate a unique id for the cache. + // If it is not a sst file, then nobody should be using this id. + uint64_t file_number; + FileType file_type; + WalFileType log_type; + ParseFileName(RemoveEpoch(basename(fname_)), &file_number, &file_type, + &log_type); + if (max_size < kMaxVarint64Length && file_number > 0) { + char* rid = id; + rid = EncodeVarint64(rid, file_number); + return static_cast(rid - id); + } + return 0; +} + +/******************** Writablefile ******************/ + +CloudStorageWritableFile::CloudStorageWritableFile( + CloudEnv* env, const std::string& local_fname, const std::string& bucket, + const std::string& cloud_fname, const EnvOptions& options) + : env_(env), + fname_(local_fname), + bucket_(bucket), + cloud_fname_(cloud_fname) { + auto fname_no_epoch = RemoveEpoch(fname_); + // Is this a manifest file? + is_manifest_ = IsManifestFile(fname_no_epoch); + assert(IsSstFile(fname_no_epoch) || is_manifest_); + + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), + "[%s] CloudWritableFile bucket %s opened local file %s " + "cloud file %s manifest %d", + Name(), bucket.c_str(), fname_.c_str(), cloud_fname.c_str(), + is_manifest_); + + auto* file_to_open = &fname_; + auto local_env = env_->GetBaseEnv(); + Status s; + if (is_manifest_) { + s = local_env->FileExists(fname_); + if (!s.ok() && !s.IsNotFound()) { + status_ = s; + return; + } + if (s.ok()) { + // Manifest exists. Instead of overwriting the MANIFEST (which could be + // bad if we crash mid-write), write to the temporary file and do an + // atomic rename on Sync() (Sync means we have a valid data in the + // MANIFEST, so we can crash after it) + tmp_file_ = fname_ + ".tmp"; + file_to_open = &tmp_file_; + } + } + + s = local_env->NewWritableFile(*file_to_open, &local_file_, options); + if (!s.ok()) { + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[%s] CloudWritableFile src %s %s", Name(), fname_.c_str(), + s.ToString().c_str()); + status_ = s; + } +} + +CloudStorageWritableFile::~CloudStorageWritableFile() { + if (local_file_ != nullptr) { + Close(); + } +} + +Status CloudStorageWritableFile::Close() { + if (local_file_ == nullptr) { // already closed + return status_; + } + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), + "[%s] CloudWritableFile closing %s", Name(), fname_.c_str()); + assert(status_.ok()); + + // close local file + Status st = local_file_->Close(); + if (!st.ok()) { + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[%s] CloudWritableFile closing error on local %s\n", Name(), + fname_.c_str()); + return st; + } + local_file_.reset(); + + if (!is_manifest_) { + status_ = env_->CopyLocalFileToDest(fname_, cloud_fname_); + if (!status_.ok()) { + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[%s] CloudWritableFile closing PutObject failed on local file %s", + Name(), fname_.c_str()); + return status_; + } + + // delete local file + if (!env_->GetCloudEnvOptions().keep_local_sst_files) { + status_ = env_->GetBaseEnv()->DeleteFile(fname_); + if (!status_.ok()) { + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[%s] CloudWritableFile closing delete failed on local file %s", + Name(), fname_.c_str()); + return status_; + } + } + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), + "[%s] CloudWritableFile closed file %s", Name(), fname_.c_str()); + } + return Status::OK(); +} + +// Sync a file to stable storage +Status CloudStorageWritableFile::Sync() { + if (local_file_ == nullptr) { + return status_; + } + assert(status_.ok()); + + // sync local file + Status stat = local_file_->Sync(); + + if (stat.ok() && !tmp_file_.empty()) { + assert(is_manifest_); + // We are writing to the temporary file. On a first sync we need to rename + // the file to the real filename. + stat = env_->GetBaseEnv()->RenameFile(tmp_file_, fname_); + // Note: this is not thread safe, but we know that manifest writes happen + // from the same thread, so we are fine. + tmp_file_.clear(); + } + + // We copy MANIFEST to cloud on every Sync() + if (is_manifest_ && stat.ok()) { + stat = env_->CopyLocalFileToDest(fname_, cloud_fname_); + if (stat.ok()) { + Log(InfoLogLevel::DEBUG_LEVEL, env_->GetLogger(), + "[%s] CloudWritableFile made manifest %s durable to " + "bucket %s bucketpath %s.", Name(), + fname_.c_str(), bucket_.c_str(), cloud_fname_.c_str()); + } else { + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[%s] CloudWritableFile failed to make manifest %s durable to " + "bucket %s bucketpath %s: %s", Name(), + fname_.c_str(), bucket_.c_str(), cloud_fname_.c_str(), + stat.ToString().c_str()); + } + } + return stat; +} + +CloudStorageProvider::~CloudStorageProvider() {} + +Status CloudStorageProvider::Prepare(CloudEnv *env) { + Status status; + if (env->HasDestBucket()) { + // create dest bucket if specified + if (ExistsBucket(env->GetDestBucketName()).ok()) { + Log(InfoLogLevel::INFO_LEVEL, env->GetLogger(), + "[%s] Bucket %s already exists", Name(), + env->GetDestBucketName().c_str()); + } else if (env->GetCloudEnvOptions().create_bucket_if_missing) { + Log(InfoLogLevel::INFO_LEVEL, env->GetLogger(), + "[%s] Going to create bucket %s", Name(), + env->GetDestBucketName().c_str()); + status = CreateBucket(env->GetDestBucketName()); + } else { + status = Status::NotFound( + "Bucket not found and create_bucket_if_missing is false"); + } + if (!status.ok()) { + Log(InfoLogLevel::ERROR_LEVEL, env->GetLogger(), + "[%s] Unable to create bucket %s %s", Name(), + env->GetDestBucketName().c_str(), status.ToString().c_str()); + return status; + } + } + return status; +} + +Status CloudStorageProviderImpl::Verify() const { + if (!status_.ok()) { + return status_; + } else if (!env_) { + return Status::InvalidArgument("Storage Provider not initialized: ", Name()); + } else { + return Status::OK(); + } +} + +Status CloudStorageProviderImpl::Prepare(CloudEnv *env) { + status_ = Initialize(env); + if (status_.ok()) { + status_ = CloudStorageProvider::Prepare(env); + } + if (status_.ok()) { + env_ = env; + } + return status_; +} + +Status CloudStorageProviderImpl::Initialize(CloudEnv* env) { + env_ = env; + return Status::OK(); +} + +Status CloudStorageProviderImpl::NewCloudReadableFile( + const std::string& bucket, const std::string& fname, + std::unique_ptr* result) { + // First, check if the file exists and also find its size. We use size in + // CloudReadableFile to make sure we always read the valid ranges of the file + uint64_t size; + Status st = GetObjectSize(bucket, fname, &size); + if (!st.ok()) { + return st; + } + return DoNewCloudReadableFile(bucket, fname, size, result); +} + +Status CloudStorageProviderImpl::GetObject(const std::string& bucket_name, + const std::string& object_path, + const std::string& local_destination) { + Env* localenv = env_->GetBaseEnv(); + std::string tmp_destination = local_destination + ".tmp"; + uint64_t remote_size; + Status s = + DoGetObject(bucket_name, object_path, tmp_destination, &remote_size); + if (!s.ok()) { + localenv->DeleteFile(tmp_destination); + return s; + } + + // Check if our local file is the same as promised + uint64_t local_size{0}; + s = localenv->GetFileSize(tmp_destination, &local_size); + if (!s.ok()) { + return s; + } + if (local_size != remote_size) { + localenv->DeleteFile(tmp_destination); + s = Status::IOError("Partial download of a file " + local_destination); + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[%s] GetObject %s/%s local size %" PRIu64 " != cloud size " + "%" PRIu64 ". %s", Name(), + bucket_name.c_str(), object_path.c_str(), local_size, remote_size, + s.ToString().c_str()); + } + + if (s.ok()) { + s = localenv->RenameFile(tmp_destination, local_destination); + } + Log(InfoLogLevel::INFO_LEVEL, env_->GetLogger(), + "[%s] GetObject %s/%s size %" PRIu64 ". %s", bucket_name.c_str(), Name(), + object_path.c_str(), local_size, s.ToString().c_str()); + return s; +} + +Status CloudStorageProviderImpl::PutObject(const std::string& local_file, + const std::string& bucket_name, + const std::string& object_path) { + uint64_t fsize = 0; + // debugging paranoia. Files uploaded to Cloud can never be zero size. + auto st = env_->GetBaseEnv()->GetFileSize(local_file, &fsize); + if (!st.ok()) { + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[%s] PutObject localpath %s error getting size %s", Name(), + local_file.c_str(), st.ToString().c_str()); + return st; + } + if (fsize == 0) { + Log(InfoLogLevel::ERROR_LEVEL, env_->GetLogger(), + "[%s] PutObject localpath %s error zero size", Name(), + local_file.c_str()); + return Status::IOError(local_file + " Zero size."); + } + + return DoPutObject(local_file, bucket_name, object_path, fsize); +} + +} // namespace rocksdb diff --git a/cloud/cloud_storage_provider_impl.h b/cloud/cloud_storage_provider_impl.h new file mode 100644 index 00000000000..2e6a224565b --- /dev/null +++ b/cloud/cloud_storage_provider_impl.h @@ -0,0 +1,49 @@ +// Copyright (c) 2017 Rockset + +#pragma once + +#ifndef ROCKSDB_LITE +#include "rocksdb/cloud/cloud_storage_provider.h" + +namespace rocksdb { + +// +// All writes to this DB can be configured to be persisted +// in cloud storage. +// +class CloudStorageProviderImpl : public CloudStorageProvider { +public: + Status GetObject(const std::string& bucket_name, + const std::string& object_path, + const std::string& local_destination) override; + Status PutObject(const std::string& local_file, + const std::string& bucket_name, + const std::string& object_path) override; + Status NewCloudReadableFile( + const std::string& bucket, const std::string& fname, + std::unique_ptr* result) override; + virtual Status Prepare(CloudEnv *env) override; + virtual Status Verify() const override; + protected: + virtual Status Initialize(CloudEnv* env); + + + virtual Status DoNewCloudReadableFile( + const std::string& bucket, const std::string& fname, uint64_t fsize, + std::unique_ptr* result) = 0; + // Downloads object from the cloud into a local directory + virtual Status DoGetObject(const std::string& bucket_name, + const std::string& object_path, + const std::string& local_path, + uint64_t* remote_size) = 0; + virtual Status DoPutObject(const std::string& local_file, + const std::string& object_path, + const std::string& bucket_name, + uint64_t file_size) = 0; + + CloudEnv* env_; + Status status_; +}; +} // namespace rocksdb + +#endif // ROCKSDB_LITE diff --git a/cloud/db_cloud_impl.cc b/cloud/db_cloud_impl.cc index 51a6072bc53..989a70c03fe 100644 --- a/cloud/db_cloud_impl.cc +++ b/cloud/db_cloud_impl.cc @@ -1,19 +1,16 @@ // Copyright (c) 2017 Rockset. #ifndef ROCKSDB_LITE -#ifndef __STDC_FORMAT_MACROS -#define __STDC_FORMAT_MACROS -#endif - #include "cloud/db_cloud_impl.h" -#include +#include #include "cloud/aws/aws_env.h" #include "cloud/filename.h" #include "cloud/manifest_reader.h" #include "file/file_util.h" #include "logging/auto_roll_logger.h" +#include "rocksdb/cloud/cloud_storage_provider.h" #include "rocksdb/db.h" #include "rocksdb/env.h" #include "rocksdb/options.h" @@ -172,12 +169,13 @@ Status DBCloudImpl::Savepoint() { std::vector live_files; GetLiveFilesMetaData(&live_files); + auto & provider = cenv->GetCloudEnvOptions().storage_provider; // If an sst file does not exist in the destination path, then remember it std::vector to_copy; for (auto onefile : live_files) { auto remapped_fname = cenv->RemapFilename(onefile.name); std::string destpath = cenv->GetDestObjectPath() + "/" + remapped_fname; - if (!cenv->ExistsObject(cenv->GetDestBucketName(), destpath).ok()) { + if (!provider->ExistsObject(cenv->GetDestBucketName(), destpath).ok()) { to_copy.push_back(remapped_fname); } } @@ -193,7 +191,7 @@ Status DBCloudImpl::Savepoint() { break; } auto& onefile = to_copy[idx]; - Status s = cenv->CopyObject( + Status s = provider->CopyObject( cenv->GetSrcBucketName(), cenv->GetSrcObjectPath() + "/" + onefile, cenv->GetDestBucketName(), cenv->GetDestObjectPath() + "/" + onefile); if (!s.ok()) { @@ -299,8 +297,8 @@ Status DBCloudImpl::DoCheckpointToCloud( } auto& f = files_to_copy[idx]; - auto copy_st = - cenv->PutObject(GetName() + "/" + f.first, destination.GetBucketName(), + auto copy_st = + cenv->GetCloudEnvOptions().storage_provider->PutObject(GetName() + "/" + f.first, destination.GetBucketName(), destination.GetObjectPath() + "/" + f.second); if (!copy_st.ok()) { thread_statuses[threadId] = std::move(copy_st); diff --git a/cloud/db_cloud_impl.h b/cloud/db_cloud_impl.h index 5ef526982bb..a3680197280 100644 --- a/cloud/db_cloud_impl.h +++ b/cloud/db_cloud_impl.h @@ -7,6 +7,7 @@ #include #include +#include "rocksdb/cloud/cloud_env_options.h" #include "rocksdb/cloud/db_cloud.h" #include "rocksdb/db.h" #include "rocksdb/env.h" diff --git a/cloud/db_cloud_test.cc b/cloud/db_cloud_test.cc index b960e0a1372..d9f169b9f95 100644 --- a/cloud/db_cloud_test.cc +++ b/cloud/db_cloud_test.cc @@ -6,15 +6,17 @@ #include "rocksdb/cloud/db_cloud.h" -#include #include #include +#include #include "cloud/aws/aws_env.h" #include "cloud/aws/aws_file.h" #include "cloud/db_cloud_impl.h" #include "cloud/filename.h" #include "cloud/manifest_reader.h" +#include "file/filename.h" +#include "rocksdb/cloud/cloud_storage_provider.h" #include "rocksdb/options.h" #include "rocksdb/status.h" #include "rocksdb/table.h" @@ -61,7 +63,8 @@ class CloudTest : public testing::Test { options_.info_log, &aenv)); aenv_.reset(aenv); // delete all pre-existing contents from the bucket - Status st = aenv_->EmptyBucket(aenv_->GetSrcBucketName(), ""); + Status st = + aenv_->GetCloudEnvOptions().storage_provider->EmptyBucket(aenv_->GetSrcBucketName(), ""); ASSERT_TRUE(st.ok() || st.IsNotFound()); aenv_.reset(); @@ -223,7 +226,8 @@ class CloudTest : public testing::Test { // loop through all the local files and validate for (std::string path: localFiles) { std::string cpath = aenv_->GetSrcObjectPath() + "/" + path; - ASSERT_OK(aenv_->GetObjectSize(aenv_->GetSrcBucketName(), cpath, &cloudSize)); + ASSERT_OK(aenv_->GetCloudEnvOptions().storage_provider->GetObjectSize( + aenv_->GetSrcBucketName(), cpath, &cloudSize)); // find the size of the file on local storage std::string lpath = dbname_ + "/" + path; @@ -659,11 +663,13 @@ TEST_F(CloudTest, Savepoint) { ((CloudEnvImpl*)cloud_env.get())->RemapFilename(flist[0].name); // source path std::string spath = cloud_env->GetSrcObjectPath() + "/" + remapped_fname; - ASSERT_OK(cloud_env->ExistsObject(cloud_env->GetSrcBucketName(), spath)); + ASSERT_OK(cloud_env->GetCloudEnvOptions().storage_provider->ExistsObject( + cloud_env->GetSrcBucketName(), spath)); // Verify that the destination path does not have any sst files std::string dpath = dest_path + "/" + remapped_fname; - ASSERT_TRUE(cloud_env->ExistsObject(cloud_env->GetSrcBucketName(), dpath) + ASSERT_TRUE(cloud_env->GetCloudEnvOptions().storage_provider + ->ExistsObject(cloud_env->GetSrcBucketName(), dpath) .IsNotFound()); // write a new value to the clone @@ -676,7 +682,8 @@ TEST_F(CloudTest, Savepoint) { ASSERT_OK(cloud_db->Savepoint()); // check that the sst file is copied to dest path - ASSERT_OK(cloud_env->ExistsObject(cloud_env->GetSrcBucketName(), dpath)); + ASSERT_OK(cloud_env->GetCloudEnvOptions().storage_provider->ExistsObject( + cloud_env->GetSrcBucketName(), dpath)); ASSERT_OK(cloud_db->Flush(FlushOptions())); } { @@ -830,7 +837,8 @@ TEST_F(CloudTest, TwoDBsOneBucket) { auto firstManifestFile = aenv_->GetDestObjectPath() + "/" + ((CloudEnvImpl*)aenv_.get())->RemapFilename("MANIFEST-1"); - EXPECT_OK(aenv_->ExistsObject(aenv_->GetDestBucketName(), firstManifestFile)); + EXPECT_OK(aenv_->GetCloudEnvOptions().storage_provider->ExistsObject(aenv_->GetDestBucketName(), + firstManifestFile)); // Create two files ASSERT_OK(db_->Put(WriteOptions(), "First", "File")); ASSERT_OK(db_->Flush(FlushOptions())); @@ -891,7 +899,8 @@ TEST_F(CloudTest, TwoDBsOneBucket) { // We need to sleep a bit because file deletion happens in a different thread, // so it might not be immediately deleted. std::this_thread::sleep_for(std::chrono::milliseconds(10)); - EXPECT_TRUE(aenv_->ExistsObject(aenv_->GetDestBucketName(), firstManifestFile) + EXPECT_TRUE(aenv_->GetCloudEnvOptions().storage_provider + ->ExistsObject(aenv_->GetDestBucketName(), firstManifestFile) .IsNotFound()); CloseDB(); } diff --git a/cloud/purge.cc b/cloud/purge.cc index e29793ee523..8d916393bd0 100644 --- a/cloud/purge.cc +++ b/cloud/purge.cc @@ -2,13 +2,16 @@ #ifndef ROCKSDB_LITE #include "cloud/purge.h" + #include #include + #include "cloud/aws/aws_env.h" #include "cloud/db_cloud_impl.h" #include "cloud/filename.h" #include "cloud/manifest_reader.h" #include "file/filename.h" +#include "rocksdb/cloud/cloud_storage_provider.h" #include "rocksdb/db.h" #include "rocksdb/env.h" #include "rocksdb/options.h" @@ -129,14 +132,14 @@ Status CloudEnvImpl::FindObsoleteFiles(const std::string& bucket_name_prefix, std::unique_ptr result; std::string mpath = iter->second; - BucketObjectMetadata objects; - st = ListObjects(bucket_name_prefix, mpath, &objects); + std::vector objects; + st = GetCloudEnvOptions().storage_provider->ListObjects(bucket_name_prefix, mpath, &objects); if (!st.ok()) { Log(InfoLogLevel::ERROR_LEVEL, info_log_, "[pg] Unable to list objects in bucketprefix %s path_prefix %s. %s", bucket_name_prefix.c_str(), mpath.c_str(), st.ToString().c_str()); } - for (auto& o : objects.pathnames) { + for (auto& o : objects) { all_files.push_back(mpath + "/" + o); } } @@ -167,7 +170,7 @@ Status CloudEnvImpl::FindObsoleteDbid( for (auto iter = dbid_list.begin(); iter != dbid_list.end(); ++iter) { std::unique_ptr result; std::string path = CloudManifestFile(iter->second); - st = ExistsObject(GetDestBucketName(), path); + st = GetCloudEnvOptions().storage_provider->ExistsObject(GetDestBucketName(), path); // this dbid can be cleaned up if (st.IsNotFound()) { to_delete_list->push_back(iter->first); @@ -199,7 +202,7 @@ Status CloudEnvImpl::extractParents(const std::string& bucket_name_prefix, // download IDENTITY std::string cloudfile = iter->second + "/IDENTITY"; std::string localfile = scratch + "/.rockset_IDENTITY." + random; - st = GetObject(bucket_name_prefix, cloudfile, localfile); + st = GetCloudEnvOptions().storage_provider->GetObject(bucket_name_prefix, cloudfile, localfile); if (!st.ok() && !st.IsNotFound()) { Log(InfoLogLevel::ERROR_LEVEL, info_log_, "[pg] Unable to download IDENTITY file from " diff --git a/db/db_test_util.cc b/db/db_test_util.cc index 501116d4d4a..d07585b10de 100644 --- a/db/db_test_util.cc +++ b/db/db_test_util.cc @@ -10,6 +10,7 @@ #include "db/db_test_util.h" #include "db/forward_iterator.h" #include "util/stderr_logger.h" +#include "rocksdb/cloud/cloud_storage_provider.h" #include "rocksdb/env_encryption.h" #ifdef USE_AWS #include "cloud/cloud_env_impl.h" @@ -733,7 +734,7 @@ void DBTestBase::Destroy(const Options& options, bool delete_cf_paths) { #ifdef USE_AWS if (s3_env_) { AwsEnv* aenv = static_cast(s3_env_); - Status st = aenv->EmptyBucket(aenv->GetSrcBucketName(), dbname_); + Status st = aenv->GetCloudEnvOptions().storage_provider->EmptyBucket(aenv->GetSrcBucketName(), dbname_); ASSERT_TRUE(st.ok() || st.IsNotFound()); for (int r = 0; r < 10; ++r) { // The existance is not propagated atomically in S3, so wait until diff --git a/include/rocksdb/cloud/cloud_env_options.h b/include/rocksdb/cloud/cloud_env_options.h index 88d2e58f37e..bed2a1733c9 100644 --- a/include/rocksdb/cloud/cloud_env_options.h +++ b/include/rocksdb/cloud/cloud_env_options.h @@ -19,9 +19,9 @@ struct ClientConfiguration; namespace rocksdb { -class BucketObjectMetadata; class CloudEnv; class CloudLogController; +class CloudStorageProvider; enum CloudType : unsigned char { kCloudNone = 0x0, // Not really a cloud env @@ -43,13 +43,14 @@ enum LogType : unsigned char { // Type of AWS access credentials enum class AwsAccessType { - kUndefined, // Use AWS SDK's default credential chain - kSimple, - kInstance, + kUndefined, + kDefault, // Use AWS SDK's default credential chain + kSimple, // Use simple secret/access key creds + kInstance, // Use Provider credentials kTaskRole, - kEnvironment, - kConfig, - kAnonymous, + kEnvironment, // Use credentials from environment + kConfig, // Use credentials from config file + kAnonymous, // Use anonymous credentials }; // Credentials needed to access AWS cloud service @@ -72,8 +73,8 @@ class AwsCloudAccessCredentials { Status GetCredentialsProvider( std::shared_ptr* result) const; - private: AwsAccessType GetAccessType() const; + private: Status CheckCredentials(const AwsAccessType& aws_type) const; public: @@ -152,19 +153,20 @@ inline bool operator!=(const BucketOptions& lhs, const BucketOptions& rhs) { } class AwsCloudOptions { -public: - static Status GetClientConfiguration(CloudEnv *env, - const std::string& region, - Aws::Client::ClientConfiguration* config); + public: + static Status GetClientConfiguration( + CloudEnv* env, const std::string& region, + Aws::Client::ClientConfiguration* config); }; // // The cloud environment for rocksdb. It allows configuring the rocksdb // Environent used for the cloud. // -class CloudEnvOptions { -private: - public: +struct CloudEnvOptions { + std::shared_ptr storage_provider; + std::shared_ptr log_controller; + BucketOptions src_bucket; BucketOptions dest_bucket; // Specify the type of cloud-service to use. @@ -317,25 +319,30 @@ typedef std::map DbidList; // The Cloud environment // class CloudEnv : public Env { +public: + static const std::string kCloudEnvOpts /*="CloudEnvOptions"*/; + static const std::string kCloudEnvName /*="CloudEnv"*/; protected: CloudEnvOptions cloud_env_options; Env* base_env_; // The underlying env - std::unique_ptr cloud_log_controller_; - CloudEnv(const CloudEnvOptions& options, Env *base, const std::shared_ptr& logger); -public: + CloudEnv(const CloudEnvOptions& options, Env* base, const std::shared_ptr& logger); + + public: std::shared_ptr info_log_; // informational messages + virtual ~CloudEnv(); // Returns the underlying env Env* GetBaseEnv() { return base_env_; } + virtual const char *Name() const { return kCloudEnvName.c_str(); } + const std::shared_ptr & GetLogger() const { + return info_log_; + } + virtual Status PreloadCloudManifest(const std::string& local_dbname) = 0; - // Empties all contents of the associated cloud storage bucket. - virtual Status EmptyBucket(const std::string& bucket_prefix, - const std::string& path_prefix) = 0; - // Reads a file from the cloud virtual Status NewSequentialFileCloud(const std::string& bucket_prefix, const std::string& fname, @@ -391,44 +398,11 @@ class CloudEnv : public Env { return cloud_env_options; } - // returns all the objects that have the specified path prefix and - // are stored in a cloud bucket - virtual Status ListObjects(const std::string& bucket_name_prefix, - const std::string& bucket_object_prefix, - BucketObjectMetadata* meta) = 0; - - // Delete the specified object from the specified cloud bucket - virtual Status DeleteObject(const std::string& bucket_name_prefix, - const std::string& bucket_object_path) = 0; - - // Does the specified object exist in the cloud storage - virtual Status ExistsObject(const std::string& bucket_name_prefix, - const std::string& bucket_object_path) = 0; - - // Get the size of the object in cloud storage - virtual Status GetObjectSize(const std::string& bucket_name_prefix, - const std::string& bucket_object_path, - uint64_t* filesize) = 0; - - // Copy the specified cloud object from one location in the cloud - // storage to another location in cloud storage - virtual Status CopyObject(const std::string& bucket_name_prefix_src, - const std::string& bucket_object_path_src, - const std::string& bucket_name_prefix_dest, - const std::string& bucket_object_path_dest) = 0; - - // Downloads object from the cloud into a local directory - virtual Status GetObject(const std::string& bucket_name_prefix, - const std::string& bucket_object_path, - const std::string& local_path) = 0; - - // Uploads object to the cloud - virtual Status PutObject(const std::string& local_path, - const std::string& bucket_name_prefix, - const std::string& bucket_object_path) = 0; - // Deletes file from a destination bucket. virtual Status DeleteCloudFileFromDest(const std::string& fname) = 0; + // Deletes file from a destination bucket. + virtual Status CopyLocalFileToDest(const std::string& local_name, + const std::string& cloud_name) = 0; // Create a new AWS env. // src_bucket_name: bucket name suffix where db data is read from @@ -453,15 +427,9 @@ class CloudEnv : public Env { static Status NewAwsEnv(Env* base_env, const CloudEnvOptions& env_options, const std::shared_ptr& logger, CloudEnv** cenv); -}; - -/* - * The information about all objects stored in a cloud bucket - */ -class BucketObjectMetadata { - public: - // list of all pathnames - std::vector pathnames; + virtual Status Prepare(); + virtual Status Verify() const; +protected: }; } // namespace rocksdb diff --git a/cloud/cloud_log_controller.h b/include/rocksdb/cloud/cloud_log_controller.h similarity index 81% rename from cloud/cloud_log_controller.h rename to include/rocksdb/cloud/cloud_log_controller.h index 5c3aa94dec1..2cad0563847 100644 --- a/cloud/cloud_log_controller.h +++ b/include/rocksdb/cloud/cloud_log_controller.h @@ -5,18 +5,20 @@ #include #include "rocksdb/env.h" +#include "rocksdb/options.h" #include "rocksdb/status.h" namespace rocksdb { class CloudEnv; -class CloudEnvOptions; +struct CloudEnvOptions; // Creates a new file, appends data to a file or delete an existing file via // logging into a cloud stream (such as Kinesis). // class CloudLogWritableFile : public WritableFile { public: - CloudLogWritableFile(CloudEnv* env, const std::string& fname, const EnvOptions& options); + CloudLogWritableFile(CloudEnv* env, const std::string& fname, + const EnvOptions& options); virtual ~CloudLogWritableFile(); virtual Status Flush() { @@ -52,14 +54,14 @@ class CloudLogController { // Delay in Cloud Log stream: writes to read visibility static const std::chrono::microseconds kRetryPeriod; - static const uint32_t kAppend = 0x1; // add a new record to a logfile static const uint32_t kDelete = 0x2; // delete a log file static const uint32_t kClosed = 0x4; // closing a file - CloudLogController(CloudEnv* env); virtual ~CloudLogController(); - + static const char *Type() { return "CloudLogController"; } + static Status CreateKinesisController(std::unique_ptr* result); + static Status CreateKafkaController(std::unique_ptr* result); // Create a stream to store all log files. virtual Status CreateStream(const std::string& topic) = 0; @@ -75,10 +77,10 @@ class CloudLogController { const EnvOptions& options) = 0; // Returns name of the cloud log type (Kinesis, etc.). - virtual const char *Name() const { return "cloudlog"; } + virtual const char* Name() const { return "cloudlog"; } // Directory where files are cached locally. - const std::string & GetCacheDir() const { return cache_dir_; } + const std::string& GetCacheDir() const { return cache_dir_; } Status const status() { return status_; } @@ -95,13 +97,16 @@ class CloudLogController { // Retries fnc until success or timeout has expired. typedef std::function RetryType; Status Retry(RetryType func); - Status StartTailingStream(const std::string & topic); - void StopTailingStream(); + virtual Status StartTailingStream(const std::string& topic); + virtual void StopTailingStream(); + virtual Status Prepare(CloudEnv * env); + virtual Status Verify() const; protected: CloudEnv* env_; Status status_; std::string cache_dir_; - + CloudLogController() : running_(false) { } + virtual Status Initialize(CloudEnv *env); // A cache of pathnames to their open file _escriptors std::map> cache_fds_; @@ -110,12 +115,11 @@ class CloudLogController { Slice* filename, uint64_t* offset_in_file, uint64_t* file_size, Slice* data); bool IsRunning() const { return running_; } -private: + + private: // Background thread to tail stream std::unique_ptr tid_; std::atomic running_; }; -Status CreateKinesisController(CloudEnv* env, std::unique_ptr * result); -Status CreateKafkaController(CloudEnv* env, std::unique_ptr * result); } // namespace rocksdb diff --git a/include/rocksdb/cloud/cloud_storage_provider.h b/include/rocksdb/cloud/cloud_storage_provider.h new file mode 100644 index 00000000000..1615387bddb --- /dev/null +++ b/include/rocksdb/cloud/cloud_storage_provider.h @@ -0,0 +1,188 @@ +// Copyright (c) 2016-present, Rockset, Inc. All rights reserved. +// +#pragma once + +#include + +#include "rocksdb/env.h" +#include "rocksdb/status.h" + +namespace rocksdb { +class CloudEnv; +class CloudStorageProvider; +class Logger; +struct ColumnFamilyOptions; +struct DBOptions; + +class CloudStorageReadableFile : virtual public SequentialFile, + virtual public RandomAccessFile { + public: + CloudStorageReadableFile(const std::shared_ptr& info_log, + const std::string& bucket, const std::string& fname, + uint64_t size); + + // sequential access, read data at current offset in file + virtual Status Read(size_t n, Slice* result, char* scratch) override; + + // random access, read data from specified offset in file + virtual Status Read(uint64_t offset, size_t n, Slice* result, + char* scratch) const override; + + virtual Status Skip(uint64_t n) override; + + virtual size_t GetUniqueId(char* id, size_t max_size) const override; + virtual const char* Name() const { return "cloud"; } + + protected: + virtual Status DoCloudRead(uint64_t offset, size_t n, char* scratch, + uint64_t* bytes_read) const = 0; + + std::shared_ptr info_log_; + std::string bucket_; + std::string fname_; + uint64_t offset_; + uint64_t file_size_; +}; + +// Appends to a file in S3. +class CloudStorageWritableFile : public WritableFile { + protected: + CloudEnv* env_; + const char* class_; + std::string fname_; + std::string tmp_file_; + Status status_; + std::unique_ptr local_file_; + std::string bucket_; + std::string cloud_fname_; + bool is_manifest_; + + public: + CloudStorageWritableFile(CloudEnv* env, const std::string& local_fname, + const std::string& bucket, + const std::string& cloud_fname, + const EnvOptions& options); + + virtual ~CloudStorageWritableFile(); + + virtual Status Append(const Slice& data) override { + assert(status_.ok()); + // write to temporary file + return local_file_->Append(data); + } + + Status PositionedAppend(const Slice& data, uint64_t offset) override { + return local_file_->PositionedAppend(data, offset); + } + Status Truncate(uint64_t size) override { + return local_file_->Truncate(size); + } + Status Fsync() override { return local_file_->Fsync(); } + bool IsSyncThreadSafe() const override { + return local_file_->IsSyncThreadSafe(); + } + bool use_direct_io() const override { return local_file_->use_direct_io(); } + size_t GetRequiredBufferAlignment() const override { + return local_file_->GetRequiredBufferAlignment(); + } + uint64_t GetFileSize() override { return local_file_->GetFileSize(); } + size_t GetUniqueId(char* id, size_t max_size) const override { + return local_file_->GetUniqueId(id, max_size); + } + Status InvalidateCache(size_t offset, size_t length) override { + return local_file_->InvalidateCache(offset, length); + } + Status RangeSync(uint64_t offset, uint64_t nbytes) override { + return local_file_->RangeSync(offset, nbytes); + } + Status Allocate(uint64_t offset, uint64_t len) override { + return local_file_->Allocate(offset, len); + } + + virtual Status Flush() override { + assert(status_.ok()); + return local_file_->Flush(); + } + + virtual Status Sync() override; + + virtual Status status() { return status_; } + + virtual Status Close() override; + virtual const char* Name() const { return "cloud"; } +}; + +class CloudStorageProvider { + public: + static Status CreateS3Provider(std::unique_ptr* result); + virtual ~CloudStorageProvider(); + static const char *Type() { return "CloudStorageProvider"; } + virtual const char* Name() const { return "cloud"; } + virtual Status CreateBucket(const std::string& bucket_name) = 0; + virtual Status ExistsBucket(const std::string& bucket_name) = 0; + + // Empties all contents of the associated cloud storage bucket. + virtual Status EmptyBucket(const std::string& bucket_name, + const std::string& object_path) = 0; + // Delete the specified object from the specified cloud bucket + virtual Status DeleteObject(const std::string& bucket_name, + const std::string& object_path) = 0; + + // Does the specified object exist in the cloud storage + // returns all the objects that have the specified path prefix and + // are stored in a cloud bucket + virtual Status ListObjects(const std::string& bucket_name, + const std::string& object_path, + std::vector* path_names) = 0; + + // Does the specified object exist in the cloud storage + virtual Status ExistsObject(const std::string& bucket_name, + const std::string& object_path) = 0; + + // Get the size of the object in cloud storage + virtual Status GetObjectSize(const std::string& bucket_name, + const std::string& object_path, + uint64_t* filesize) = 0; + + // Get the modification time of the object in cloud storage + virtual Status GetObjectModificationTime(const std::string& bucket_name, + const std::string& object_path, + uint64_t* time) = 0; + + // Get the metadata of the object in cloud storage + virtual Status GetObjectMetadata( + const std::string& bucket_name, const std::string& object_path, + std::unordered_map* metadata) = 0; + + // Copy the specified cloud object from one location in the cloud + // storage to another location in cloud storage + virtual Status CopyObject(const std::string& src_bucket_name, + const std::string& src_object_path, + const std::string& dest_bucket_name, + const std::string& dest_object_path) = 0; + + // Downloads object from the cloud into a local directory + virtual Status GetObject(const std::string& bucket_name, + const std::string& object_path, + const std::string& local_path) = 0; + // Uploads object to the cloud + virtual Status PutObject(const std::string& local_path, + const std::string& bucket_name, + const std::string& object_path) = 0; + // Updates/Sets the metadata of the object in cloud storage + virtual Status PutObjectMetadata( + const std::string& bucket_name, const std::string& object_path, + const std::unordered_map& metadata) = 0; + virtual Status NewCloudWritableFile( + const std::string& local_path, const std::string& bucket_name, + const std::string& object_path, + std::unique_ptr* result, + const EnvOptions& options) = 0; + + virtual Status NewCloudReadableFile( + const std::string& bucket, const std::string& fname, + std::unique_ptr* result) = 0; + virtual Status Prepare(CloudEnv *env); + virtual Status Verify() const = 0; +}; +} // namespace rocksdb diff --git a/include/rocksdb/cloud/db_cloud.h b/include/rocksdb/cloud/db_cloud.h index 3ca404d6d6c..af2fe4acf4f 100644 --- a/include/rocksdb/cloud/db_cloud.h +++ b/include/rocksdb/cloud/db_cloud.h @@ -6,11 +6,12 @@ #include #include -#include "rocksdb/cloud/cloud_env_options.h" #include "rocksdb/db.h" #include "rocksdb/utilities/stackable_db.h" namespace rocksdb { +class BucketOptions; +struct CheckpointToCloudOptions; // // Database with Cloud support. diff --git a/src.mk b/src.mk index 5dd94d617e1..461e6b21c98 100644 --- a/src.mk +++ b/src.mk @@ -235,6 +235,7 @@ LIB_SOURCES = \ cloud/cloud_env_impl.cc \ cloud/cloud_env_options.cc \ cloud/cloud_log_controller.cc \ + cloud/cloud_storage_provider.cc \ cloud/manifest_reader.cc \ cloud/purge.cc \ cloud/cloud_manifest.cc \ diff --git a/tools/db_bench_tool.cc b/tools/db_bench_tool.cc index 779f1ee2181..667918f6159 100644 --- a/tools/db_bench_tool.cc +++ b/tools/db_bench_tool.cc @@ -1222,9 +1222,10 @@ rocksdb::Env* CreateAwsEnv(const std::string& dbpath , return s; } -static rocksdb::Registrar - s3_reg("s3://.*", [](const std::string& uri, - std::unique_ptr* env_guard) { +static auto & registrar = rocksdb::ObjectLibrary::Default()->Register( + "s3://.*", [](const std::string& uri, + std::unique_ptr* env_guard, + std::string* /*errmsg*/) { CreateAwsEnv(uri, env_guard); return env_guard->get(); });