mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge branch 'master' into ch_canh_gs_core
This commit is contained in:
commit
4fe0457e59
@ -3,6 +3,9 @@
|
||||
# shellcheck disable=SC2086
|
||||
# shellcheck disable=SC2024
|
||||
|
||||
# Avoid overlaps with previous runs
|
||||
dmesg --clear
|
||||
|
||||
set -x
|
||||
|
||||
# Thread Fuzzer allows to check more permutations of possible thread scheduling
|
||||
|
@ -286,6 +286,18 @@ The server successfully detected this situation and will download merged part fr
|
||||
M(S3WriteRequestsThrottling, "Number of 429 and 503 errors in POST, DELETE, PUT and PATCH requests to S3 storage.") \
|
||||
M(S3WriteRequestsRedirects, "Number of redirects in POST, DELETE, PUT and PATCH requests to S3 storage.") \
|
||||
\
|
||||
M(DiskS3ReadMicroseconds, "Time of GET and HEAD requests to DiskS3 storage.") \
|
||||
M(DiskS3ReadRequestsCount, "Number of GET and HEAD requests to DiskS3 storage.") \
|
||||
M(DiskS3ReadRequestsErrors, "Number of non-throttling errors in GET and HEAD requests to DiskS3 storage.") \
|
||||
M(DiskS3ReadRequestsThrottling, "Number of 429 and 503 errors in GET and HEAD requests to DiskS3 storage.") \
|
||||
M(DiskS3ReadRequestsRedirects, "Number of redirects in GET and HEAD requests to DiskS3 storage.") \
|
||||
\
|
||||
M(DiskS3WriteMicroseconds, "Time of POST, DELETE, PUT and PATCH requests to DiskS3 storage.") \
|
||||
M(DiskS3WriteRequestsCount, "Number of POST, DELETE, PUT and PATCH requests to DiskS3 storage.") \
|
||||
M(DiskS3WriteRequestsErrors, "Number of non-throttling errors in POST, DELETE, PUT and PATCH requests to DiskS3 storage.") \
|
||||
M(DiskS3WriteRequestsThrottling, "Number of 429 and 503 errors in POST, DELETE, PUT and PATCH requests to DiskS3 storage.") \
|
||||
M(DiskS3WriteRequestsRedirects, "Number of redirects in POST, DELETE, PUT and PATCH requests to DiskS3 storage.") \
|
||||
\
|
||||
M(ReadBufferFromS3Microseconds, "Time spend in reading from S3.") \
|
||||
M(ReadBufferFromS3Bytes, "Bytes read from S3.") \
|
||||
M(ReadBufferFromS3RequestsErrors, "Number of exceptions while reading from S3.") \
|
||||
|
@ -478,11 +478,7 @@ template <> void inline copyOverlap<32, true>(UInt8 * op, const UInt8 *& match,
|
||||
/// See also https://stackoverflow.com/a/30669632
|
||||
|
||||
template <size_t copy_amount, bool use_shuffle>
|
||||
bool NO_INLINE decompressImpl(
|
||||
const char * const source,
|
||||
char * const dest,
|
||||
size_t source_size,
|
||||
size_t dest_size)
|
||||
bool NO_INLINE decompressImpl(const char * const source, char * const dest, size_t source_size, size_t dest_size)
|
||||
{
|
||||
const UInt8 * ip = reinterpret_cast<const UInt8 *>(source);
|
||||
UInt8 * op = reinterpret_cast<UInt8 *>(dest);
|
||||
@ -515,6 +511,18 @@ bool NO_INLINE decompressImpl(
|
||||
|
||||
const unsigned token = *ip++;
|
||||
length = token >> 4;
|
||||
|
||||
UInt8 * copy_end;
|
||||
size_t real_length;
|
||||
|
||||
/// It might be true fairly often for well-compressed columns.
|
||||
/// ATST it may hurt performance in other cases because this condition is hard to predict (especially if the number of zeros is ~50%).
|
||||
/// In such cases this `if` will significantly increase number of mispredicted instructions. But seems like it results in a
|
||||
/// noticeable slowdown only for implementations with `copy_amount` > 8. Probably because they use havier instructions.
|
||||
if constexpr (copy_amount == 8)
|
||||
if (length == 0)
|
||||
goto decompress_match;
|
||||
|
||||
if (length == 0x0F)
|
||||
{
|
||||
if (unlikely(ip + 1 >= input_end))
|
||||
@ -524,7 +532,7 @@ bool NO_INLINE decompressImpl(
|
||||
|
||||
/// Copy literals.
|
||||
|
||||
UInt8 * copy_end = op + length;
|
||||
copy_end = op + length;
|
||||
|
||||
/// input: Hello, world
|
||||
/// ^-ip
|
||||
@ -541,7 +549,7 @@ bool NO_INLINE decompressImpl(
|
||||
return false;
|
||||
|
||||
// Due to implementation specifics the copy length is always a multiple of copy_amount
|
||||
size_t real_length = 0;
|
||||
real_length = 0;
|
||||
|
||||
static_assert(copy_amount == 8 || copy_amount == 16 || copy_amount == 32);
|
||||
if constexpr (copy_amount == 8)
|
||||
@ -552,9 +560,9 @@ bool NO_INLINE decompressImpl(
|
||||
real_length = (((length >> 5) + 1) * 32);
|
||||
|
||||
if (unlikely(ip + real_length >= input_end + ADDITIONAL_BYTES_AT_END_OF_BUFFER))
|
||||
return false;
|
||||
return false;
|
||||
|
||||
wildCopy<copy_amount>(op, ip, copy_end); /// Here we can write up to copy_amount - 1 bytes after buffer.
|
||||
wildCopy<copy_amount>(op, ip, copy_end); /// Here we can write up to copy_amount - 1 bytes after buffer.
|
||||
|
||||
if (copy_end == output_end)
|
||||
return true;
|
||||
@ -562,6 +570,8 @@ bool NO_INLINE decompressImpl(
|
||||
ip += length;
|
||||
op = copy_end;
|
||||
|
||||
decompress_match:
|
||||
|
||||
if (unlikely(ip + 1 >= input_end))
|
||||
return false;
|
||||
|
||||
|
@ -116,7 +116,8 @@ std::unique_ptr<Aws::S3::S3Client> getClient(const Poco::Util::AbstractConfigura
|
||||
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
|
||||
config.getString(config_prefix + ".region", ""),
|
||||
context->getRemoteHostFilter(), context->getGlobalContext()->getSettingsRef().s3_max_redirects,
|
||||
context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging);
|
||||
context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging,
|
||||
/* for_disk_s3 = */ true);
|
||||
|
||||
S3::URI uri(Poco::URI(config.getString(config_prefix + ".endpoint")));
|
||||
if (uri.key.back() != '/')
|
||||
|
@ -42,6 +42,18 @@ namespace ProfileEvents
|
||||
extern const Event S3WriteRequestsErrors;
|
||||
extern const Event S3WriteRequestsThrottling;
|
||||
extern const Event S3WriteRequestsRedirects;
|
||||
|
||||
extern const Event DiskS3ReadMicroseconds;
|
||||
extern const Event DiskS3ReadRequestsCount;
|
||||
extern const Event DiskS3ReadRequestsErrors;
|
||||
extern const Event DiskS3ReadRequestsThrottling;
|
||||
extern const Event DiskS3ReadRequestsRedirects;
|
||||
|
||||
extern const Event DiskS3WriteMicroseconds;
|
||||
extern const Event DiskS3WriteRequestsCount;
|
||||
extern const Event DiskS3WriteRequestsErrors;
|
||||
extern const Event DiskS3WriteRequestsThrottling;
|
||||
extern const Event DiskS3WriteRequestsRedirects;
|
||||
}
|
||||
|
||||
namespace CurrentMetrics
|
||||
@ -62,11 +74,13 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration(
|
||||
const String & force_region_,
|
||||
const RemoteHostFilter & remote_host_filter_,
|
||||
unsigned int s3_max_redirects_,
|
||||
bool enable_s3_requests_logging_)
|
||||
bool enable_s3_requests_logging_,
|
||||
bool for_disk_s3_)
|
||||
: force_region(force_region_)
|
||||
, remote_host_filter(remote_host_filter_)
|
||||
, s3_max_redirects(s3_max_redirects_)
|
||||
, enable_s3_requests_logging(enable_s3_requests_logging_)
|
||||
, for_disk_s3(for_disk_s3_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -112,6 +126,7 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config
|
||||
, remote_host_filter(client_configuration.remote_host_filter)
|
||||
, s3_max_redirects(client_configuration.s3_max_redirects)
|
||||
, enable_s3_requests_logging(client_configuration.enable_s3_requests_logging)
|
||||
, for_disk_s3(client_configuration.for_disk_s3)
|
||||
, extra_headers(client_configuration.extra_headers)
|
||||
{
|
||||
}
|
||||
@ -176,6 +191,46 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
PocoHTTPClient::S3MetricKind PocoHTTPClient::getMetricKind(const Aws::Http::HttpRequest & request)
|
||||
{
|
||||
switch (request.GetMethod())
|
||||
{
|
||||
case Aws::Http::HttpMethod::HTTP_GET:
|
||||
case Aws::Http::HttpMethod::HTTP_HEAD:
|
||||
return S3MetricKind::Read;
|
||||
case Aws::Http::HttpMethod::HTTP_POST:
|
||||
case Aws::Http::HttpMethod::HTTP_DELETE:
|
||||
case Aws::Http::HttpMethod::HTTP_PUT:
|
||||
case Aws::Http::HttpMethod::HTTP_PATCH:
|
||||
return S3MetricKind::Write;
|
||||
}
|
||||
throw Exception("Unsupported request method", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void PocoHTTPClient::addMetric(const Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount) const
|
||||
{
|
||||
const ProfileEvents::Event events_map[static_cast<size_t>(S3MetricType::EnumSize)][static_cast<size_t>(S3MetricKind::EnumSize)] = {
|
||||
{ProfileEvents::S3ReadMicroseconds, ProfileEvents::S3WriteMicroseconds},
|
||||
{ProfileEvents::S3ReadRequestsCount, ProfileEvents::S3WriteRequestsCount},
|
||||
{ProfileEvents::S3ReadRequestsErrors, ProfileEvents::S3WriteRequestsErrors},
|
||||
{ProfileEvents::S3ReadRequestsThrottling, ProfileEvents::S3WriteRequestsThrottling},
|
||||
{ProfileEvents::S3ReadRequestsRedirects, ProfileEvents::S3WriteRequestsRedirects},
|
||||
};
|
||||
|
||||
const ProfileEvents::Event disk_s3_events_map[static_cast<size_t>(S3MetricType::EnumSize)][static_cast<size_t>(S3MetricKind::EnumSize)] = {
|
||||
{ProfileEvents::DiskS3ReadMicroseconds, ProfileEvents::DiskS3WriteMicroseconds},
|
||||
{ProfileEvents::DiskS3ReadRequestsCount, ProfileEvents::DiskS3WriteRequestsCount},
|
||||
{ProfileEvents::DiskS3ReadRequestsErrors, ProfileEvents::DiskS3WriteRequestsErrors},
|
||||
{ProfileEvents::DiskS3ReadRequestsThrottling, ProfileEvents::DiskS3WriteRequestsThrottling},
|
||||
{ProfileEvents::DiskS3ReadRequestsRedirects, ProfileEvents::DiskS3WriteRequestsRedirects},
|
||||
};
|
||||
|
||||
S3MetricKind kind = getMetricKind(request);
|
||||
|
||||
ProfileEvents::increment(events_map[static_cast<unsigned int>(type)][static_cast<unsigned int>(kind)], amount);
|
||||
if (for_disk_s3)
|
||||
ProfileEvents::increment(disk_s3_events_map[static_cast<unsigned int>(type)][static_cast<unsigned int>(kind)], amount);
|
||||
}
|
||||
|
||||
void PocoHTTPClient::makeRequestInternal(
|
||||
Aws::Http::HttpRequest & request,
|
||||
@ -189,45 +244,7 @@ void PocoHTTPClient::makeRequestInternal(
|
||||
if (enable_s3_requests_logging)
|
||||
LOG_TEST(log, "Make request to: {}", uri);
|
||||
|
||||
enum class S3MetricType
|
||||
{
|
||||
Microseconds,
|
||||
Count,
|
||||
Errors,
|
||||
Throttling,
|
||||
Redirects,
|
||||
|
||||
EnumSize,
|
||||
};
|
||||
|
||||
auto select_metric = [&request](S3MetricType type)
|
||||
{
|
||||
const ProfileEvents::Event events_map[][2] = {
|
||||
{ProfileEvents::S3ReadMicroseconds, ProfileEvents::S3WriteMicroseconds},
|
||||
{ProfileEvents::S3ReadRequestsCount, ProfileEvents::S3WriteRequestsCount},
|
||||
{ProfileEvents::S3ReadRequestsErrors, ProfileEvents::S3WriteRequestsErrors},
|
||||
{ProfileEvents::S3ReadRequestsThrottling, ProfileEvents::S3WriteRequestsThrottling},
|
||||
{ProfileEvents::S3ReadRequestsRedirects, ProfileEvents::S3WriteRequestsRedirects},
|
||||
};
|
||||
|
||||
static_assert((sizeof(events_map) / sizeof(events_map[0])) == static_cast<unsigned int>(S3MetricType::EnumSize));
|
||||
|
||||
switch (request.GetMethod())
|
||||
{
|
||||
case Aws::Http::HttpMethod::HTTP_GET:
|
||||
case Aws::Http::HttpMethod::HTTP_HEAD:
|
||||
return events_map[static_cast<unsigned int>(type)][0]; // Read
|
||||
case Aws::Http::HttpMethod::HTTP_POST:
|
||||
case Aws::Http::HttpMethod::HTTP_DELETE:
|
||||
case Aws::Http::HttpMethod::HTTP_PUT:
|
||||
case Aws::Http::HttpMethod::HTTP_PATCH:
|
||||
return events_map[static_cast<unsigned int>(type)][1]; // Write
|
||||
}
|
||||
|
||||
throw Exception("Unsupported request method", ErrorCodes::NOT_IMPLEMENTED);
|
||||
};
|
||||
|
||||
ProfileEvents::increment(select_metric(S3MetricType::Count));
|
||||
addMetric(request, S3MetricType::Count);
|
||||
CurrentMetrics::Increment metric_increment{CurrentMetrics::S3Requests};
|
||||
|
||||
try
|
||||
@ -334,7 +351,7 @@ void PocoHTTPClient::makeRequestInternal(
|
||||
auto & response_body_stream = session->receiveResponse(poco_response);
|
||||
|
||||
watch.stop();
|
||||
ProfileEvents::increment(select_metric(S3MetricType::Microseconds), watch.elapsedMicroseconds());
|
||||
addMetric(request, S3MetricType::Microseconds, watch.elapsedMicroseconds());
|
||||
|
||||
int status_code = static_cast<int>(poco_response.getStatus());
|
||||
|
||||
@ -349,7 +366,7 @@ void PocoHTTPClient::makeRequestInternal(
|
||||
if (enable_s3_requests_logging)
|
||||
LOG_TEST(log, "Redirecting request to new location: {}", location);
|
||||
|
||||
ProfileEvents::increment(select_metric(S3MetricType::Redirects));
|
||||
addMetric(request, S3MetricType::Redirects);
|
||||
|
||||
continue;
|
||||
}
|
||||
@ -387,7 +404,7 @@ void PocoHTTPClient::makeRequestInternal(
|
||||
LOG_WARNING(log, "Response for request contain <Error> tag in body, settings internal server error (500 code)");
|
||||
response->SetResponseCode(Aws::Http::HttpResponseCode::INTERNAL_SERVER_ERROR);
|
||||
|
||||
ProfileEvents::increment(select_metric(S3MetricType::Errors));
|
||||
addMetric(request, S3MetricType::Errors);
|
||||
if (error_report)
|
||||
error_report(request_configuration);
|
||||
|
||||
@ -401,11 +418,11 @@ void PocoHTTPClient::makeRequestInternal(
|
||||
|
||||
if (status_code == 429 || status_code == 503)
|
||||
{ // API throttling
|
||||
ProfileEvents::increment(select_metric(S3MetricType::Throttling));
|
||||
addMetric(request, S3MetricType::Throttling);
|
||||
}
|
||||
else if (status_code >= 300)
|
||||
{
|
||||
ProfileEvents::increment(select_metric(S3MetricType::Errors));
|
||||
addMetric(request, S3MetricType::Errors);
|
||||
if (status_code >= 500 && error_report)
|
||||
error_report(request_configuration);
|
||||
}
|
||||
@ -423,7 +440,7 @@ void PocoHTTPClient::makeRequestInternal(
|
||||
response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION);
|
||||
response->SetClientErrorMessage(getCurrentExceptionMessage(false));
|
||||
|
||||
ProfileEvents::increment(select_metric(S3MetricType::Errors));
|
||||
addMetric(request, S3MetricType::Errors);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -44,6 +44,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration
|
||||
const RemoteHostFilter & remote_host_filter;
|
||||
unsigned int s3_max_redirects;
|
||||
bool enable_s3_requests_logging;
|
||||
bool for_disk_s3;
|
||||
HeaderCollection extra_headers;
|
||||
|
||||
void updateSchemeAndRegion();
|
||||
@ -55,7 +56,8 @@ private:
|
||||
const String & force_region_,
|
||||
const RemoteHostFilter & remote_host_filter_,
|
||||
unsigned int s3_max_redirects_,
|
||||
bool enable_s3_requests_logging_
|
||||
bool enable_s3_requests_logging_,
|
||||
bool for_disk_s3_
|
||||
);
|
||||
|
||||
/// Constructor of Aws::Client::ClientConfiguration must be called after AWS SDK initialization.
|
||||
@ -113,18 +115,42 @@ public:
|
||||
Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const override;
|
||||
|
||||
private:
|
||||
|
||||
void makeRequestInternal(
|
||||
Aws::Http::HttpRequest & request,
|
||||
std::shared_ptr<PocoHTTPResponse> & response,
|
||||
Aws::Utils::RateLimits::RateLimiterInterface * readLimiter,
|
||||
Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const;
|
||||
|
||||
enum class S3MetricType
|
||||
{
|
||||
Microseconds,
|
||||
Count,
|
||||
Errors,
|
||||
Throttling,
|
||||
Redirects,
|
||||
|
||||
EnumSize,
|
||||
};
|
||||
|
||||
enum class S3MetricKind
|
||||
{
|
||||
Read,
|
||||
Write,
|
||||
|
||||
EnumSize,
|
||||
};
|
||||
|
||||
static S3MetricKind getMetricKind(const Aws::Http::HttpRequest & request);
|
||||
void addMetric(const Aws::Http::HttpRequest & request, S3MetricType type, ProfileEvents::Count amount = 1) const;
|
||||
|
||||
std::function<ClientConfigurationPerRequest(const Aws::Http::HttpRequest &)> per_request_configuration;
|
||||
std::function<void(const ClientConfigurationPerRequest &)> error_report;
|
||||
ConnectionTimeouts timeouts;
|
||||
const RemoteHostFilter & remote_host_filter;
|
||||
unsigned int s3_max_redirects;
|
||||
bool enable_s3_requests_logging;
|
||||
bool for_disk_s3;
|
||||
const HeaderCollection extra_headers;
|
||||
};
|
||||
|
||||
|
@ -87,7 +87,8 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeaders)
|
||||
region,
|
||||
remote_host_filter,
|
||||
s3_max_redirects,
|
||||
enable_s3_requests_logging
|
||||
enable_s3_requests_logging,
|
||||
/* for_disk_s3 = */ false
|
||||
);
|
||||
|
||||
client_configuration.endpointOverride = uri.endpoint;
|
||||
|
@ -543,7 +543,7 @@ public:
|
||||
/// AWS API tries credentials providers one by one. Some of providers (like ProfileConfigFileAWSCredentialsProvider) can be
|
||||
/// quite verbose even if nobody configured them. So we use our provider first and only after it use default providers.
|
||||
{
|
||||
DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, configuration.enable_s3_requests_logging);
|
||||
DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, configuration.enable_s3_requests_logging, configuration.for_disk_s3);
|
||||
AddProvider(std::make_shared<AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider>(aws_client_configuration));
|
||||
}
|
||||
|
||||
@ -580,7 +580,7 @@ public:
|
||||
}
|
||||
else if (Aws::Utils::StringUtils::ToLower(ec2_metadata_disabled.c_str()) != "true")
|
||||
{
|
||||
DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, configuration.enable_s3_requests_logging);
|
||||
DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, configuration.enable_s3_requests_logging, configuration.for_disk_s3);
|
||||
|
||||
/// See MakeDefaultHttpResourceClientConfiguration().
|
||||
/// This is part of EC2 metadata client, but unfortunately it can't be accessed from outside
|
||||
@ -700,9 +700,10 @@ namespace S3
|
||||
const String & force_region,
|
||||
const RemoteHostFilter & remote_host_filter,
|
||||
unsigned int s3_max_redirects,
|
||||
bool enable_s3_requests_logging)
|
||||
bool enable_s3_requests_logging,
|
||||
bool for_disk_s3)
|
||||
{
|
||||
return PocoHTTPClientConfiguration(force_region, remote_host_filter, s3_max_redirects, enable_s3_requests_logging);
|
||||
return PocoHTTPClientConfiguration(force_region, remote_host_filter, s3_max_redirects, enable_s3_requests_logging, for_disk_s3);
|
||||
}
|
||||
|
||||
URI::URI(const Poco::URI & uri_)
|
||||
|
@ -45,7 +45,8 @@ public:
|
||||
const String & force_region,
|
||||
const RemoteHostFilter & remote_host_filter,
|
||||
unsigned int s3_max_redirects,
|
||||
bool enable_s3_requests_logging);
|
||||
bool enable_s3_requests_logging,
|
||||
bool for_disk_s3);
|
||||
|
||||
private:
|
||||
ClientFactory();
|
||||
|
@ -1076,7 +1076,8 @@ void StorageS3::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration
|
||||
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
|
||||
settings.auth_settings.region,
|
||||
ctx->getRemoteHostFilter(), ctx->getGlobalContext()->getSettingsRef().s3_max_redirects,
|
||||
ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging);
|
||||
ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging,
|
||||
/* for_disk_s3 = */ false);
|
||||
|
||||
client_configuration.endpointOverride = upd.uri.endpoint;
|
||||
client_configuration.maxConnections = upd.rw_settings.max_connections;
|
||||
|
@ -33,9 +33,10 @@ def cluster():
|
||||
|
||||
init_list = {
|
||||
"ReadBufferFromS3Bytes": 0,
|
||||
"S3ReadMicroseconds": 0,
|
||||
"ReadBufferFromS3Microseconds": 0,
|
||||
"ReadBufferFromS3RequestsErrors": 0,
|
||||
"WriteBufferFromS3Bytes": 0,
|
||||
"S3ReadMicroseconds": 0,
|
||||
"S3ReadRequestsCount": 0,
|
||||
"S3ReadRequestsErrorsTotal": 0,
|
||||
"S3ReadRequestsErrors503": 0,
|
||||
@ -45,7 +46,16 @@ init_list = {
|
||||
"S3WriteRequestsErrorsTotal": 0,
|
||||
"S3WriteRequestsErrors503": 0,
|
||||
"S3WriteRequestsRedirects": 0,
|
||||
"WriteBufferFromS3Bytes": 0,
|
||||
"DiskS3ReadMicroseconds": 0,
|
||||
"DiskS3ReadRequestsCount": 0,
|
||||
"DiskS3ReadRequestsErrorsTotal": 0,
|
||||
"DiskS3ReadRequestsErrors503": 0,
|
||||
"DiskS3ReadRequestsRedirects": 0,
|
||||
"DiskS3WriteMicroseconds": 0,
|
||||
"DiskS3WriteRequestsCount": 0,
|
||||
"DiskS3WriteRequestsErrorsTotal": 0,
|
||||
"DiskS3WriteRequestsErrors503": 0,
|
||||
"DiskS3WriteRequestsRedirects": 0,
|
||||
}
|
||||
|
||||
|
||||
|
22
tests/performance/lz4.xml
Normal file
22
tests/performance/lz4.xml
Normal file
@ -0,0 +1,22 @@
|
||||
<test>
|
||||
<create_query>create table t_lz4(a UInt64) engine=MergeTree order by tuple()</create_query>
|
||||
<create_query>create table t_lz4_norm(a UInt64) engine=MergeTree order by tuple()</create_query>
|
||||
<create_query>create table t_lz4_uncomp(a UInt32) engine=MergeTree order by a</create_query>
|
||||
|
||||
<fill_query>insert into t_lz4 select number % 100 from numbers_mt(5e7) order by rand()</fill_query>
|
||||
<fill_query>optimize table t_lz4 final</fill_query>
|
||||
|
||||
<fill_query>insert into t_lz4_norm select number from numbers_mt(5e7) order by rand()</fill_query>
|
||||
<fill_query>optimize table t_lz4_norm final</fill_query>
|
||||
|
||||
<fill_query>insert into t_lz4_uncomp select number from numbers_mt(5e7)</fill_query>
|
||||
<fill_query>optimize table t_lz4_uncomp final</fill_query>
|
||||
|
||||
<query>select a from t_lz4 format Null</query>
|
||||
<query>select a from t_lz4_norm format Null</query>
|
||||
<query>select a from t_lz4_uncomp format Null</query>
|
||||
|
||||
<drop_query>drop table t_lz4</drop_query>
|
||||
<drop_query>drop table t_lz4_norm</drop_query>
|
||||
<drop_query>drop table t_lz4_uncomp</drop_query>
|
||||
</test>
|
39
tests/performance/lz4_hits_columns.xml
Normal file
39
tests/performance/lz4_hits_columns.xml
Normal file
@ -0,0 +1,39 @@
|
||||
<test>
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>column</name>
|
||||
<values>
|
||||
<value>ClientIP</value>
|
||||
<value>ClientTimeZone</value>
|
||||
<value>CookieEnable</value>
|
||||
<value>CounterClass</value>
|
||||
<value>CounterID</value>
|
||||
<value>EventDate</value>
|
||||
<value>EventTime</value>
|
||||
<value>GoodEvent</value>
|
||||
<value>HitColor</value>
|
||||
<value>JavaEnable</value>
|
||||
<value>OpenerName</value>
|
||||
<value>PageCharset</value>
|
||||
<value>ParamCurrency</value>
|
||||
<value>ParamPrice</value>
|
||||
<value>Referer</value>
|
||||
<value>RefererCategoryID</value>
|
||||
<value>RefererHash</value>
|
||||
<value>RegionID</value>
|
||||
<value>SearchPhrase</value>
|
||||
<value>SilverlightVersion4</value>
|
||||
<value>Title</value>
|
||||
<value>TraficSourceID</value>
|
||||
<value>URLCategoryID</value>
|
||||
<value>UserAgent</value>
|
||||
<value>UserAgentMinor</value>
|
||||
<value>UserID</value>
|
||||
<value>WatchID</value>
|
||||
<value>WindowName</value>
|
||||
</values>
|
||||
</substitution>
|
||||
</substitutions>
|
||||
|
||||
<query>select {column} from hits_100m_single format Null</query>
|
||||
</test>
|
@ -361,6 +361,8 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress
|
||||
|
||||
#endif
|
||||
|
||||
#if !defined(OS_DARWIN) && !defined(OS_FREEBSD)
|
||||
|
||||
uint32_t getInode(const char * self)
|
||||
{
|
||||
std::ifstream maps("/proc/self/maps");
|
||||
@ -386,6 +388,8 @@ uint32_t getInode(const char * self)
|
||||
return 0;
|
||||
}
|
||||
|
||||
#endif
|
||||
|
||||
int main(int/* argc*/, char* argv[])
|
||||
{
|
||||
char self[4096] = {0};
|
||||
@ -409,6 +413,7 @@ int main(int/* argc*/, char* argv[])
|
||||
else
|
||||
name = file_path;
|
||||
|
||||
#if !defined(OS_DARWIN) && !defined(OS_FREEBSD)
|
||||
/// get inode of this executable
|
||||
uint32_t inode = getInode(self);
|
||||
if (inode == 0)
|
||||
@ -460,6 +465,7 @@ int main(int/* argc*/, char* argv[])
|
||||
printf("No target executable - decompression only was performed.\n");
|
||||
return 0;
|
||||
}
|
||||
#endif
|
||||
|
||||
int input_fd = open(self, O_RDONLY);
|
||||
if (input_fd == -1)
|
||||
@ -522,19 +528,21 @@ int main(int/* argc*/, char* argv[])
|
||||
|
||||
if (has_exec)
|
||||
{
|
||||
#if !defined(OS_DARWIN) && !defined(OS_FREEBSD)
|
||||
/// write one byte to the lock in case other copies of compressed are running to indicate that
|
||||
/// execution should be performed
|
||||
write(lock, "1", 1);
|
||||
|
||||
#endif
|
||||
execv(self, argv);
|
||||
|
||||
/// This part of code will be reached only if error happened
|
||||
perror("execv");
|
||||
return 1;
|
||||
}
|
||||
|
||||
#if !defined(OS_DARWIN) && !defined(OS_FREEBSD)
|
||||
/// since inodes can be reused - it's a precaution if lock file already exists and have size of 1
|
||||
ftruncate(lock, 0);
|
||||
#endif
|
||||
|
||||
printf("No target executable - decompression only was performed.\n");
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user