2022-11-04 16:03:12 +00:00
|
|
|
#include "config.h"
|
2022-09-07 07:16:32 +00:00
|
|
|
|
|
|
|
#if USE_AWS_S3
|
|
|
|
|
2022-09-28 11:21:32 +00:00
|
|
|
# include <Storages/StorageHudi.h>
|
|
|
|
# include <Common/logger_useful.h>
|
2022-08-22 09:37:20 +00:00
|
|
|
|
2022-11-03 18:17:39 +00:00
|
|
|
# include <Formats/FormatFactory.h>
|
2022-09-28 11:21:32 +00:00
|
|
|
# include <IO/S3Common.h>
|
|
|
|
# include <Storages/StorageFactory.h>
|
|
|
|
# include <Storages/checkAndGetLiteralArgument.h>
|
|
|
|
# include <aws/core/auth/AWSCredentials.h>
|
|
|
|
# include <aws/s3/S3Client.h>
|
|
|
|
# include <aws/s3/model/ListObjectsV2Request.h>
|
2022-08-22 09:37:20 +00:00
|
|
|
|
2022-09-28 11:21:32 +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-29 20:15:27 +00:00
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
2022-08-22 09:37:20 +00:00
|
|
|
extern const int S3_ERROR;
|
2022-08-08 14:10:50 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
StorageHudi::StorageHudi(
|
2022-11-03 18:14:10 +00:00
|
|
|
const StorageS3Configuration & configuration_,
|
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-11-03 18:14:10 +00:00
|
|
|
ContextPtr context_,
|
|
|
|
std::optional<FormatSettings> format_settings_)
|
2022-08-26 18:17:32 +00:00
|
|
|
: IStorage(table_id_)
|
2022-11-03 18:14:10 +00:00
|
|
|
, base_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers}
|
2022-08-22 09:37:20 +00:00
|
|
|
, log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")"))
|
2022-11-03 18:14:10 +00:00
|
|
|
, table_path(base_configuration.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-11-03 18:14:10 +00:00
|
|
|
auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys), configuration_.format);
|
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-11-03 18:17:39 +00:00
|
|
|
|
2022-11-03 18:14:10 +00:00
|
|
|
StorageS3Configuration new_configuration;
|
|
|
|
new_configuration.url = new_uri;
|
|
|
|
new_configuration.auth_settings.access_key_id = configuration_.auth_settings.access_key_id;
|
2022-11-03 18:17:39 +00:00
|
|
|
new_configuration.auth_settings.secret_access_key = configuration_.auth_settings.secret_access_key;
|
2022-11-03 18:14:10 +00:00
|
|
|
new_configuration.format = configuration_.format;
|
2022-11-03 18:17:39 +00:00
|
|
|
|
2022-08-24 15:07:37 +00:00
|
|
|
if (columns_.empty())
|
2022-08-22 09:37:20 +00:00
|
|
|
{
|
2022-11-03 18:17:39 +00:00
|
|
|
columns_ = StorageS3::getTableStructureFromData(
|
|
|
|
new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr);
|
2022-08-26 18:17:32 +00:00
|
|
|
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>(
|
2022-11-03 18:17:39 +00:00
|
|
|
new_configuration,
|
|
|
|
table_id_,
|
|
|
|
columns_,
|
|
|
|
constraints_,
|
|
|
|
comment,
|
|
|
|
context_,
|
|
|
|
format_settings_,
|
|
|
|
/* distributed_processing_ */ false,
|
|
|
|
nullptr);
|
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,
|
2022-11-04 16:03:12 +00:00
|
|
|
size_t num_streams)
|
2022-09-28 11:21:32 +00:00
|
|
|
{
|
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-09-29 20:15:27 +00:00
|
|
|
std::string StorageHudi::generateQueryFromKeys(std::vector<std::string> && keys, String format)
|
2022-08-26 18:17:32 +00:00
|
|
|
{
|
2022-09-29 20:15:27 +00:00
|
|
|
// make format lowercase
|
|
|
|
std::transform(format.begin(), format.end(), format.begin(), [](unsigned char c) { return std::tolower(c); });
|
2022-08-24 15:07:37 +00:00
|
|
|
|
2022-09-29 20:15:27 +00:00
|
|
|
// filter only files with specific format
|
|
|
|
std::erase_if(keys, [&format](const std::string & s) { return std::filesystem::path(s).extension() != "." + format; });
|
|
|
|
|
|
|
|
// for each partition path take only latest file
|
2022-08-24 15:07:37 +00:00
|
|
|
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2022-09-28 17:30:15 +00:00
|
|
|
// every filename contains metadata split 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;
|
2022-09-29 20:15:27 +00:00
|
|
|
if (engine_args.empty() || engine_args.size() < 3)
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
|
|
|
"Storage Hudi requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]");
|
2022-08-22 09:37:20 +00:00
|
|
|
|
|
|
|
|
2022-11-03 18:14:10 +00:00
|
|
|
StorageS3Configuration configuration;
|
2022-11-03 18:17:39 +00:00
|
|
|
|
2022-11-03 18:14:10 +00:00
|
|
|
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");
|
2022-11-03 18:17:39 +00:00
|
|
|
|
|
|
|
if (engine_args.size() == 4)
|
2022-11-03 18:14:10 +00:00
|
|
|
configuration.format = checkAndGetLiteralArgument<String>(engine_args[3], "format");
|
2022-11-03 18:17:39 +00:00
|
|
|
|
2022-11-03 18:14:10 +00:00
|
|
|
if (configuration.format == "auto")
|
|
|
|
configuration.format = "Parquet";
|
2022-11-03 18:17:39 +00:00
|
|
|
|
2022-11-03 18:14:10 +00:00
|
|
|
auto format_settings = getFormatSettings(args.getContext());
|
2022-08-08 14:10:50 +00:00
|
|
|
|
2022-08-22 09:37:20 +00:00
|
|
|
return std::make_shared<StorageHudi>(
|
2022-11-03 18:17:39 +00:00
|
|
|
configuration, args.table_id, args.columns, args.constraints, args.comment, args.getContext(), format_settings);
|
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
|