ClickHouse/src/Storages/StorageHudi.cpp

238 lines
6.9 KiB
C++
Raw Normal View History

2022-08-25 08:48:49 +00:00
#include <Common/config.h>
2022-09-07 07:16:32 +00:00
#if USE_AWS_S3
#include <Storages/StorageHudi.h>
2022-08-22 09:37:20 +00:00
#include <Common/logger_useful.h>
2022-08-25 08:48:49 +00:00
#include <IO/S3Common.h>
2022-08-08 14:10:50 +00:00
#include <Storages/StorageFactory.h>
2022-08-22 09:37:20 +00:00
#include <Storages/checkAndGetLiteralArgument.h>
#include <aws/core/auth/AWSCredentials.h>
#include <aws/s3/S3Client.h>
#include <aws/s3/model/ListObjectsV2Request.h>
2022-08-25 08:48:49 +00:00
#include <QueryPipeline/Pipe.h>
2022-08-08 14:10:50 +00:00
2022-08-26 18:17:32 +00:00
namespace DB
{
2022-08-08 14:10:50 +00:00
2022-08-26 18:17:32 +00:00
namespace ErrorCodes
{
2022-09-06 18:12:51 +00:00
extern const int BAD_ARGUMENTS;
2022-08-22 09:37:20 +00:00
extern const int S3_ERROR;
2022-08-08 14:10:50 +00:00
}
StorageHudi::StorageHudi(
2022-08-22 09:37:20 +00:00
const S3::URI & uri_,
const String & access_key_,
2022-08-24 15:07:37 +00:00
const String & secret_access_key_,
2022-08-08 14:10:50 +00:00
const StorageID & table_id_,
2022-08-26 18:17:32 +00:00
ColumnsDescription columns_,
2022-08-24 15:07:37 +00:00
const ConstraintsDescription & constraints_,
const String & comment,
2022-08-26 18:17:32 +00:00
ContextPtr context_)
: IStorage(table_id_)
2022-08-25 08:48:49 +00:00
, base_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}})
2022-08-22 09:37:20 +00:00
, log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")"))
2022-08-30 17:38:02 +00:00
, table_path(uri_.key)
2022-08-22 09:37:20 +00:00
{
2022-08-24 15:07:37 +00:00
StorageInMemoryMetadata storage_metadata;
2022-09-28 11:03:43 +00:00
StorageS3::updateS3Configuration(context_, base_configuration);
2022-08-25 08:48:49 +00:00
auto keys = getKeysFromS3();
2022-08-22 09:37:20 +00:00
2022-08-25 08:48:49 +00:00
auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys));
2022-08-26 18:17:32 +00:00
2022-08-25 08:48:49 +00:00
LOG_DEBUG(log, "New uri: {}", new_uri);
2022-08-30 17:38:02 +00:00
LOG_DEBUG(log, "Table path: {}", table_path);
2022-08-25 08:48:49 +00:00
auto s3_uri = S3::URI(Poco::URI(new_uri));
2022-08-24 15:07:37 +00:00
if (columns_.empty())
2022-08-22 09:37:20 +00:00
{
2022-08-26 18:17:32 +00:00
columns_
= StorageS3::getTableStructureFromData(String("Parquet"), s3_uri, access_key_, secret_access_key_, "", false, {}, context_);
storage_metadata.setColumns(columns_);
2022-08-24 15:07:37 +00:00
}
2022-08-25 08:48:49 +00:00
else
2022-08-24 15:07:37 +00:00
storage_metadata.setColumns(columns_);
2022-08-22 09:37:20 +00:00
2022-08-24 15:07:37 +00:00
storage_metadata.setConstraints(constraints_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata);
2022-08-25 08:48:49 +00:00
s3engine = std::make_shared<StorageS3>(
s3_uri,
access_key_,
secret_access_key_,
table_id_,
String("Parquet"), // format name
base_configuration.rw_settings,
columns_,
constraints_,
comment,
context_,
2022-08-26 18:17:32 +00:00
std::nullopt);
2022-08-24 15:07:37 +00:00
}
2022-08-22 09:37:20 +00:00
2022-08-24 15:07:37 +00:00
Pipe StorageHudi::read(
2022-08-26 18:17:32 +00:00
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
unsigned num_streams)
2022-09-28 11:03:43 +00:00
{
StorageS3::updateS3Configuration(context, base_configuration);
2022-08-26 18:17:32 +00:00
return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams);
2022-08-25 08:48:49 +00:00
}
2022-08-22 09:37:20 +00:00
2022-08-26 18:17:32 +00:00
std::vector<std::string> StorageHudi::getKeysFromS3()
{
2022-08-24 15:07:37 +00:00
std::vector<std::string> keys;
2022-08-25 08:48:49 +00:00
const auto & client = base_configuration.client;
2022-08-26 18:17:32 +00:00
2022-08-24 15:07:37 +00:00
Aws::S3::Model::ListObjectsV2Request request;
Aws::S3::Model::ListObjectsV2Outcome outcome;
bool is_finished{false};
2022-08-25 08:48:49 +00:00
const auto bucket{base_configuration.uri.bucket};
2022-08-24 15:07:37 +00:00
request.SetBucket(bucket);
2022-08-30 17:38:02 +00:00
request.SetPrefix(table_path);
2022-08-24 15:07:37 +00:00
while (!is_finished)
{
outcome = client->ListObjectsV2(request);
if (!outcome.IsSuccess())
throw Exception(
ErrorCodes::S3_ERROR,
"Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}",
quoteString(bucket),
2022-08-30 17:38:02 +00:00
quoteString(table_path),
2022-08-24 15:07:37 +00:00
backQuote(outcome.GetError().GetExceptionName()),
quoteString(outcome.GetError().GetMessage()));
const auto & result_batch = outcome.GetResult().GetContents();
for (const auto & obj : result_batch)
{
2022-08-30 17:38:02 +00:00
const auto & filename = obj.GetKey().substr(table_path.size()); // object name without tablepath prefix
2022-08-24 15:07:37 +00:00
keys.push_back(filename);
2022-08-30 17:38:02 +00:00
LOG_DEBUG(log, "Found file: {}", filename);
2022-08-24 15:07:37 +00:00
}
request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken());
is_finished = !outcome.GetResult().GetIsTruncated();
}
return keys;
}
2022-08-26 18:17:32 +00:00
std::string StorageHudi::generateQueryFromKeys(std::vector<std::string> && keys)
{
2022-08-24 15:07:37 +00:00
// filter only .parquet files
2022-08-26 18:17:32 +00:00
std::erase_if(
keys,
[](const std::string & s)
{
2022-09-28 11:03:43 +00:00
return std::filesystem::path(s).extension() != ".parquet";
2022-08-26 18:17:32 +00:00
});
2022-08-24 15:07:37 +00:00
// for each partition path take only latest parquet file
std::unordered_map<std::string, std::pair<std::string, uint64_t>> latest_parquets;
2022-08-26 18:17:32 +00:00
for (const auto & key : keys)
{
2022-08-24 15:07:37 +00:00
auto slash = key.find_last_of("/");
std::string path;
2022-08-26 18:17:32 +00:00
if (slash == std::string::npos)
{
2022-08-24 15:07:37 +00:00
path = "";
2022-08-26 18:17:32 +00:00
}
else
{
2022-08-24 15:07:37 +00:00
path = key.substr(0, slash);
}
// every filename contains metadata splitted by "_", timestamp is after last "_"
2022-08-26 18:17:32 +00:00
uint64_t timestamp = std::stoul(key.substr(key.find_last_of("_") + 1));
2022-08-24 15:07:37 +00:00
auto it = latest_parquets.find(path);
2022-08-26 18:17:32 +00:00
if (it != latest_parquets.end())
{
if (it->second.second < timestamp)
{
2022-08-24 15:07:37 +00:00
it->second = {key, timestamp};
}
2022-08-26 18:17:32 +00:00
}
else
{
2022-08-24 15:07:37 +00:00
latest_parquets[path] = {key, timestamp};
}
}
std::vector<std::string> filtered_keys;
2022-08-26 18:17:32 +00:00
std::transform(
latest_parquets.begin(), latest_parquets.end(), std::back_inserter(filtered_keys), [](const auto & kv) { return kv.second.first; });
2022-08-24 15:07:37 +00:00
std::string new_query;
2022-08-26 18:17:32 +00:00
for (auto && key : filtered_keys)
{
if (!new_query.empty())
{
2022-08-24 15:07:37 +00:00
new_query += ",";
}
new_query += key;
}
new_query = "{" + new_query + "}";
return new_query;
}
2022-08-08 14:10:50 +00:00
void registerStorageHudi(StorageFactory & factory)
{
2022-08-26 18:17:32 +00:00
factory.registerStorage(
"Hudi",
[](const StorageFactory::Arguments & args)
2022-08-08 14:10:50 +00:00
{
2022-08-22 09:37:20 +00:00
auto & engine_args = args.engine_args;
if (engine_args.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments");
auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext());
configuration.url = checkAndGetLiteralArgument<String>(engine_args[0], "url");
configuration.auth_settings.access_key_id = checkAndGetLiteralArgument<String>(engine_args[1], "access_key_id");
configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument<String>(engine_args[2], "secret_access_key");
S3::URI s3_uri(Poco::URI(configuration.url));
2022-08-08 14:10:50 +00:00
2022-08-22 09:37:20 +00:00
return std::make_shared<StorageHudi>(
s3_uri,
configuration.auth_settings.access_key_id,
configuration.auth_settings.secret_access_key,
args.table_id,
args.columns,
args.constraints,
args.comment,
args.getContext());
2022-08-25 08:48:49 +00:00
},
{
2022-08-26 18:17:32 +00:00
.supports_settings = true,
.supports_schema_inference = true,
.source_access_type = AccessType::S3,
});
2022-08-08 14:10:50 +00:00
}
}
2022-09-07 07:16:32 +00:00
#endif