Merge branch 'master' into patch-25

This commit is contained in:
Denny Crane 2023-06-06 10:32:32 -03:00 committed by GitHub
commit 08331ebcfe
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
94 changed files with 1427 additions and 1279 deletions

View File

@ -46,7 +46,12 @@ jobs:
- name: Python unit tests - name: Python unit tests
run: | run: |
cd "$GITHUB_WORKSPACE/tests/ci" cd "$GITHUB_WORKSPACE/tests/ci"
echo "Testing the main ci directory"
python3 -m unittest discover -s . -p '*_test.py' python3 -m unittest discover -s . -p '*_test.py'
for dir in *_lambda/; do
echo "Testing $dir"
python3 -m unittest discover -s "$dir" -p '*_test.py'
done
DockerHubPushAarch64: DockerHubPushAarch64:
needs: CheckLabels needs: CheckLabels
runs-on: [self-hosted, style-checker-aarch64] runs-on: [self-hosted, style-checker-aarch64]

View File

@ -626,7 +626,9 @@ if args.report == "main":
message_array.append(str(faster_queries) + " faster") message_array.append(str(faster_queries) + " faster")
if slower_queries: if slower_queries:
if slower_queries > 3: # This threshold should be synchronized with the value in https://github.com/ClickHouse/ClickHouse/blob/master/tests/ci/performance_comparison_check.py#L225
# False positives rate should be < 1%: https://shorturl.at/CDEK8
if slower_queries > 5:
status = "failure" status = "failure"
message_array.append(str(slower_queries) + " slower") message_array.append(str(slower_queries) + " slower")

View File

@ -10,14 +10,14 @@ Columns:
- `user` (String) The user who made the query. Keep in mind that for distributed processing, queries are sent to remote servers under the `default` user. The field contains the username for a specific query, not for a query that this query initiated. - `user` (String) The user who made the query. Keep in mind that for distributed processing, queries are sent to remote servers under the `default` user. The field contains the username for a specific query, not for a query that this query initiated.
- `address` (String) The IP address the request was made from. The same for distributed processing. To track where a distributed query was originally made from, look at `system.processes` on the query requestor server. - `address` (String) The IP address the request was made from. The same for distributed processing. To track where a distributed query was originally made from, look at `system.processes` on the query requestor server.
- `elapsed` (Float64) The time in seconds since request execution started. - `elapsed` (Float64) The time in seconds since request execution started.
- `rows_read` (UInt64) The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. - `read_rows` (UInt64) The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers.
- `bytes_read` (UInt64) The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. - `read_bytes` (UInt64) The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers.
- `total_rows_approx` (UInt64) The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known. - `total_rows_approx` (UInt64) The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known.
- `memory_usage` (UInt64) Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) setting. - `memory_usage` (Int64) Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) setting.
- `query` (String) The query text. For `INSERT`, it does not include the data to insert. - `query` (String) The query text. For `INSERT`, it does not include the data to insert.
- `query_id` (String) Query ID, if defined. - `query_id` (String) Query ID, if defined.
- `is_cancelled` (Int8) Was query cancelled. - `is_cancelled` (UInt8) Was query cancelled.
- `is_all_data_sent` (Int8) Was all data sent to the client (in other words query had been finished on the server). - `is_all_data_sent` (UInt8) Was all data sent to the client (in other words query had been finished on the server).
```sql ```sql
SELECT * FROM system.processes LIMIT 10 FORMAT Vertical; SELECT * FROM system.processes LIMIT 10 FORMAT Vertical;

View File

@ -0,0 +1,28 @@
---
slug: /en/operations/system-tables/user_processes
---
# user_processes
This system table can be used to get overview of memory usage and ProfileEvents of users.
Columns:
- `user` ([String](../../sql-reference/data-types/string.md)) — User name.
- `memory_usage` ([Int64](../../sql-reference/data-types/int-uint#int-ranges)) Sum of RAM used by all processes of the user. It might not include some types of dedicated memory. See the [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) setting.
- `peak_memory_usage` ([Int64](../../sql-reference/data-types/int-uint#int-ranges)) — The peak of memory usage of the user. It can be reset when no queries are run for the user.
- `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/map)) Summary of ProfileEvents that measure different metrics for the user. The description of them could be found in the table [system.events](../../operations/system-tables/events.md#system_tables-events)
```sql
SELECT * FROM system.user_processes LIMIT 10 FORMAT Vertical;
```
```response
Row 1:
──────
user: default
memory_usage: 9832
peak_memory_usage: 9832
ProfileEvents: {'Query':5,'SelectQuery':5,'QueriesWithSubqueries':38,'SelectQueriesWithSubqueries':38,'QueryTimeMicroseconds':842048,'SelectQueryTimeMicroseconds':842048,'ReadBufferFromFileDescriptorRead':6,'ReadBufferFromFileDescriptorReadBytes':234,'IOBufferAllocs':3,'IOBufferAllocBytes':98493,'ArenaAllocChunks':283,'ArenaAllocBytes':1482752,'FunctionExecute':670,'TableFunctionExecute':16,'DiskReadElapsedMicroseconds':19,'NetworkSendElapsedMicroseconds':684,'NetworkSendBytes':139498,'SelectedRows':6076,'SelectedBytes':685802,'ContextLock':1140,'RWLockAcquiredReadLocks':193,'RWLockReadersWaitMilliseconds':4,'RealTimeMicroseconds':1585163,'UserTimeMicroseconds':889767,'SystemTimeMicroseconds':13630,'SoftPageFaults':1947,'OSCPUWaitMicroseconds':6,'OSCPUVirtualTimeMicroseconds':903251,'OSReadChars':28631,'OSWriteChars':28888,'QueryProfilerRuns':3,'LogTrace':79,'LogDebug':24}
1 row in set. Elapsed: 0.010 sec.
```

View File

@ -130,15 +130,31 @@ void LocalServer::initialize(Poco::Util::Application & self)
}); });
#endif #endif
IOThreadPool::initialize( getIOThreadPool().initialize(
config().getUInt("max_io_thread_pool_size", 100), config().getUInt("max_io_thread_pool_size", 100),
config().getUInt("max_io_thread_pool_free_size", 0), config().getUInt("max_io_thread_pool_free_size", 0),
config().getUInt("io_thread_pool_queue_size", 10000)); config().getUInt("io_thread_pool_queue_size", 10000));
OutdatedPartsLoadingThreadPool::initialize(
config().getUInt("max_outdated_parts_loading_thread_pool_size", 16), const size_t active_parts_loading_threads = config().getUInt("max_active_parts_loading_thread_pool_size", 64);
getActivePartsLoadingThreadPool().initialize(
active_parts_loading_threads,
0, // We don't need any threads one all the parts will be loaded 0, // We don't need any threads one all the parts will be loaded
config().getUInt("max_outdated_parts_loading_thread_pool_size", 16)); active_parts_loading_threads);
const size_t outdated_parts_loading_threads = config().getUInt("max_outdated_parts_loading_thread_pool_size", 32);
getOutdatedPartsLoadingThreadPool().initialize(
outdated_parts_loading_threads,
0, // We don't need any threads one all the parts will be loaded
outdated_parts_loading_threads);
getOutdatedPartsLoadingThreadPool().setMaxTurboThreads(active_parts_loading_threads);
const size_t cleanup_threads = config().getUInt("max_parts_cleaning_thread_pool_size", 128);
getPartsCleaningThreadPool().initialize(
cleanup_threads,
0, // We don't need any threads one all the parts will be deleted
cleanup_threads);
} }

View File

@ -683,21 +683,36 @@ try
}); });
#endif #endif
IOThreadPool::initialize( getIOThreadPool().initialize(
server_settings.max_io_thread_pool_size, server_settings.max_io_thread_pool_size,
server_settings.max_io_thread_pool_free_size, server_settings.max_io_thread_pool_free_size,
server_settings.io_thread_pool_queue_size); server_settings.io_thread_pool_queue_size);
BackupsIOThreadPool::initialize( getBackupsIOThreadPool().initialize(
server_settings.max_backups_io_thread_pool_size, server_settings.max_backups_io_thread_pool_size,
server_settings.max_backups_io_thread_pool_free_size, server_settings.max_backups_io_thread_pool_free_size,
server_settings.backups_io_thread_pool_queue_size); server_settings.backups_io_thread_pool_queue_size);
OutdatedPartsLoadingThreadPool::initialize( getActivePartsLoadingThreadPool().initialize(
server_settings.max_active_parts_loading_thread_pool_size,
0, // We don't need any threads once all the parts will be loaded
server_settings.max_active_parts_loading_thread_pool_size);
getOutdatedPartsLoadingThreadPool().initialize(
server_settings.max_outdated_parts_loading_thread_pool_size, server_settings.max_outdated_parts_loading_thread_pool_size,
0, // We don't need any threads one all the parts will be loaded 0, // We don't need any threads once all the parts will be loaded
server_settings.max_outdated_parts_loading_thread_pool_size); server_settings.max_outdated_parts_loading_thread_pool_size);
/// It could grow if we need to synchronously wait until all the data parts will be loaded.
getOutdatedPartsLoadingThreadPool().setMaxTurboThreads(
server_settings.max_active_parts_loading_thread_pool_size
);
getPartsCleaningThreadPool().initialize(
server_settings.max_parts_cleaning_thread_pool_size,
0, // We don't need any threads one all the parts will be deleted
server_settings.max_parts_cleaning_thread_pool_size);
/// Initialize global local cache for remote filesystem. /// Initialize global local cache for remote filesystem.
if (config().has("local_cache_for_remote_fs")) if (config().has("local_cache_for_remote_fs"))
{ {
@ -1226,6 +1241,36 @@ try
global_context->getMessageBrokerSchedulePool().increaseThreadsCount(server_settings_.background_message_broker_schedule_pool_size); global_context->getMessageBrokerSchedulePool().increaseThreadsCount(server_settings_.background_message_broker_schedule_pool_size);
global_context->getDistributedSchedulePool().increaseThreadsCount(server_settings_.background_distributed_schedule_pool_size); global_context->getDistributedSchedulePool().increaseThreadsCount(server_settings_.background_distributed_schedule_pool_size);
getIOThreadPool().reloadConfiguration(
server_settings.max_io_thread_pool_size,
server_settings.max_io_thread_pool_free_size,
server_settings.io_thread_pool_queue_size);
getBackupsIOThreadPool().reloadConfiguration(
server_settings.max_backups_io_thread_pool_size,
server_settings.max_backups_io_thread_pool_free_size,
server_settings.backups_io_thread_pool_queue_size);
getActivePartsLoadingThreadPool().reloadConfiguration(
server_settings.max_active_parts_loading_thread_pool_size,
0, // We don't need any threads once all the parts will be loaded
server_settings.max_active_parts_loading_thread_pool_size);
getOutdatedPartsLoadingThreadPool().reloadConfiguration(
server_settings.max_outdated_parts_loading_thread_pool_size,
0, // We don't need any threads once all the parts will be loaded
server_settings.max_outdated_parts_loading_thread_pool_size);
/// It could grow if we need to synchronously wait until all the data parts will be loaded.
getOutdatedPartsLoadingThreadPool().setMaxTurboThreads(
server_settings.max_active_parts_loading_thread_pool_size
);
getPartsCleaningThreadPool().reloadConfiguration(
server_settings.max_parts_cleaning_thread_pool_size,
0, // We don't need any threads one all the parts will be deleted
server_settings.max_parts_cleaning_thread_pool_size);
if (config->has("resources")) if (config->has("resources"))
{ {
global_context->getResourceManager()->updateConfiguration(*config); global_context->getResourceManager()->updateConfiguration(*config);

View File

@ -161,7 +161,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s
/* dest_key= */ blob_path[0], /* dest_key= */ blob_path[0],
request_settings, request_settings,
object_attributes, object_attributes,
threadPoolCallbackRunner<void>(BackupsIOThreadPool::get(), "BackupReaderS3"), threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupReaderS3"),
/* for_disk_s3= */ true); /* for_disk_s3= */ true);
return file_size; return file_size;
@ -212,7 +212,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src
fs::path(s3_uri.key) / path_in_backup, fs::path(s3_uri.key) / path_in_backup,
request_settings, request_settings,
{}, {},
threadPoolCallbackRunner<void>(BackupsIOThreadPool::get(), "BackupWriterS3")); threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupWriterS3"));
return; /// copied! return; /// copied!
} }
} }
@ -224,7 +224,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src
void BackupWriterS3::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) void BackupWriterS3::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length)
{ {
copyDataToS3File(create_read_buffer, start_pos, length, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, request_settings, {}, copyDataToS3File(create_read_buffer, start_pos, length, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, request_settings, {},
threadPoolCallbackRunner<void>(BackupsIOThreadPool::get(), "BackupWriterS3")); threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupWriterS3"));
} }
BackupWriterS3::~BackupWriterS3() = default; BackupWriterS3::~BackupWriterS3() = default;
@ -258,7 +258,7 @@ std::unique_ptr<WriteBuffer> BackupWriterS3::writeFile(const String & file_name)
DBMS_DEFAULT_BUFFER_SIZE, DBMS_DEFAULT_BUFFER_SIZE,
request_settings, request_settings,
std::nullopt, std::nullopt,
threadPoolCallbackRunner<void>(BackupsIOThreadPool::get(), "BackupWriterS3"), threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupWriterS3"),
write_settings); write_settings);
} }

View File

@ -278,7 +278,7 @@ public:
static Int32 cancelled_status() { return exit_after_signals.load(); } static Int32 cancelled_status() { return exit_after_signals.load(); }
}; };
/// This signal handler is set only for SIGINT. /// This signal handler is set for SIGINT and SIGQUIT.
void interruptSignalHandler(int signum) void interruptSignalHandler(int signum)
{ {
if (QueryInterruptHandler::try_stop()) if (QueryInterruptHandler::try_stop())
@ -317,6 +317,9 @@ void ClientBase::setupSignalHandler()
if (sigaction(SIGINT, &new_act, nullptr)) if (sigaction(SIGINT, &new_act, nullptr))
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER); throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
if (sigaction(SIGQUIT, &new_act, nullptr))
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
} }

View File

@ -137,6 +137,8 @@
M(ObjectStorageAzureThreadsActive, "Number of threads in the AzureObjectStorage thread pool running a task.") \ M(ObjectStorageAzureThreadsActive, "Number of threads in the AzureObjectStorage thread pool running a task.") \
M(MergeTreePartsLoaderThreads, "Number of threads in the MergeTree parts loader thread pool.") \ M(MergeTreePartsLoaderThreads, "Number of threads in the MergeTree parts loader thread pool.") \
M(MergeTreePartsLoaderThreadsActive, "Number of threads in the MergeTree parts loader thread pool running a task.") \ M(MergeTreePartsLoaderThreadsActive, "Number of threads in the MergeTree parts loader thread pool running a task.") \
M(MergeTreeOutdatedPartsLoaderThreads, "Number of threads in the threadpool for loading Outdated data parts.") \
M(MergeTreeOutdatedPartsLoaderThreadsActive, "Number of active threads in the threadpool for loading Outdated data parts.") \
M(MergeTreePartsCleanerThreads, "Number of threads in the MergeTree parts cleaner thread pool.") \ M(MergeTreePartsCleanerThreads, "Number of threads in the MergeTree parts cleaner thread pool.") \
M(MergeTreePartsCleanerThreadsActive, "Number of threads in the MergeTree parts cleaner thread pool running a task.") \ M(MergeTreePartsCleanerThreadsActive, "Number of threads in the MergeTree parts cleaner thread pool running a task.") \
M(SystemReplicasThreads, "Number of threads in the system.replicas thread pool.") \ M(SystemReplicasThreads, "Number of threads in the system.replicas thread pool.") \

View File

@ -21,7 +21,9 @@ namespace DB
M(UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 0) \ M(UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 0) \
M(UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0) \ M(UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0) \
M(UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0) \ M(UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0) \
M(UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The maximum number of threads that would be used for loading outdated data parts on startup", 0) \ M(UInt64, max_active_parts_loading_thread_pool_size, 64, "The number of threads to load active set of data parts (Active ones) at startup.", 0) \
M(UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The number of threads to load inactive set of data parts (Outdated ones) at startup.", 0) \
M(UInt64, max_parts_cleaning_thread_pool_size, 128, "The number of threads for concurrent removal of inactive data parts.", 0) \
M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \ M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \
M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \
M(UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0) \ M(UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0) \

View File

@ -51,19 +51,11 @@ void SerializationUUID::deserializeTextQuoted(IColumn & column, ReadBuffer & ist
{ {
assertChar('\'', istr); assertChar('\'', istr);
char * next_pos = find_first_symbols<'\\', '\''>(istr.position(), istr.buffer().end()); char * next_pos = find_first_symbols<'\\', '\''>(istr.position(), istr.buffer().end());
size_t len = next_pos - istr.position(); const size_t len = next_pos - istr.position();
if ((len == 32) && (istr.position()[32] == '\'')) if ((len == 32 || len == 36) && istr.position()[len] == '\'')
{ {
parseUUIDWithoutSeparator( uuid = parseUUID(std::span(reinterpret_cast<const UInt8 *>(istr.position()), len));
reinterpret_cast<const UInt8 *>(istr.position()), std::reverse_iterator<UInt8 *>(reinterpret_cast<UInt8 *>(&uuid) + 16)); istr.ignore(len + 1);
istr.ignore(33);
fast = true;
}
else if ((len == 36) && (istr.position()[36] == '\''))
{
parseUUID(
reinterpret_cast<const UInt8 *>(istr.position()), std::reverse_iterator<UInt8 *>(reinterpret_cast<UInt8 *>(&uuid) + 16));
istr.ignore(37);
fast = true; fast = true;
} }
else else

View File

@ -1219,7 +1219,7 @@ off_t CachedOnDiskReadBufferFromFile::getPosition()
void CachedOnDiskReadBufferFromFile::assertCorrectness() const void CachedOnDiskReadBufferFromFile::assertCorrectness() const
{ {
if (!CachedObjectStorage::canUseReadThroughCache() if (!CachedObjectStorage::canUseReadThroughCache(settings)
&& !settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) && !settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache usage is not allowed (query_id: {})", query_id); throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache usage is not allowed (query_id: {})", query_id);
} }

View File

@ -36,7 +36,7 @@ ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(
with_cache = settings.remote_fs_cache with_cache = settings.remote_fs_cache
&& settings.enable_filesystem_cache && settings.enable_filesystem_cache
&& (!query_id.empty() || settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache); && (!query_id.empty() || settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache || !settings.avoid_readthrough_cache_outside_query_context);
} }
SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(const StoredObject & object) SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(const StoredObject & object)

View File

@ -57,7 +57,7 @@ ReadSettings CachedObjectStorage::patchSettings(const ReadSettings & read_settin
ReadSettings modified_settings{read_settings}; ReadSettings modified_settings{read_settings};
modified_settings.remote_fs_cache = cache; modified_settings.remote_fs_cache = cache;
if (!canUseReadThroughCache()) if (!canUseReadThroughCache(read_settings))
modified_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true; modified_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true;
return object_storage->patchSettings(modified_settings); return object_storage->patchSettings(modified_settings);
@ -227,8 +227,11 @@ String CachedObjectStorage::getObjectsNamespace() const
return object_storage->getObjectsNamespace(); return object_storage->getObjectsNamespace();
} }
bool CachedObjectStorage::canUseReadThroughCache() bool CachedObjectStorage::canUseReadThroughCache(const ReadSettings & settings)
{ {
if (!settings.avoid_readthrough_cache_outside_query_context)
return true;
return CurrentThread::isInitialized() return CurrentThread::isInitialized()
&& CurrentThread::get().getQueryContext() && CurrentThread::get().getQueryContext()
&& !CurrentThread::getQueryId().empty(); && !CurrentThread::getQueryId().empty();

View File

@ -112,7 +112,9 @@ public:
WriteSettings getAdjustedSettingsFromMetadataFile(const WriteSettings & settings, const std::string & path) const override; WriteSettings getAdjustedSettingsFromMetadataFile(const WriteSettings & settings, const std::string & path) const override;
static bool canUseReadThroughCache(); const FileCacheSettings & getCacheSettings() const { return cache_settings; }
static bool canUseReadThroughCache(const ReadSettings & settings);
private: private:
FileCache::Key getCacheKey(const std::string & path) const; FileCache::Key getCacheKey(const std::string & path) const;

View File

@ -596,7 +596,8 @@ void DiskObjectStorage::writeFileUsingBlobWritingFunction(const String & path, W
{ {
LOG_TEST(log, "Write file: {}", path); LOG_TEST(log, "Write file: {}", path);
auto transaction = createObjectStorageTransaction(); auto transaction = createObjectStorageTransaction();
return transaction->writeFileUsingBlobWritingFunction(path, mode, std::move(write_blob_function)); transaction->writeFileUsingBlobWritingFunction(path, mode, std::move(write_blob_function));
transaction->commit();
} }
void DiskObjectStorage::applyNewSettings( void DiskObjectStorage::applyNewSettings(

View File

@ -710,8 +710,6 @@ void DiskObjectStorageTransaction::writeFileUsingBlobWritingFunction(
metadata_transaction->createMetadataFile(path, blob_name, object_size); metadata_transaction->createMetadataFile(path, blob_name, object_size);
else else
metadata_transaction->addBlobToMetadata(path, blob_name, object_size); metadata_transaction->addBlobToMetadata(path, blob_name, object_size);
metadata_transaction->commit();
} }

View File

@ -364,7 +364,7 @@ std::unique_ptr<ReadBuffer> FormatFactory::wrapReadBufferIfNeeded(
settings.max_download_buffer_size); settings.max_download_buffer_size);
res = wrapInParallelReadBufferIfSupported( res = wrapInParallelReadBufferIfSupported(
buf, threadPoolCallbackRunner<void>(IOThreadPool::get(), "ParallelRead"), buf, threadPoolCallbackRunner<void>(getIOThreadPool().get(), "ParallelRead"),
max_download_threads, settings.max_download_buffer_size, file_size); max_download_threads, settings.max_download_buffer_size, file_size);
} }

View File

@ -31,6 +31,7 @@ namespace ErrorCodes
extern const int CANNOT_PARSE_QUOTED_STRING; extern const int CANNOT_PARSE_QUOTED_STRING;
extern const int CANNOT_PARSE_DATETIME; extern const int CANNOT_PARSE_DATETIME;
extern const int CANNOT_PARSE_DATE; extern const int CANNOT_PARSE_DATE;
extern const int CANNOT_PARSE_UUID;
extern const int INCORRECT_DATA; extern const int INCORRECT_DATA;
extern const int ATTEMPT_TO_READ_AFTER_EOF; extern const int ATTEMPT_TO_READ_AFTER_EOF;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
@ -46,48 +47,45 @@ inline void parseHex(IteratorSrc src, IteratorDst dst)
dst[dst_pos] = unhex2(reinterpret_cast<const char *>(&src[src_pos])); dst[dst_pos] = unhex2(reinterpret_cast<const char *>(&src[src_pos]));
} }
void parseUUID(const UInt8 * src36, UInt8 * dst16) UUID parseUUID(std::span<const UInt8> src)
{ {
/// If string is not like UUID - implementation specific behaviour. UUID uuid;
const auto * src_ptr = src.data();
auto * dst = reinterpret_cast<UInt8 *>(&uuid);
const auto size = src.size();
parseHex<4>(&src36[0], &dst16[0]); #if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
parseHex<2>(&src36[9], &dst16[4]); const std::reverse_iterator dst_it(dst + sizeof(UUID));
parseHex<2>(&src36[14], &dst16[6]); #endif
parseHex<2>(&src36[19], &dst16[8]); if (size == 36)
parseHex<6>(&src36[24], &dst16[10]); {
} #if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
parseHex<4>(src_ptr, dst_it + 8);
parseHex<2>(src_ptr + 9, dst_it + 12);
parseHex<2>(src_ptr + 14, dst_it + 14);
parseHex<2>(src_ptr + 19, dst_it);
parseHex<6>(src_ptr + 24, dst_it + 2);
#else
parseHex<4>(src_ptr, dst);
parseHex<2>(src_ptr + 9, dst + 4);
parseHex<2>(src_ptr + 14, dst + 6);
parseHex<2>(src_ptr + 19, dst + 8);
parseHex<6>(src_ptr + 24, dst + 10);
#endif
}
else if (size == 32)
{
#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
parseHex<8>(src_ptr, dst_it + 8);
parseHex<8>(src_ptr + 16, dst_it);
#else
parseHex<16>(src_ptr, dst);
#endif
}
else
throw Exception(ErrorCodes::CANNOT_PARSE_UUID, "Unexpected length when trying to parse UUID ({})", size);
void parseUUIDWithoutSeparator(const UInt8 * src36, UInt8 * dst16) return uuid;
{
/// If string is not like UUID - implementation specific behaviour.
parseHex<16>(&src36[0], &dst16[0]);
}
/** Function used when byte ordering is important when parsing uuid
* ex: When we create an UUID type
*/
void parseUUID(const UInt8 * src36, std::reverse_iterator<UInt8 *> dst16)
{
/// If string is not like UUID - implementation specific behaviour.
/// FIXME This code looks like trash.
parseHex<4>(&src36[0], dst16 + 8);
parseHex<2>(&src36[9], dst16 + 12);
parseHex<2>(&src36[14], dst16 + 14);
parseHex<2>(&src36[19], dst16);
parseHex<6>(&src36[24], dst16 + 2);
}
/** Function used when byte ordering is important when parsing uuid
* ex: When we create an UUID type
*/
void parseUUIDWithoutSeparator(const UInt8 * src36, std::reverse_iterator<UInt8 *> dst16)
{
/// If string is not like UUID - implementation specific behaviour.
parseHex<8>(&src36[0], dst16 + 8);
parseHex<8>(&src36[16], dst16);
} }
void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf) void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf)

View File

@ -8,6 +8,7 @@
#include <algorithm> #include <algorithm>
#include <iterator> #include <iterator>
#include <bit> #include <bit>
#include <span>
#include <type_traits> #include <type_traits>
@ -623,12 +624,6 @@ struct NullOutput
void push_back(char) {} /// NOLINT void push_back(char) {} /// NOLINT
}; };
void parseUUID(const UInt8 * src36, UInt8 * dst16);
void parseUUIDWithoutSeparator(const UInt8 * src36, UInt8 * dst16);
void parseUUID(const UInt8 * src36, std::reverse_iterator<UInt8 *> dst16);
void parseUUIDWithoutSeparator(const UInt8 * src36, std::reverse_iterator<UInt8 *> dst16);
template <typename ReturnType> template <typename ReturnType>
ReturnType readDateTextFallback(LocalDate & date, ReadBuffer & buf); ReturnType readDateTextFallback(LocalDate & date, ReadBuffer & buf);
@ -770,6 +765,8 @@ inline bool tryReadDateText(ExtendedDayNum & date, ReadBuffer & buf)
return readDateTextImpl<bool>(date, buf); return readDateTextImpl<bool>(date, buf);
} }
UUID parseUUID(std::span<const UInt8> src);
template <typename ReturnType = void> template <typename ReturnType = void>
inline ReturnType readUUIDTextImpl(UUID & uuid, ReadBuffer & buf) inline ReturnType readUUIDTextImpl(UUID & uuid, ReadBuffer & buf)
{ {
@ -797,12 +794,9 @@ inline ReturnType readUUIDTextImpl(UUID & uuid, ReadBuffer & buf)
return ReturnType(false); return ReturnType(false);
} }
} }
parseUUID(reinterpret_cast<const UInt8 *>(s), std::reverse_iterator<UInt8 *>(reinterpret_cast<UInt8 *>(&uuid) + 16));
} }
else
parseUUIDWithoutSeparator(reinterpret_cast<const UInt8 *>(s), std::reverse_iterator<UInt8 *>(reinterpret_cast<UInt8 *>(&uuid) + 16));
uuid = parseUUID({reinterpret_cast<const UInt8 *>(s), size});
return ReturnType(true); return ReturnType(true);
} }
else else

View File

@ -99,6 +99,8 @@ struct ReadSettings
bool read_from_filesystem_cache_if_exists_otherwise_bypass_cache = false; bool read_from_filesystem_cache_if_exists_otherwise_bypass_cache = false;
bool enable_filesystem_cache_log = false; bool enable_filesystem_cache_log = false;
bool is_file_cache_persistent = false; /// Some files can be made non-evictable. bool is_file_cache_persistent = false; /// Some files can be made non-evictable.
/// Don't populate cache when the read is not part of query execution (e.g. background thread).
bool avoid_readthrough_cache_outside_query_context = true;
size_t filesystem_cache_max_download_size = (128UL * 1024 * 1024 * 1024); size_t filesystem_cache_max_download_size = (128UL * 1024 * 1024 * 1024);
bool skip_download_if_exceeds_query_cache = true; bool skip_download_if_exceeds_query_cache = true;

View File

@ -9,8 +9,12 @@ namespace CurrentMetrics
extern const Metric IOThreadsActive; extern const Metric IOThreadsActive;
extern const Metric BackupsIOThreads; extern const Metric BackupsIOThreads;
extern const Metric BackupsIOThreadsActive; extern const Metric BackupsIOThreadsActive;
extern const Metric OutdatedPartsLoadingThreads; extern const Metric MergeTreePartsLoaderThreads;
extern const Metric OutdatedPartsLoadingThreadsActive; extern const Metric MergeTreePartsLoaderThreadsActive;
extern const Metric MergeTreePartsCleanerThreads;
extern const Metric MergeTreePartsCleanerThreadsActive;
extern const Metric MergeTreeOutdatedPartsLoaderThreads;
extern const Metric MergeTreeOutdatedPartsLoaderThreadsActive;
} }
namespace DB namespace DB
@ -21,88 +25,117 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
std::unique_ptr<ThreadPool> IOThreadPool::instance;
void IOThreadPool::initialize(size_t max_threads, size_t max_free_threads, size_t queue_size) StaticThreadPool::StaticThreadPool(
const String & name_,
CurrentMetrics::Metric threads_metric_,
CurrentMetrics::Metric threads_active_metric_)
: name(name_)
, threads_metric(threads_metric_)
, threads_active_metric(threads_active_metric_)
{
}
void StaticThreadPool::initialize(size_t max_threads, size_t max_free_threads, size_t queue_size)
{ {
if (instance) if (instance)
{ throw Exception(ErrorCodes::LOGICAL_ERROR, "The {} is initialized twice", name);
throw Exception(ErrorCodes::LOGICAL_ERROR, "The IO thread pool is initialized twice");
}
/// By default enabling "turbo mode" won't affect the number of threads anyhow
max_threads_turbo = max_threads;
max_threads_normal = max_threads;
instance = std::make_unique<ThreadPool>( instance = std::make_unique<ThreadPool>(
CurrentMetrics::IOThreads, threads_metric,
CurrentMetrics::IOThreadsActive, threads_active_metric,
max_threads, max_threads,
max_free_threads, max_free_threads,
queue_size, queue_size,
/* shutdown_on_exception= */ false); /* shutdown_on_exception= */ false);
} }
ThreadPool & IOThreadPool::get() void StaticThreadPool::reloadConfiguration(size_t max_threads, size_t max_free_threads, size_t queue_size)
{ {
if (!instance) if (!instance)
{ throw Exception(ErrorCodes::LOGICAL_ERROR, "The {} is not initialized", name);
throw Exception(ErrorCodes::LOGICAL_ERROR, "The IO thread pool is not initialized");
} instance->setMaxThreads(turbo_mode_enabled > 0 ? max_threads_turbo : max_threads);
instance->setMaxFreeThreads(max_free_threads);
instance->setQueueSize(queue_size);
}
ThreadPool & StaticThreadPool::get()
{
if (!instance)
throw Exception(ErrorCodes::LOGICAL_ERROR, "The {} is not initialized", name);
return *instance; return *instance;
} }
std::unique_ptr<ThreadPool> BackupsIOThreadPool::instance; void StaticThreadPool::enableTurboMode()
void BackupsIOThreadPool::initialize(size_t max_threads, size_t max_free_threads, size_t queue_size)
{
if (instance)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "The BackupsIO thread pool is initialized twice");
}
instance = std::make_unique<ThreadPool>(
CurrentMetrics::BackupsIOThreads,
CurrentMetrics::BackupsIOThreadsActive,
max_threads,
max_free_threads,
queue_size,
/* shutdown_on_exception= */ false);
}
ThreadPool & BackupsIOThreadPool::get()
{ {
if (!instance) if (!instance)
{ throw Exception(ErrorCodes::LOGICAL_ERROR, "The {} is not initialized", name);
throw Exception(ErrorCodes::LOGICAL_ERROR, "The BackupsIO thread pool is not initialized");
}
return *instance; std::lock_guard lock(mutex);
++turbo_mode_enabled;
if (turbo_mode_enabled == 1)
instance->setMaxThreads(max_threads_turbo);
} }
std::unique_ptr<ThreadPool> OutdatedPartsLoadingThreadPool::instance; void StaticThreadPool::disableTurboMode()
void OutdatedPartsLoadingThreadPool::initialize(size_t max_threads, size_t max_free_threads, size_t queue_size)
{
if (instance)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "The PartsLoadingThreadPool thread pool is initialized twice");
}
instance = std::make_unique<ThreadPool>(
CurrentMetrics::OutdatedPartsLoadingThreads,
CurrentMetrics::OutdatedPartsLoadingThreadsActive,
max_threads,
max_free_threads,
queue_size,
/* shutdown_on_exception= */ false);
}
ThreadPool & OutdatedPartsLoadingThreadPool::get()
{ {
if (!instance) if (!instance)
{ throw Exception(ErrorCodes::LOGICAL_ERROR, "The {} is not initialized", name);
throw Exception(ErrorCodes::LOGICAL_ERROR, "The PartsLoadingThreadPool thread pool is not initialized");
}
return *instance; std::lock_guard lock(mutex);
--turbo_mode_enabled;
if (turbo_mode_enabled == 0)
instance->setMaxThreads(max_threads_normal);
}
void StaticThreadPool::setMaxTurboThreads(size_t max_threads_turbo_)
{
if (!instance)
throw Exception(ErrorCodes::LOGICAL_ERROR, "The {} is not initialized", name);
std::lock_guard lock(mutex);
max_threads_turbo = max_threads_turbo_;
if (turbo_mode_enabled > 0)
instance->setMaxThreads(max_threads_turbo);
}
StaticThreadPool & getIOThreadPool()
{
static StaticThreadPool instance("IOThreadPool", CurrentMetrics::IOThreads, CurrentMetrics::IOThreadsActive);
return instance;
}
StaticThreadPool & getBackupsIOThreadPool()
{
static StaticThreadPool instance("BackupsIOThreadPool", CurrentMetrics::BackupsIOThreads, CurrentMetrics::BackupsIOThreadsActive);
return instance;
}
StaticThreadPool & getActivePartsLoadingThreadPool()
{
static StaticThreadPool instance("MergeTreePartsLoaderThreadPool", CurrentMetrics::MergeTreePartsLoaderThreads, CurrentMetrics::MergeTreePartsLoaderThreadsActive);
return instance;
}
StaticThreadPool & getPartsCleaningThreadPool()
{
static StaticThreadPool instance("MergeTreePartsCleanerThreadPool", CurrentMetrics::MergeTreePartsCleanerThreads, CurrentMetrics::MergeTreePartsCleanerThreadsActive);
return instance;
}
StaticThreadPool & getOutdatedPartsLoadingThreadPool()
{
static StaticThreadPool instance("MergeTreeOutdatedPartsLoaderThreadPool", CurrentMetrics::MergeTreeOutdatedPartsLoaderThreads, CurrentMetrics::MergeTreeOutdatedPartsLoaderThreadsActive);
return instance;
} }
} }

View File

@ -1,48 +1,64 @@
#pragma once #pragma once
#include <base/types.h>
#include <Common/ThreadPool_fwd.h> #include <Common/ThreadPool_fwd.h>
#include <Common/CurrentMetrics.h>
#include <cstdlib> #include <cstdlib>
#include <memory> #include <memory>
#include <mutex>
namespace DB namespace DB
{ {
/* class StaticThreadPool
* ThreadPool used for the IO.
*/
class IOThreadPool
{ {
static std::unique_ptr<ThreadPool> instance;
public: public:
static void initialize(size_t max_threads, size_t max_free_threads, size_t queue_size); StaticThreadPool(
static ThreadPool & get(); const String & name_,
CurrentMetrics::Metric threads_metric_,
CurrentMetrics::Metric threads_active_metric_);
ThreadPool & get();
void initialize(size_t max_threads, size_t max_free_threads, size_t queue_size);
void reloadConfiguration(size_t max_threads, size_t max_free_threads, size_t queue_size);
/// At runtime we can increase the number of threads up the specified limit
/// This is needed to utilize as much a possible resources to accomplish some task.
void setMaxTurboThreads(size_t max_threads_turbo_);
void enableTurboMode();
void disableTurboMode();
private:
const String name;
const CurrentMetrics::Metric threads_metric;
const CurrentMetrics::Metric threads_active_metric;
std::unique_ptr<ThreadPool> instance;
std::mutex mutex;
size_t max_threads_turbo = 0;
size_t max_threads_normal = 0;
/// If this counter is > 0 - this specific mode is enabled
size_t turbo_mode_enabled = 0;
}; };
/// ThreadPool used for the IO.
StaticThreadPool & getIOThreadPool();
/* /// ThreadPool used for the Backup IO.
* ThreadPool used for the Backup IO. StaticThreadPool & getBackupsIOThreadPool();
*/
class BackupsIOThreadPool
{
static std::unique_ptr<ThreadPool> instance;
public: /// ThreadPool used for the loading of Outdated data parts for MergeTree tables.
static void initialize(size_t max_threads, size_t max_free_threads, size_t queue_size); StaticThreadPool & getActivePartsLoadingThreadPool();
static ThreadPool & get();
};
/// ThreadPool used for deleting data parts for MergeTree tables.
StaticThreadPool & getPartsCleaningThreadPool();
/* /// This ThreadPool is used for the loading of Outdated data parts for MergeTree tables.
* ThreadPool used for the loading of Outdated data parts for MergeTree tables. /// Normally we will just load Outdated data parts concurrently in background, but in
*/ /// case when we need to synchronously wait for the loading to be finished, we can increase
class OutdatedPartsLoadingThreadPool /// the number of threads by calling enableTurboMode() :-)
{ StaticThreadPool & getOutdatedPartsLoadingThreadPool();
static std::unique_ptr<ThreadPool> instance;
public:
static void initialize(size_t max_threads, size_t max_free_threads, size_t queue_size);
static ThreadPool & get();
};
} }

View File

@ -20,20 +20,35 @@ void formatHex(IteratorSrc src, IteratorDst dst, size_t num_bytes)
} }
} }
/** Function used when byte ordering is important when parsing uuid std::array<char, 36> formatUUID(const UUID & uuid)
* ex: When we create an UUID type
*/
void formatUUID(std::reverse_iterator<const UInt8 *> src16, UInt8 * dst36)
{ {
formatHex(src16 + 8, &dst36[0], 4); std::array<char, 36> dst;
dst36[8] = '-'; const auto * src_ptr = reinterpret_cast<const UInt8 *>(&uuid);
formatHex(src16 + 12, &dst36[9], 2); auto * dst_ptr = dst.data();
dst36[13] = '-'; #if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
formatHex(src16 + 14, &dst36[14], 2); const std::reverse_iterator src_it(src_ptr + 16);
dst36[18] = '-'; formatHex(src_it + 8, dst_ptr, 4);
formatHex(src16, &dst36[19], 2); dst[8] = '-';
dst36[23] = '-'; formatHex(src_it + 12, dst_ptr + 9, 2);
formatHex(src16 + 2, &dst36[24], 6); dst[13] = '-';
formatHex(src_it + 14, dst_ptr + 14, 2);
dst[18] = '-';
formatHex(src_it, dst_ptr + 19, 2);
dst[23] = '-';
formatHex(src_it + 2, dst_ptr + 24, 6);
#else
formatHex(src_ptr, dst_ptr, 4);
dst[8] = '-';
formatHex(src_ptr + 4, dst_ptr + 9, 2);
dst[13] = '-';
formatHex(src_ptr + 6, dst_ptr + 14, 2);
dst[18] = '-';
formatHex(src_ptr + 8, dst_ptr + 19, 2);
dst[23] = '-';
formatHex(src_ptr + 10, dst_ptr + 24, 6);
#endif
return dst;
} }
void writeIPv4Text(const IPv4 & ip, WriteBuffer & buf) void writeIPv4Text(const IPv4 & ip, WriteBuffer & buf)

View File

@ -625,13 +625,15 @@ inline void writeXMLStringForTextElement(std::string_view s, WriteBuffer & buf)
writeXMLStringForTextElement(s.data(), s.data() + s.size(), buf); writeXMLStringForTextElement(s.data(), s.data() + s.size(), buf);
} }
void formatUUID(std::reverse_iterator<const UInt8 *> src16, UInt8 * dst36); /// @brief Serialize `uuid` into an array of characters in big-endian byte order.
/// @param uuid UUID to serialize.
/// @return Array of characters in big-endian byte order.
std::array<char, 36> formatUUID(const UUID & uuid);
inline void writeUUIDText(const UUID & uuid, WriteBuffer & buf) inline void writeUUIDText(const UUID & uuid, WriteBuffer & buf)
{ {
char s[36]; const auto serialized_uuid = formatUUID(uuid);
formatUUID(std::reverse_iterator<const UInt8 *>(reinterpret_cast<const UInt8 *>(&uuid) + 16), reinterpret_cast<UInt8 *>(s)); buf.write(serialized_uuid.data(), serialized_uuid.size());
buf.write(s, sizeof(s));
} }
void writeIPv4Text(const IPv4 & ip, WriteBuffer & buf); void writeIPv4Text(const IPv4 & ip, WriteBuffer & buf);

View File

@ -149,7 +149,7 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment:
auto add_to_result = [&](const FileSegmentMetadata & file_segment_metadata) auto add_to_result = [&](const FileSegmentMetadata & file_segment_metadata)
{ {
FileSegmentPtr file_segment; FileSegmentPtr file_segment;
if (file_segment_metadata.valid()) if (!file_segment_metadata.evicting())
{ {
file_segment = file_segment_metadata.file_segment; file_segment = file_segment_metadata.file_segment;
if (file_segment->isDownloaded()) if (file_segment->isDownloaded())

View File

@ -85,7 +85,7 @@ public:
EMPTY, EMPTY,
/** /**
* A newly created file segment never has DOWNLOADING state until call to getOrSetDownloader * A newly created file segment never has DOWNLOADING state until call to getOrSetDownloader
* because each cache user might acquire multiple file segments and reads them one by one, * because each cache user might acquire multiple file segments and read them one by one,
* so only user which actually needs to read this segment earlier than others - becomes a downloader. * so only user which actually needs to read this segment earlier than others - becomes a downloader.
*/ */
DOWNLOADING, DOWNLOADING,

View File

@ -85,6 +85,7 @@ public:
virtual void removeAll(const CacheGuard::Lock &) = 0; virtual void removeAll(const CacheGuard::Lock &) = 0;
/// From lowest to highest priority.
virtual void iterate(IterateFunc && func, const CacheGuard::Lock &) = 0; virtual void iterate(IterateFunc && func, const CacheGuard::Lock &) = 0;
private: private:

View File

@ -346,6 +346,16 @@ void LockedKey::removeAllReleasable()
++it; ++it;
continue; continue;
} }
else if (it->second->evicting())
{
/// File segment is currently a removal candidate,
/// we do not know if it will be removed or not yet,
/// but its size is currently accounted as potentially removed,
/// so if we remove file segment now, we break the freeable_count
/// calculation in tryReserve.
++it;
continue;
}
auto file_segment = it->second->file_segment; auto file_segment = it->second->file_segment;
it = removeFileSegment(file_segment->offset(), file_segment->lock()); it = removeFileSegment(file_segment->offset(), file_segment->lock());

View File

@ -22,7 +22,7 @@ struct FileSegmentMetadata : private boost::noncopyable
size_t size() const; size_t size() const;
bool valid() const { return !removal_candidate.load(); } bool evicting() const { return removal_candidate.load(); }
Priority::Iterator getQueueIterator() const { return file_segment->getQueueIterator(); } Priority::Iterator getQueueIterator() const { return file_segment->getQueueIterator(); }

View File

@ -44,6 +44,9 @@ ThreadPoolCallbackRunner<Result, Callback> threadPoolCallbackRunner(ThreadPool &
auto future = task->get_future(); auto future = task->get_future();
/// ThreadPool is using "bigger is higher priority" instead of "smaller is more priority".
/// Note: calling method scheduleOrThrowOnError in intentional, because we don't want to throw exceptions
/// in critical places where this callback runner is used (e.g. loading or deletion of parts)
my_pool->scheduleOrThrowOnError([my_task = std::move(task)]{ (*my_task)(); }, priority); my_pool->scheduleOrThrowOnError([my_task = std::move(task)]{ (*my_task)(); }, priority);
return future; return future;

View File

@ -256,8 +256,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(const avro
if (tmp.length() != 36) if (tmp.length() != 36)
throw ParsingException(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", tmp); throw ParsingException(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", tmp);
UUID uuid; const UUID uuid = parseUUID({reinterpret_cast<const UInt8 *>(tmp.data()), tmp.length()});
parseUUID(reinterpret_cast<const UInt8 *>(tmp.data()), std::reverse_iterator<UInt8 *>(reinterpret_cast<UInt8 *>(&uuid) + 16));
assert_cast<DataTypeUUID::ColumnType &>(column).insertValue(uuid); assert_cast<DataTypeUUID::ColumnType &>(column).insertValue(uuid);
return true; return true;
}; };

View File

@ -329,9 +329,8 @@ AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeF
return {schema, [](const IColumn & column, size_t row_num, avro::Encoder & encoder) return {schema, [](const IColumn & column, size_t row_num, avro::Encoder & encoder)
{ {
const auto & uuid = assert_cast<const DataTypeUUID::ColumnType &>(column).getElement(row_num); const auto & uuid = assert_cast<const DataTypeUUID::ColumnType &>(column).getElement(row_num);
std::array<UInt8, 36> s; const auto serialized_uuid = formatUUID(uuid);
formatUUID(std::reverse_iterator<const UInt8 *>(reinterpret_cast<const UInt8 *>(&uuid) + 16), s.data()); encoder.encodeBytes(reinterpret_cast<const uint8_t *>(serialized_uuid.data()), serialized_uuid.size());
encoder.encodeBytes(reinterpret_cast<const uint8_t *>(s.data()), s.size());
}}; }};
} }
case TypeIndex::Array: case TypeIndex::Array:

View File

@ -202,6 +202,13 @@ bool DataPartStorageOnDiskBase::isStoredOnRemoteDisk() const
return volume->getDisk()->isRemote(); return volume->getDisk()->isRemote();
} }
std::optional<String> DataPartStorageOnDiskBase::getCacheName() const
{
if (volume->getDisk()->supportsCache())
return volume->getDisk()->getCacheName();
return std::nullopt;
}
bool DataPartStorageOnDiskBase::supportZeroCopyReplication() const bool DataPartStorageOnDiskBase::supportZeroCopyReplication() const
{ {
return volume->getDisk()->supportZeroCopyReplication(); return volume->getDisk()->supportZeroCopyReplication();

View File

@ -36,6 +36,7 @@ public:
std::string getDiskName() const override; std::string getDiskName() const override;
std::string getDiskType() const override; std::string getDiskType() const override;
bool isStoredOnRemoteDisk() const override; bool isStoredOnRemoteDisk() const override;
std::optional<String> getCacheName() const override;
bool supportZeroCopyReplication() const override; bool supportZeroCopyReplication() const override;
bool supportParallelWrite() const override; bool supportParallelWrite() const override;
bool isBroken() const override; bool isBroken() const override;

View File

@ -149,6 +149,7 @@ public:
virtual std::string getDiskName() const = 0; virtual std::string getDiskName() const = 0;
virtual std::string getDiskType() const = 0; virtual std::string getDiskType() const = 0;
virtual bool isStoredOnRemoteDisk() const { return false; } virtual bool isStoredOnRemoteDisk() const { return false; }
virtual std::optional<String> getCacheName() const { return std::nullopt; }
virtual bool supportZeroCopyReplication() const { return false; } virtual bool supportZeroCopyReplication() const { return false; }
virtual bool supportParallelWrite() const = 0; virtual bool supportParallelWrite() const = 0;
virtual bool isBroken() const = 0; virtual bool isBroken() const = 0;

View File

@ -130,10 +130,6 @@ namespace ProfileEvents
namespace CurrentMetrics namespace CurrentMetrics
{ {
extern const Metric DelayedInserts; extern const Metric DelayedInserts;
extern const Metric MergeTreePartsLoaderThreads;
extern const Metric MergeTreePartsLoaderThreadsActive;
extern const Metric MergeTreePartsCleanerThreads;
extern const Metric MergeTreePartsCleanerThreadsActive;
} }
@ -1425,71 +1421,17 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPartWithRetries(
UNREACHABLE(); UNREACHABLE();
} }
std::vector<MergeTreeData::LoadPartResult> MergeTreeData::loadDataPartsFromDisk( std::vector<MergeTreeData::LoadPartResult> MergeTreeData::loadDataPartsFromDisk(PartLoadingTreeNodes & parts_to_load)
ThreadPool & pool,
size_t num_parts,
std::queue<PartLoadingTreeNodes> & parts_queue,
const MergeTreeSettingsPtr & settings)
{ {
/// Parallel loading of data parts. const size_t num_parts = parts_to_load.size();
pool.setMaxThreads(std::min(static_cast<size_t>(settings->max_part_loading_threads), num_parts));
size_t num_threads = pool.getMaxThreads();
LOG_DEBUG(log, "Going to use {} threads to load parts", num_threads);
std::vector<size_t> parts_per_thread(num_threads, num_parts / num_threads); LOG_DEBUG(log, "Will load {} number of parts using {} threads", num_parts, getActivePartsLoadingThreadPool().get().getMaxThreads());
for (size_t i = 0ul; i < num_parts % num_threads; ++i)
++parts_per_thread[i];
/// Prepare data parts for parallel loading. Threads will focus on given disk first, then steal /// Shuffle all the parts randomly to possible speed up loading them from JBOD.
/// others' tasks when finish current disk part loading process. std::shuffle(parts_to_load.begin(), parts_to_load.end(), thread_local_rng);
std::vector<PartLoadingTreeNodes> threads_parts(num_threads);
std::set<size_t> remaining_thread_parts;
std::queue<size_t> threads_queue;
for (size_t i = 0; i < num_threads; ++i) auto runner = threadPoolCallbackRunner<void>(getActivePartsLoadingThreadPool().get(), "ActiveParts");
{ std::vector<std::future<void>> parts_futures;
remaining_thread_parts.insert(i);
threads_queue.push(i);
}
while (!parts_queue.empty())
{
assert(!threads_queue.empty());
size_t i = threads_queue.front();
auto & need_parts = parts_per_thread[i];
assert(need_parts > 0);
auto & thread_parts = threads_parts[i];
auto & current_parts = parts_queue.front();
assert(!current_parts.empty());
auto parts_to_grab = std::min(need_parts, current_parts.size());
thread_parts.insert(thread_parts.end(), current_parts.end() - parts_to_grab, current_parts.end());
current_parts.resize(current_parts.size() - parts_to_grab);
need_parts -= parts_to_grab;
/// Before processing next thread, change disk if possible.
/// Different threads will likely start loading parts from different disk,
/// which may improve read parallelism for JBOD.
/// If current disk still has some parts, push it to the tail.
if (!current_parts.empty())
parts_queue.push(std::move(current_parts));
parts_queue.pop();
/// If current thread still want some parts, push it to the tail.
if (need_parts > 0)
threads_queue.push(i);
threads_queue.pop();
}
assert(threads_queue.empty());
assert(std::all_of(threads_parts.begin(), threads_parts.end(), [](const auto & parts)
{
return !parts.empty();
}));
std::mutex part_select_mutex; std::mutex part_select_mutex;
std::mutex part_loading_mutex; std::mutex part_loading_mutex;
@ -1498,81 +1440,77 @@ std::vector<MergeTreeData::LoadPartResult> MergeTreeData::loadDataPartsFromDisk(
try try
{ {
for (size_t thread = 0; thread < num_threads; ++thread) while (true)
{ {
pool.scheduleOrThrowOnError([&, thread, thread_group = CurrentThread::getGroup()] bool are_parts_to_load_empty = false;
{ {
SCOPE_EXIT_SAFE( std::lock_guard lock(part_select_mutex);
if (thread_group) are_parts_to_load_empty = parts_to_load.empty();
CurrentThread::detachFromGroupIfNotDetached(); }
);
if (thread_group)
CurrentThread::attachToGroupIfDetached(thread_group);
while (true) if (are_parts_to_load_empty)
{
/// Wait for all scheduled tasks.
/// We have to use .get() method to rethrow any exception that could occur.
for (auto & future: parts_futures)
future.get();
parts_futures.clear();
/// At this point it is possible, that some other parts appeared in the queue for processing (parts_to_load),
/// because we added them from inside the pool.
/// So we need to recheck it.
}
PartLoadingTree::NodePtr current_part;
{
std::lock_guard lock(part_select_mutex);
if (parts_to_load.empty())
break;
current_part = parts_to_load.back();
parts_to_load.pop_back();
}
parts_futures.push_back(runner(
[&, part = std::move(current_part)]()
{ {
PartLoadingTree::NodePtr thread_part;
size_t thread_idx = thread;
{
std::lock_guard lock{part_select_mutex};
if (remaining_thread_parts.empty())
return;
/// Steal task if nothing to do
if (threads_parts[thread].empty())
{
// Try random steal tasks from the next thread
std::uniform_int_distribution<size_t> distribution(0, remaining_thread_parts.size() - 1);
auto it = remaining_thread_parts.begin();
std::advance(it, distribution(thread_local_rng));
thread_idx = *it;
}
auto & thread_parts = threads_parts[thread_idx];
thread_part = thread_parts.back();
thread_parts.pop_back();
if (thread_parts.empty())
remaining_thread_parts.erase(thread_idx);
}
/// Pass a separate mutex to guard the set of parts, because this lambda /// Pass a separate mutex to guard the set of parts, because this lambda
/// is called concurrently but with already locked @data_parts_mutex. /// is called concurrently but with already locked @data_parts_mutex.
auto res = loadDataPartWithRetries( auto res = loadDataPartWithRetries(
thread_part->info, thread_part->name, thread_part->disk, part->info, part->name, part->disk,
DataPartState::Active, part_loading_mutex, loading_parts_initial_backoff_ms, DataPartState::Active, part_loading_mutex, loading_parts_initial_backoff_ms,
loading_parts_max_backoff_ms, loading_parts_max_tries); loading_parts_max_backoff_ms, loading_parts_max_tries);
thread_part->is_loaded = true; part->is_loaded = true;
bool is_active_part = res.part->getState() == DataPartState::Active; bool is_active_part = res.part->getState() == DataPartState::Active;
/// If part is broken or duplicate or should be removed according to transaction /// If part is broken or duplicate or should be removed according to transaction
/// and it has any covered parts then try to load them to replace this part. /// and it has any covered parts then try to load them to replace this part.
if (!is_active_part && !thread_part->children.empty()) if (!is_active_part && !part->children.empty())
{ {
std::lock_guard lock{part_select_mutex}; std::lock_guard lock{part_select_mutex};
for (const auto & [_, node] : thread_part->children) for (const auto & [_, node] : part->children)
threads_parts[thread].push_back(node); parts_to_load.push_back(node);
remaining_thread_parts.insert(thread);
} }
{ {
std::lock_guard lock(part_loading_mutex); std::lock_guard lock(part_loading_mutex);
loaded_parts.push_back(std::move(res)); loaded_parts.push_back(std::move(res));
} }
} }, Priority{0}));
});
} }
} }
catch (...) catch (...)
{ {
/// If this is not done, then in case of an exception, tasks will be destroyed before the threads are completed, and it will be bad. /// Wait for all scheduled tasks
pool.wait(); /// A future becomes invalid after .get() call
/// + .wait() method is used not to throw any exception here.
for (auto & future: parts_futures)
if (future.valid())
future.wait();
throw; throw;
} }
pool.wait();
return loaded_parts; return loaded_parts;
} }
@ -1679,9 +1617,12 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
} }
} }
ThreadPool pool(CurrentMetrics::MergeTreePartsLoaderThreads, CurrentMetrics::MergeTreePartsLoaderThreadsActive, disks.size()); auto runner = threadPoolCallbackRunner<void>(getActivePartsLoadingThreadPool().get(), "ActiveParts");
std::vector<PartLoadingTree::PartLoadingInfos> parts_to_load_by_disk(disks.size()); std::vector<PartLoadingTree::PartLoadingInfos> parts_to_load_by_disk(disks.size());
std::vector<std::future<void>> disks_futures;
disks_futures.reserve(disks.size());
for (size_t i = 0; i < disks.size(); ++i) for (size_t i = 0; i < disks.size(); ++i)
{ {
const auto & disk_ptr = disks[i]; const auto & disk_ptr = disks[i];
@ -1690,7 +1631,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
auto & disk_parts = parts_to_load_by_disk[i]; auto & disk_parts = parts_to_load_by_disk[i];
pool.scheduleOrThrowOnError([&, disk_ptr]() disks_futures.push_back(runner([&, disk_ptr]()
{ {
for (auto it = disk_ptr->iterateDirectory(relative_data_path); it->isValid(); it->next()) for (auto it = disk_ptr->iterateDirectory(relative_data_path); it->isValid(); it->next())
{ {
@ -1703,38 +1644,31 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
if (auto part_info = MergeTreePartInfo::tryParsePartName(it->name(), format_version)) if (auto part_info = MergeTreePartInfo::tryParsePartName(it->name(), format_version))
disk_parts.emplace_back(*part_info, it->name(), disk_ptr); disk_parts.emplace_back(*part_info, it->name(), disk_ptr);
} }
}); }, Priority{0}));
} }
pool.wait(); /// For iteration to be completed
/// Any exception will be re-thrown.
for (auto & future : disks_futures)
future.get();
disks_futures.clear();
PartLoadingTree::PartLoadingInfos parts_to_load; PartLoadingTree::PartLoadingInfos parts_to_load;
for (auto & disk_parts : parts_to_load_by_disk) for (auto & disk_parts : parts_to_load_by_disk)
std::move(disk_parts.begin(), disk_parts.end(), std::back_inserter(parts_to_load)); std::move(disk_parts.begin(), disk_parts.end(), std::back_inserter(parts_to_load));
auto loading_tree = PartLoadingTree::build(std::move(parts_to_load)); auto loading_tree = PartLoadingTree::build(std::move(parts_to_load));
/// Collect parts by disks' names.
std::map<String, PartLoadingTreeNodes> disk_part_map; size_t num_parts = 0;
PartLoadingTreeNodes active_parts;
/// Collect only "the most covering" parts from the top level of the tree. /// Collect only "the most covering" parts from the top level of the tree.
loading_tree.traverse(/*recursive=*/ false, [&](const auto & node) loading_tree.traverse(/*recursive=*/ false, [&](const auto & node)
{ {
disk_part_map[node->disk->getName()].emplace_back(node); active_parts.emplace_back(node);
}); });
size_t num_parts = 0; num_parts += active_parts.size();
std::queue<PartLoadingTreeNodes> parts_queue;
for (auto & [disk_name, disk_parts] : disk_part_map)
{
LOG_INFO(log, "Found {} parts for disk '{}' to load", disk_parts.size(), disk_name);
if (disk_parts.empty())
continue;
num_parts += disk_parts.size();
parts_queue.push(std::move(disk_parts));
}
auto part_lock = lockParts(); auto part_lock = lockParts();
LOG_TEST(log, "loadDataParts: clearing data_parts_indexes (had {} parts)", data_parts_indexes.size()); LOG_TEST(log, "loadDataParts: clearing data_parts_indexes (had {} parts)", data_parts_indexes.size());
@ -1754,7 +1688,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
if (num_parts > 0) if (num_parts > 0)
{ {
auto loaded_parts = loadDataPartsFromDisk(pool, num_parts, parts_queue, settings); auto loaded_parts = loadDataPartsFromDisk(active_parts);
for (const auto & res : loaded_parts) for (const auto & res : loaded_parts)
{ {
@ -1783,10 +1717,12 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
if (settings->in_memory_parts_enable_wal) if (settings->in_memory_parts_enable_wal)
{ {
pool.setMaxThreads(disks.size());
std::vector<MutableDataPartsVector> disks_wal_parts(disks.size()); std::vector<MutableDataPartsVector> disks_wal_parts(disks.size());
std::mutex wal_init_lock; std::mutex wal_init_lock;
std::vector<std::future<void>> wal_disks_futures;
wal_disks_futures.reserve(disks.size());
for (size_t i = 0; i < disks.size(); ++i) for (size_t i = 0; i < disks.size(); ++i)
{ {
const auto & disk_ptr = disks[i]; const auto & disk_ptr = disks[i];
@ -1795,7 +1731,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
auto & disk_wal_parts = disks_wal_parts[i]; auto & disk_wal_parts = disks_wal_parts[i];
pool.scheduleOrThrowOnError([&, disk_ptr]() wal_disks_futures.push_back(runner([&, disk_ptr]()
{ {
for (auto it = disk_ptr->iterateDirectory(relative_data_path); it->isValid(); it->next()) for (auto it = disk_ptr->iterateDirectory(relative_data_path); it->isValid(); it->next())
{ {
@ -1821,10 +1757,14 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
disk_wal_parts.push_back(std::move(part)); disk_wal_parts.push_back(std::move(part));
} }
} }
}); }, Priority{0}));
} }
pool.wait(); /// For for iteration to be completed
/// Any exception will be re-thrown.
for (auto & future : wal_disks_futures)
future.get();
wal_disks_futures.clear();
MutableDataPartsVector parts_from_wal; MutableDataPartsVector parts_from_wal;
for (auto & disk_wal_parts : disks_wal_parts) for (auto & disk_wal_parts : disks_wal_parts)
@ -1925,7 +1865,7 @@ try
std::atomic_size_t num_loaded_parts = 0; std::atomic_size_t num_loaded_parts = 0;
auto runner = threadPoolCallbackRunner<void>(OutdatedPartsLoadingThreadPool::get(), "OutdatedParts"); auto runner = threadPoolCallbackRunner<void>(getOutdatedPartsLoadingThreadPool().get(), "OutdatedParts");
std::vector<std::future<void>> parts_futures; std::vector<std::future<void>> parts_futures;
while (true) while (true)
@ -1938,8 +1878,10 @@ try
if (is_async && outdated_data_parts_loading_canceled) if (is_async && outdated_data_parts_loading_canceled)
{ {
/// Wait for every scheduled task /// Wait for every scheduled task
/// In case of any exception it will be re-thrown and server will be terminated.
for (auto & future : parts_futures) for (auto & future : parts_futures)
future.wait(); future.get();
parts_futures.clear();
LOG_DEBUG(log, LOG_DEBUG(log,
"Stopped loading outdated data parts because task was canceled. " "Stopped loading outdated data parts because task was canceled. "
@ -1973,7 +1915,7 @@ try
/// Wait for every scheduled task /// Wait for every scheduled task
for (auto & future : parts_futures) for (auto & future : parts_futures)
future.wait(); future.get();
LOG_DEBUG(log, "Loaded {} outdated data parts {}", LOG_DEBUG(log, "Loaded {} outdated data parts {}",
num_loaded_parts, is_async ? "asynchronously" : "synchronously"); num_loaded_parts, is_async ? "asynchronously" : "synchronously");
@ -1999,6 +1941,13 @@ void MergeTreeData::waitForOutdatedPartsToBeLoaded() const TSA_NO_THREAD_SAFETY_
if (isStaticStorage()) if (isStaticStorage())
return; return;
/// We need to load parts as fast as possible
getOutdatedPartsLoadingThreadPool().enableTurboMode();
SCOPE_EXIT({
/// Let's lower the number of threads e.g. for later ATTACH queries to behave as usual
getOutdatedPartsLoadingThreadPool().disableTurboMode();
});
LOG_TRACE(log, "Will wait for outdated data parts to be loaded"); LOG_TRACE(log, "Will wait for outdated data parts to be loaded");
std::unique_lock lock(outdated_data_parts_mutex); std::unique_lock lock(outdated_data_parts_mutex);
@ -2420,20 +2369,15 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t
} }
}; };
if (settings->max_part_removal_threads <= 1 || parts_to_remove.size() <= settings->concurrent_part_removal_threshold) if (parts_to_remove.size() <= settings->concurrent_part_removal_threshold)
{ {
remove_single_thread(); remove_single_thread();
return; return;
} }
/// Parallel parts removal. /// Parallel parts removal.
size_t num_threads = settings->max_part_removal_threads;
if (!num_threads)
num_threads = getNumberOfPhysicalCPUCores() * 2;
num_threads = std::min<size_t>(num_threads, parts_to_remove.size());
std::mutex part_names_mutex; std::mutex part_names_mutex;
ThreadPool pool(CurrentMetrics::MergeTreePartsCleanerThreads, CurrentMetrics::MergeTreePartsCleanerThreadsActive, auto runner = threadPoolCallbackRunner<void>(getPartsCleaningThreadPool().get(), "PartsCleaning");
num_threads, num_threads, /* unlimited queue size */ 0);
/// This flag disallow straightforward concurrent parts removal. It's required only in case /// This flag disallow straightforward concurrent parts removal. It's required only in case
/// when we have parts on zero-copy disk + at least some of them were mutated. /// when we have parts on zero-copy disk + at least some of them were mutated.
@ -2453,27 +2397,27 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t
LOG_DEBUG( LOG_DEBUG(
log, "Removing {} parts from filesystem (concurrently): Parts: [{}]", parts_to_remove.size(), fmt::join(parts_to_remove, ", ")); log, "Removing {} parts from filesystem (concurrently): Parts: [{}]", parts_to_remove.size(), fmt::join(parts_to_remove, ", "));
std::vector<std::future<void>> parts_to_remove_futures;
parts_to_remove_futures.reserve(parts_to_remove.size());
for (const DataPartPtr & part : parts_to_remove) for (const DataPartPtr & part : parts_to_remove)
{ {
pool.scheduleOrThrowOnError([&part, &part_names_mutex, part_names_succeed, thread_group = CurrentThread::getGroup()] parts_to_remove_futures.push_back(runner([&part, &part_names_mutex, part_names_succeed, thread_group = CurrentThread::getGroup()]
{ {
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachFromGroupIfNotDetached();
);
if (thread_group)
CurrentThread::attachToGroupIfDetached(thread_group);
asMutableDeletingPart(part)->remove(); asMutableDeletingPart(part)->remove();
if (part_names_succeed) if (part_names_succeed)
{ {
std::lock_guard lock(part_names_mutex); std::lock_guard lock(part_names_mutex);
part_names_succeed->insert(part->name); part_names_succeed->insert(part->name);
} }
}); }, Priority{0}));
} }
pool.wait(); /// Any exception will be re-thrown.
for (auto & future : parts_to_remove_futures)
future.get();
parts_to_remove_futures.clear();
return; return;
} }
@ -2544,20 +2488,15 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t
return independent_ranges; return independent_ranges;
}; };
auto schedule_parts_removal = [this, &pool, &part_names_mutex, part_names_succeed]( std::vector<std::future<void>> part_removal_futures;
auto schedule_parts_removal = [this, &runner, &part_names_mutex, part_names_succeed, &part_removal_futures](
const MergeTreePartInfo & range, DataPartsVector && parts_in_range) const MergeTreePartInfo & range, DataPartsVector && parts_in_range)
{ {
/// Below, range should be captured by copy to avoid use-after-scope on exception from pool /// Below, range should be captured by copy to avoid use-after-scope on exception from pool
pool.scheduleOrThrowOnError( part_removal_futures.push_back(runner(
[this, range, &part_names_mutex, part_names_succeed, thread_group = CurrentThread::getGroup(), batch = std::move(parts_in_range)] [this, range, &part_names_mutex, part_names_succeed, batch = std::move(parts_in_range)]
{ {
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachFromGroupIfNotDetached();
);
if (thread_group)
CurrentThread::attachToGroupIfDetached(thread_group);
LOG_TRACE(log, "Removing {} parts in blocks range {}", batch.size(), range.getPartNameForLogs()); LOG_TRACE(log, "Removing {} parts in blocks range {}", batch.size(), range.getPartNameForLogs());
for (const auto & part : batch) for (const auto & part : batch)
@ -2569,7 +2508,7 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t
part_names_succeed->insert(part->name); part_names_succeed->insert(part->name);
} }
} }
}); }, Priority{0}));
}; };
RemovalRanges independent_ranges = split_into_independent_ranges(parts_to_remove, /* split_times */ 0); RemovalRanges independent_ranges = split_into_independent_ranges(parts_to_remove, /* split_times */ 0);
@ -2632,7 +2571,11 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t
LOG_TRACE(log, "Will remove {} big parts separately: {}", excluded_parts.size(), fmt::join(excluded_parts, ", ")); LOG_TRACE(log, "Will remove {} big parts separately: {}", excluded_parts.size(), fmt::join(excluded_parts, ", "));
independent_ranges = split_into_independent_ranges(excluded_parts, /* split_times */ 0); independent_ranges = split_into_independent_ranges(excluded_parts, /* split_times */ 0);
pool.wait();
/// Any exception will be re-thrown.
for (auto & future : part_removal_futures)
future.get();
part_removal_futures.clear();
for (size_t i = 0; i < independent_ranges.infos.size(); ++i) for (size_t i = 0; i < independent_ranges.infos.size(); ++i)
{ {
@ -2641,7 +2584,10 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t
schedule_parts_removal(range, std::move(parts_in_range)); schedule_parts_removal(range, std::move(parts_in_range));
} }
pool.wait(); /// Any exception will be re-thrown.
for (auto & future : part_removal_futures)
future.get();
part_removal_futures.clear();
if (parts_to_remove.size() != sum_of_ranges + excluded_parts.size()) if (parts_to_remove.size() != sum_of_ranges + excluded_parts.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::LOGICAL_ERROR,

View File

@ -1519,11 +1519,7 @@ private:
size_t max_backoff_ms, size_t max_backoff_ms,
size_t max_tries); size_t max_tries);
std::vector<LoadPartResult> loadDataPartsFromDisk( std::vector<LoadPartResult> loadDataPartsFromDisk(PartLoadingTreeNodes & parts_to_load);
ThreadPool & pool,
size_t num_parts,
std::queue<PartLoadingTreeNodes> & parts_queue,
const MergeTreeSettingsPtr & settings);
void loadDataPartsFromWAL(MutableDataPartsVector & parts_from_wal); void loadDataPartsFromWAL(MutableDataPartsVector & parts_from_wal);

View File

@ -143,8 +143,6 @@ struct Settings;
M(Bool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \ M(Bool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \
M(Bool, materialize_ttl_recalculate_only, false, "Only recalculate ttl info when MATERIALIZE TTL", 0) \ M(Bool, materialize_ttl_recalculate_only, false, "Only recalculate ttl info when MATERIALIZE TTL", 0) \
M(Bool, enable_mixed_granularity_parts, true, "Enable parts with adaptive and non adaptive granularity", 0) \ M(Bool, enable_mixed_granularity_parts, true, "Enable parts with adaptive and non adaptive granularity", 0) \
M(MaxThreads, max_part_loading_threads, 0, "The number of threads to load data parts at startup.", 0) \
M(MaxThreads, max_part_removal_threads, 0, "The number of threads for concurrent removal of inactive data parts. One is usually enough, but in 'Google Compute Environment SSD Persistent Disks' file removal (unlink) operation is extraordinarily slow and you probably have to increase this number (recommended is up to 16).", 0) \
M(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \ M(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \
M(UInt64, zero_copy_concurrent_part_removal_max_split_times, 5, "Max recursion depth for splitting independent Outdated parts ranges into smaller subranges (highly not recommended to change)", 0) \ M(UInt64, zero_copy_concurrent_part_removal_max_split_times, 5, "Max recursion depth for splitting independent Outdated parts ranges into smaller subranges (highly not recommended to change)", 0) \
M(Float, zero_copy_concurrent_part_removal_max_postpone_ratio, static_cast<Float32>(0.05), "Max percentage of top level parts to postpone removal in order to get smaller independent ranges (highly not recommended to change)", 0) \ M(Float, zero_copy_concurrent_part_removal_max_postpone_ratio, static_cast<Float32>(0.05), "Max percentage of top level parts to postpone removal in order to get smaller independent ranges (highly not recommended to change)", 0) \
@ -192,6 +190,9 @@ struct Settings;
M(UInt64, write_ahead_log_bytes_to_fsync, 100ULL * 1024 * 1024, "Obsolete setting, does nothing.", 0) \ M(UInt64, write_ahead_log_bytes_to_fsync, 100ULL * 1024 * 1024, "Obsolete setting, does nothing.", 0) \
M(UInt64, write_ahead_log_interval_ms_to_fsync, 100, "Obsolete setting, does nothing.", 0) \ M(UInt64, write_ahead_log_interval_ms_to_fsync, 100, "Obsolete setting, does nothing.", 0) \
M(Bool, in_memory_parts_insert_sync, false, "Obsolete setting, does nothing.", 0) \ M(Bool, in_memory_parts_insert_sync, false, "Obsolete setting, does nothing.", 0) \
M(MaxThreads, max_part_loading_threads, 0, "Obsolete setting, does nothing.", 0) \
M(MaxThreads, max_part_removal_threads, 0, "Obsolete setting, does nothing.", 0) \
/// Settings that should not change after the creation of a table. /// Settings that should not change after the creation of a table.
/// NOLINTNEXTLINE /// NOLINTNEXTLINE
#define APPLY_FOR_IMMUTABLE_MERGE_TREE_SETTINGS(M) \ #define APPLY_FOR_IMMUTABLE_MERGE_TREE_SETTINGS(M) \

View File

@ -105,7 +105,7 @@ struct MergeTreeSource::AsyncReadingState
AsyncReadingState() AsyncReadingState()
{ {
control = std::make_shared<Control>(); control = std::make_shared<Control>();
callback_runner = threadPoolCallbackRunner<void>(IOThreadPool::get(), "MergeTreeRead"); callback_runner = threadPoolCallbackRunner<void>(getIOThreadPool().get(), "MergeTreeRead");
} }
~AsyncReadingState() ~AsyncReadingState()

View File

@ -766,7 +766,7 @@ public:
DBMS_DEFAULT_BUFFER_SIZE, DBMS_DEFAULT_BUFFER_SIZE,
configuration_.request_settings, configuration_.request_settings,
std::nullopt, std::nullopt,
threadPoolCallbackRunner<void>(IOThreadPool::get(), "S3ParallelWrite"), threadPoolCallbackRunner<void>(getIOThreadPool().get(), "S3ParallelWrite"),
context->getWriteSettings()), context->getWriteSettings()),
compression_method, compression_method,
3); 3);

View File

@ -194,7 +194,7 @@ private:
futures.push_back( futures.push_back(
scheduleFromThreadPool<void>( scheduleFromThreadPool<void>(
std::move(worker), std::move(worker),
IOThreadPool::get(), getIOThreadPool().get(),
"DP_BytesOnDisk")); "DP_BytesOnDisk"));
} }

View File

@ -0,0 +1,57 @@
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>
#include <Core/Settings.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Context.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/ProfileEventsExt.h>
#include <Storages/System/StorageSystemUserProcesses.h>
namespace DB
{
NamesAndTypesList StorageSystemUserProcesses::getNamesAndTypes()
{
return {
{"user", std::make_shared<DataTypeString>()},
{"memory_usage", std::make_shared<DataTypeInt64>()},
{"peak_memory_usage", std::make_shared<DataTypeInt64>()},
{"ProfileEvents", std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), std::make_shared<DataTypeUInt64>())},
};
}
NamesAndAliases StorageSystemUserProcesses::getNamesAndAliases()
{
return {
{"ProfileEvents.Names", {std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())}, "mapKeys(ProfileEvents)"},
{"ProfileEvents.Values", {std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>())}, "mapValues(ProfileEvents)"}};
}
void StorageSystemUserProcesses::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
const auto user_info = context->getProcessList().getUserInfo(true);
for (const auto & [user, info] : user_info)
{
size_t i = 0;
res_columns[i++]->insert(user);
res_columns[i++]->insert(info.memory_usage);
res_columns[i++]->insert(info.peak_memory_usage);
{
IColumn * column = res_columns[i++].get();
if (info.profile_counters)
ProfileEvents::dumpToMapColumn(*info.profile_counters, column, true);
else
{
column->insertDefault();
}
}
}
}
}

View File

@ -0,0 +1,29 @@
#pragma once
#include <Storages/System/IStorageSystemOneBlock.h>
namespace DB
{
class Context;
/** Implements `processes` system table, which allows you to get information about the queries that are currently executing.
*/
class StorageSystemUserProcesses final : public IStorageSystemOneBlock<StorageSystemUserProcesses>
{
public:
std::string getName() const override { return "SystemUserProcesses"; }
static NamesAndTypesList getNamesAndTypes();
static NamesAndAliases getNamesAndAliases();
protected:
using IStorageSystemOneBlock::IStorageSystemOneBlock;
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override;
};
}

View File

@ -36,6 +36,7 @@
#include <Storages/System/StorageSystemPartsColumns.h> #include <Storages/System/StorageSystemPartsColumns.h>
#include <Storages/System/StorageSystemProjectionPartsColumns.h> #include <Storages/System/StorageSystemProjectionPartsColumns.h>
#include <Storages/System/StorageSystemProcesses.h> #include <Storages/System/StorageSystemProcesses.h>
#include <Storages/System/StorageSystemUserProcesses.h>
#include <Storages/System/StorageSystemReplicas.h> #include <Storages/System/StorageSystemReplicas.h>
#include <Storages/System/StorageSystemReplicationQueue.h> #include <Storages/System/StorageSystemReplicationQueue.h>
#include <Storages/System/StorageSystemDistributionQueue.h> #include <Storages/System/StorageSystemDistributionQueue.h>
@ -185,6 +186,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b
attach<StorageSystemRemoteDataPaths>(context, system_database, "remote_data_paths"); attach<StorageSystemRemoteDataPaths>(context, system_database, "remote_data_paths");
attach<StorageSystemCertificates>(context, system_database, "certificates"); attach<StorageSystemCertificates>(context, system_database, "certificates");
attach<StorageSystemNamedCollections>(context, system_database, "named_collections"); attach<StorageSystemNamedCollections>(context, system_database, "named_collections");
attach<StorageSystemUserProcesses>(context, system_database, "user_processes");
if (has_zookeeper) if (has_zookeeper)
{ {

View File

@ -2,30 +2,27 @@
"""The lambda to decrease/increase ASG desired capacity based on current queue""" """The lambda to decrease/increase ASG desired capacity based on current queue"""
import json
import logging import logging
import time
from dataclasses import dataclass from dataclasses import dataclass
from pprint import pformat from pprint import pformat
from typing import Any, List, Literal, Optional, Tuple from typing import Any, List, Literal, Optional, Tuple
import boto3 # type: ignore import boto3 # type: ignore
import requests # type: ignore
RUNNER_TYPE_LABELS = [ from lambda_shared import (
"builder", CHException,
"func-tester", ClickHouseHelper,
"func-tester-aarch64", RUNNER_TYPE_LABELS,
"fuzzer-unit-tester", get_parameter_from_ssm,
"stress-tester", )
"style-checker",
"style-checker-aarch64",
]
### Update comment on the change ###
# 4 HOUR - is a balance to get the most precise values # 4 HOUR - is a balance to get the most precise values
# - Our longest possible running check is around 5h on the worst scenario # - Our longest possible running check is around 5h on the worst scenario
# - The long queue won't be wiped out and replaced, so the measurmenet is fine # - The long queue won't be wiped out and replaced, so the measurmenet is fine
# - If the data is spoiled by something, we are from the bills perspective # - If the data is spoiled by something, we are from the bills perspective
# Changed it to 3 HOUR: in average we have 1h tasks, but p90 is around 2h.
# With 4h we have too much wasted computing time in case of issues with DB
QUEUE_QUERY = f"""SELECT QUEUE_QUERY = f"""SELECT
last_status AS status, last_status AS status,
toUInt32(count()) AS length, toUInt32(count()) AS length,
@ -40,7 +37,7 @@ FROM
FROM default.workflow_jobs FROM default.workflow_jobs
WHERE has(labels, 'self-hosted') WHERE has(labels, 'self-hosted')
AND hasAny({RUNNER_TYPE_LABELS}, labels) AND hasAny({RUNNER_TYPE_LABELS}, labels)
AND started_at > now() - INTERVAL 4 HOUR AND started_at > now() - INTERVAL 3 HOUR
GROUP BY ALL GROUP BY ALL
HAVING last_status IN ('in_progress', 'queued') HAVING last_status IN ('in_progress', 'queued')
) )
@ -68,64 +65,14 @@ def get_scales(runner_type: str) -> Tuple[int, int]:
# 10. I am trying 7 now. # 10. I am trying 7 now.
# UPDATE THE COMMENT ON CHANGES # UPDATE THE COMMENT ON CHANGES
scale_up = 7 scale_up = 7
elif runner_type == "limited-tester":
# The limited runners should inflate and deflate faster
scale_down = 1
scale_up = 2
return scale_down, scale_up return scale_down, scale_up
### VENDORING CH_CLIENT = None # type: Optional[ClickHouseHelper]
def get_parameter_from_ssm(name, decrypt=True, client=None):
if not client:
client = boto3.client("ssm", region_name="us-east-1")
return client.get_parameter(Name=name, WithDecryption=decrypt)["Parameter"]["Value"]
class CHException(Exception):
pass
class ClickHouseHelper:
def __init__(
self,
url: Optional[str] = None,
user: Optional[str] = None,
password: Optional[str] = None,
):
self.url = url
self.auth = {}
if user:
self.auth["X-ClickHouse-User"] = user
if password:
self.auth["X-ClickHouse-Key"] = password
def _select_and_get_json_each_row(self, db, query):
params = {
"database": db,
"query": query,
"default_format": "JSONEachRow",
}
for i in range(5):
response = None
try:
response = requests.get(self.url, params=params, headers=self.auth)
response.raise_for_status()
return response.text
except Exception as ex:
logging.warning("Cannot fetch data with exception %s", str(ex))
if response:
logging.warning("Reponse text %s", response.text)
time.sleep(0.1 * i)
raise CHException("Cannot fetch data from clickhouse")
def select_json_each_row(self, db, query):
text = self._select_and_get_json_each_row(db, query)
result = []
for line in text.split("\n"):
if line:
result.append(json.loads(line))
return result
CH_CLIENT = ClickHouseHelper(get_parameter_from_ssm("clickhouse-test-stat-url"), "play")
def set_capacity( def set_capacity(
@ -170,7 +117,17 @@ def set_capacity(
# Finally, should the capacity be even changed # Finally, should the capacity be even changed
stop = stop or asg["DesiredCapacity"] == desired_capacity stop = stop or asg["DesiredCapacity"] == desired_capacity
if stop: if stop:
logging.info(
"Do not increase ASG %s capacity, current capacity=%s, "
"maximum capacity=%s, running jobs=%s, queue size=%s",
asg["AutoScalingGroupName"],
desired_capacity,
asg["MaxSize"],
running,
queued,
)
return return
logging.info( logging.info(
"The ASG %s capacity will be increased to %s, current capacity=%s, " "The ASG %s capacity will be increased to %s, current capacity=%s, "
"maximum capacity=%s, running jobs=%s, queue size=%s", "maximum capacity=%s, running jobs=%s, queue size=%s",
@ -195,6 +152,15 @@ def set_capacity(
desired_capacity = min(desired_capacity, asg["MaxSize"]) desired_capacity = min(desired_capacity, asg["MaxSize"])
stop = stop or asg["DesiredCapacity"] == desired_capacity stop = stop or asg["DesiredCapacity"] == desired_capacity
if stop: if stop:
logging.info(
"Do not decrease ASG %s capacity, current capacity=%s, "
"minimum capacity=%s, running jobs=%s, queue size=%s",
asg["AutoScalingGroupName"],
desired_capacity,
asg["MinSize"],
running,
queued,
)
return return
logging.info( logging.info(
@ -219,6 +185,9 @@ def main(dry_run: bool = True) -> None:
asg_client = boto3.client("autoscaling") asg_client = boto3.client("autoscaling")
try: try:
global CH_CLIENT global CH_CLIENT
CH_CLIENT = CH_CLIENT or ClickHouseHelper(
get_parameter_from_ssm("clickhouse-test-stat-url"), "play"
)
queues = CH_CLIENT.select_json_each_row("default", QUEUE_QUERY) queues = CH_CLIENT.select_json_each_row("default", QUEUE_QUERY)
except CHException as ex: except CHException as ex:
logging.exception( logging.exception(

View File

@ -4,7 +4,7 @@ import unittest
from dataclasses import dataclass from dataclasses import dataclass
from typing import Any, List from typing import Any, List
from autoscale_runners_lambda.app import set_capacity, Queue from app import set_capacity, Queue
@dataclass @dataclass

View File

@ -0,0 +1 @@
../lambda_shared_package/lambda_shared

View File

@ -1 +1 @@
requests<2.30 ../lambda_shared_package

View File

@ -9,9 +9,10 @@ import json
import re import re
import time import time
import jwt
import requests # type: ignore import requests # type: ignore
import boto3 # type: ignore
from lambda_shared.pr import CATEGORY_TO_LABEL, check_pr_description
from lambda_shared.token import get_cached_access_token
NEED_RERUN_ON_EDITED = { NEED_RERUN_ON_EDITED = {
@ -27,123 +28,6 @@ MAX_RETRY = 5
DEBUG_INFO = {} # type: Dict[str, Any] DEBUG_INFO = {} # type: Dict[str, Any]
# Descriptions are used in .github/PULL_REQUEST_TEMPLATE.md, keep comments there
# updated accordingly
# The following lists are append only, try to avoid editing them
# They still could be cleaned out after the decent time though.
LABELS = {
"pr-backward-incompatible": ["Backward Incompatible Change"],
"pr-bugfix": [
"Bug Fix",
"Bug Fix (user-visible misbehavior in an official stable release)",
"Bug Fix (user-visible misbehaviour in official stable or prestable release)",
"Bug Fix (user-visible misbehavior in official stable or prestable release)",
],
"pr-build": [
"Build/Testing/Packaging Improvement",
"Build Improvement",
"Build/Testing Improvement",
"Build",
"Packaging Improvement",
],
"pr-documentation": [
"Documentation (changelog entry is not required)",
"Documentation",
],
"pr-feature": ["New Feature"],
"pr-improvement": ["Improvement"],
"pr-not-for-changelog": [
"Not for changelog (changelog entry is not required)",
"Not for changelog",
],
"pr-performance": ["Performance Improvement"],
}
CATEGORY_TO_LABEL = {c: lb for lb, categories in LABELS.items() for c in categories}
def check_pr_description(pr_body: str) -> Tuple[str, str]:
"""The function checks the body to being properly formatted according to
.github/PULL_REQUEST_TEMPLATE.md, if the first returned string is not empty,
then there is an error."""
lines = list(map(lambda x: x.strip(), pr_body.split("\n") if pr_body else []))
lines = [re.sub(r"\s+", " ", line) for line in lines]
# Check if body contains "Reverts ClickHouse/ClickHouse#36337"
if [
True
for line in lines
if re.match(r"\AReverts {GITHUB_REPOSITORY}#[\d]+\Z", line)
]:
return "", LABELS["pr-not-for-changelog"][0]
category = ""
entry = ""
description_error = ""
i = 0
while i < len(lines):
if re.match(r"(?i)^[#>*_ ]*change\s*log\s*category", lines[i]):
i += 1
if i >= len(lines):
break
# Can have one empty line between header and the category
# itself. Filter it out.
if not lines[i]:
i += 1
if i >= len(lines):
break
category = re.sub(r"^[-*\s]*", "", lines[i])
i += 1
# Should not have more than one category. Require empty line
# after the first found category.
if i >= len(lines):
break
if lines[i]:
second_category = re.sub(r"^[-*\s]*", "", lines[i])
description_error = (
"More than one changelog category specified: "
f"'{category}', '{second_category}'"
)
return description_error, category
elif re.match(
r"(?i)^[#>*_ ]*(short\s*description|change\s*log\s*entry)", lines[i]
):
i += 1
# Can have one empty line between header and the entry itself.
# Filter it out.
if i < len(lines) and not lines[i]:
i += 1
# All following lines until empty one are the changelog entry.
entry_lines = []
while i < len(lines) and lines[i]:
entry_lines.append(lines[i])
i += 1
entry = " ".join(entry_lines)
# Don't accept changelog entries like '...'.
entry = re.sub(r"[#>*_.\- ]", "", entry)
# Don't accept changelog entries like 'Close #12345'.
entry = re.sub(r"^[\w\-\s]{0,10}#?\d{5,6}\.?$", "", entry)
else:
i += 1
if not category:
description_error = "Changelog category is empty"
# Filter out the PR categories that are not for changelog.
elif re.match(
r"(?i)doc|((non|in|not|un)[-\s]*significant)|(not[ ]*for[ ]*changelog)",
category,
):
pass # to not check the rest of the conditions
elif category not in CATEGORY_TO_LABEL:
description_error, category = f"Category '{category}' is not valid", ""
elif not entry:
description_error = f"Changelog entry required for category '{category}'"
return description_error, category
class Worker(Thread): class Worker(Thread):
def __init__( def __init__(
@ -166,58 +50,6 @@ class Worker(Thread):
self.queue.task_done() self.queue.task_done()
def get_installation_id(jwt_token):
headers = {
"Authorization": f"Bearer {jwt_token}",
"Accept": "application/vnd.github.v3+json",
}
response = requests.get("https://api.github.com/app/installations", headers=headers)
response.raise_for_status()
data = response.json()
for installation in data:
if installation["account"]["login"] == "ClickHouse":
installation_id = installation["id"]
return installation_id
def get_access_token(jwt_token, installation_id):
headers = {
"Authorization": f"Bearer {jwt_token}",
"Accept": "application/vnd.github.v3+json",
}
response = requests.post(
f"https://api.github.com/app/installations/{installation_id}/access_tokens",
headers=headers,
)
response.raise_for_status()
data = response.json()
return data["token"]
def get_key_and_app_from_aws():
secret_name = "clickhouse_github_secret_key"
session = boto3.session.Session()
client = session.client(
service_name="secretsmanager",
)
get_secret_value_response = client.get_secret_value(SecretId=secret_name)
data = json.loads(get_secret_value_response["SecretString"])
return data["clickhouse-app-key"], int(data["clickhouse-app-id"])
def get_token_from_aws():
private_key, app_id = get_key_and_app_from_aws()
payload = {
"iat": int(time.time()) - 60,
"exp": int(time.time()) + (10 * 60),
"iss": app_id,
}
encoded_jwt = jwt.encode(payload, private_key, algorithm="RS256")
installation_id = get_installation_id(encoded_jwt)
return get_access_token(encoded_jwt, installation_id)
def _exec_get_with_retry(url: str, token: str) -> dict: def _exec_get_with_retry(url: str, token: str) -> dict:
headers = {"Authorization": f"token {token}"} headers = {"Authorization": f"token {token}"}
for i in range(MAX_RETRY): for i in range(MAX_RETRY):
@ -407,7 +239,7 @@ def exec_workflow_url(urls_to_post, token):
def main(event): def main(event):
token = get_token_from_aws() token = get_cached_access_token()
DEBUG_INFO["event"] = event DEBUG_INFO["event"] = event
if event["isBase64Encoded"]: if event["isBase64Encoded"]:
event_data = json.loads(b64decode(event["body"])) event_data = json.loads(b64decode(event["body"]))

View File

@ -0,0 +1 @@
../lambda_shared_package/lambda_shared

View File

@ -1,3 +1 @@
requests<2.30 ../lambda_shared_package[token]
PyJWT
cryptography<38

View File

@ -8,32 +8,26 @@ Lambda function to:
import argparse import argparse
import sys import sys
import json
import time
from collections import namedtuple
from datetime import datetime from datetime import datetime
from typing import Dict, List, Tuple from typing import Dict, List
import jwt
import requests # type: ignore import requests # type: ignore
import boto3 # type: ignore import boto3 # type: ignore
from botocore.exceptions import ClientError # type: ignore from botocore.exceptions import ClientError # type: ignore
UNIVERSAL_LABEL = "universal" from lambda_shared import (
RUNNER_TYPE_LABELS = [ RUNNER_TYPE_LABELS,
"builder", RunnerDescription,
"func-tester", RunnerDescriptions,
"func-tester-aarch64", list_runners,
"fuzzer-unit-tester",
"stress-tester",
"style-checker",
"style-checker-aarch64",
]
RunnerDescription = namedtuple(
"RunnerDescription", ["id", "name", "tags", "offline", "busy"]
) )
RunnerDescriptions = List[RunnerDescription] from lambda_shared.token import (
get_cached_access_token,
get_key_and_app_from_aws,
get_access_token_by_key_app,
)
UNIVERSAL_LABEL = "universal"
def get_dead_runners_in_ec2(runners: RunnerDescriptions) -> RunnerDescriptions: def get_dead_runners_in_ec2(runners: RunnerDescriptions) -> RunnerDescriptions:
@ -105,138 +99,53 @@ def get_dead_runners_in_ec2(runners: RunnerDescriptions) -> RunnerDescriptions:
def get_lost_ec2_instances(runners: RunnerDescriptions) -> List[dict]: def get_lost_ec2_instances(runners: RunnerDescriptions) -> List[dict]:
client = boto3.client("ec2") client = boto3.client("ec2")
reservations = client.describe_instances( reservations = client.describe_instances(
Filters=[{"Name": "tag-key", "Values": ["github:runner-type"]}] Filters=[
{"Name": "tag-key", "Values": ["github:runner-type"]},
{"Name": "instance-state-name", "Values": ["pending", "running"]},
],
)["Reservations"] )["Reservations"]
lost_instances = [] # flatten the reservation into instances
offline_runners = [ instances = [
runner.name for runner in runners if runner.offline and not runner.busy instance
for reservation in reservations
for instance in reservation["Instances"]
] ]
# Here we refresh the runners to get the most recent state lost_instances = []
offline_runner_names = {
runner.name for runner in runners if runner.offline and not runner.busy
}
runner_names = {runner.name for runner in runners}
now = datetime.now().timestamp() now = datetime.now().timestamp()
for reservation in reservations: for instance in instances:
for instance in reservation["Instances"]: # Do not consider instances started 20 minutes ago as problematic
# Do not consider instances started 20 minutes ago as problematic if now - instance["LaunchTime"].timestamp() < 1200:
if now - instance["LaunchTime"].timestamp() < 1200: continue
continue
runner_type = [ runner_type = [
tag["Value"] tag["Value"]
for tag in instance["Tags"] for tag in instance["Tags"]
if tag["Key"] == "github:runner-type" if tag["Key"] == "github:runner-type"
][0] ][0]
# If there's no necessary labels in runner type it's fine # If there's no necessary labels in runner type it's fine
if not ( if not (UNIVERSAL_LABEL in runner_type or runner_type in RUNNER_TYPE_LABELS):
UNIVERSAL_LABEL in runner_type or runner_type in RUNNER_TYPE_LABELS continue
):
continue
if instance["InstanceId"] in offline_runners: if instance["InstanceId"] in offline_runner_names:
lost_instances.append(instance) lost_instances.append(instance)
continue continue
if instance["State"]["Name"] == "running" and ( if (
not [ instance["State"]["Name"] == "running"
runner and not instance["InstanceId"] in runner_names
for runner in runners ):
if runner.name == instance["InstanceId"] lost_instances.append(instance)
]
):
lost_instances.append(instance)
return lost_instances return lost_instances
def get_key_and_app_from_aws() -> Tuple[str, int]:
secret_name = "clickhouse_github_secret_key"
session = boto3.session.Session()
client = session.client(
service_name="secretsmanager",
)
get_secret_value_response = client.get_secret_value(SecretId=secret_name)
data = json.loads(get_secret_value_response["SecretString"])
return data["clickhouse-app-key"], int(data["clickhouse-app-id"])
def handler(event, context): def handler(event, context):
private_key, app_id = get_key_and_app_from_aws() main(get_cached_access_token(), True, True)
main(private_key, app_id, True, True)
def get_installation_id(jwt_token: str) -> int:
headers = {
"Authorization": f"Bearer {jwt_token}",
"Accept": "application/vnd.github.v3+json",
}
response = requests.get("https://api.github.com/app/installations", headers=headers)
response.raise_for_status()
data = response.json()
for installation in data:
if installation["account"]["login"] == "ClickHouse":
installation_id = installation["id"]
break
return installation_id # type: ignore
def get_access_token(jwt_token: str, installation_id: int) -> str:
headers = {
"Authorization": f"Bearer {jwt_token}",
"Accept": "application/vnd.github.v3+json",
}
response = requests.post(
f"https://api.github.com/app/installations/{installation_id}/access_tokens",
headers=headers,
)
response.raise_for_status()
data = response.json()
return data["token"] # type: ignore
def list_runners(access_token: str) -> RunnerDescriptions:
headers = {
"Authorization": f"token {access_token}",
"Accept": "application/vnd.github.v3+json",
}
per_page = 100
response = requests.get(
f"https://api.github.com/orgs/ClickHouse/actions/runners?per_page={per_page}",
headers=headers,
)
response.raise_for_status()
data = response.json()
total_runners = data["total_count"]
print("Expected total runners", total_runners)
runners = data["runners"]
# round to 0 for 0, 1 for 1..100, but to 2 for 101..200
total_pages = (total_runners - 1) // per_page + 1
print("Total pages", total_pages)
for i in range(2, total_pages + 1):
response = requests.get(
"https://api.github.com/orgs/ClickHouse/actions/runners"
f"?page={i}&per_page={per_page}",
headers=headers,
)
response.raise_for_status()
data = response.json()
runners += data["runners"]
print("Total runners", len(runners))
result = []
for runner in runners:
tags = [tag["name"] for tag in runner["labels"]]
desc = RunnerDescription(
id=runner["id"],
name=runner["name"],
tags=tags,
offline=runner["status"] == "offline",
busy=runner["busy"],
)
result.append(desc)
return result
def group_runners_by_tag( def group_runners_by_tag(
@ -265,18 +174,21 @@ def group_runners_by_tag(
def push_metrics_to_cloudwatch( def push_metrics_to_cloudwatch(
listed_runners: RunnerDescriptions, namespace: str listed_runners: RunnerDescriptions, group_name: str
) -> None: ) -> None:
client = boto3.client("cloudwatch") client = boto3.client("cloudwatch")
namespace = "RunnersMetrics"
metrics_data = [] metrics_data = []
busy_runners = sum( busy_runners = sum(
1 for runner in listed_runners if runner.busy and not runner.offline 1 for runner in listed_runners if runner.busy and not runner.offline
) )
dimensions = [{"Name": "group", "Value": group_name}]
metrics_data.append( metrics_data.append(
{ {
"MetricName": "BusyRunners", "MetricName": "BusyRunners",
"Value": busy_runners, "Value": busy_runners,
"Unit": "Count", "Unit": "Count",
"Dimensions": dimensions,
} }
) )
total_active_runners = sum(1 for runner in listed_runners if not runner.offline) total_active_runners = sum(1 for runner in listed_runners if not runner.offline)
@ -285,6 +197,7 @@ def push_metrics_to_cloudwatch(
"MetricName": "ActiveRunners", "MetricName": "ActiveRunners",
"Value": total_active_runners, "Value": total_active_runners,
"Unit": "Count", "Unit": "Count",
"Dimensions": dimensions,
} }
) )
total_runners = len(listed_runners) total_runners = len(listed_runners)
@ -293,6 +206,7 @@ def push_metrics_to_cloudwatch(
"MetricName": "TotalRunners", "MetricName": "TotalRunners",
"Value": total_runners, "Value": total_runners,
"Unit": "Count", "Unit": "Count",
"Dimensions": dimensions,
} }
) )
if total_active_runners == 0: if total_active_runners == 0:
@ -305,6 +219,7 @@ def push_metrics_to_cloudwatch(
"MetricName": "BusyRunnersRatio", "MetricName": "BusyRunnersRatio",
"Value": busy_ratio, "Value": busy_ratio,
"Unit": "Percent", "Unit": "Percent",
"Dimensions": dimensions,
} }
) )
@ -327,26 +242,16 @@ def delete_runner(access_token: str, runner: RunnerDescription) -> bool:
def main( def main(
github_secret_key: str, access_token: str,
github_app_id: int,
push_to_cloudwatch: bool, push_to_cloudwatch: bool,
delete_offline_runners: bool, delete_offline_runners: bool,
) -> None: ) -> None:
payload = {
"iat": int(time.time()) - 60,
"exp": int(time.time()) + (10 * 60),
"iss": github_app_id,
}
encoded_jwt = jwt.encode(payload, github_secret_key, algorithm="RS256")
installation_id = get_installation_id(encoded_jwt)
access_token = get_access_token(encoded_jwt, installation_id)
gh_runners = list_runners(access_token) gh_runners = list_runners(access_token)
grouped_runners = group_runners_by_tag(gh_runners) grouped_runners = group_runners_by_tag(gh_runners)
for group, group_runners in grouped_runners.items(): for group, group_runners in grouped_runners.items():
if push_to_cloudwatch: if push_to_cloudwatch:
print(f"Pushing metrics for group '{group}'") print(f"Pushing metrics for group '{group}'")
push_metrics_to_cloudwatch(group_runners, "RunnersMetrics/" + group) push_metrics_to_cloudwatch(group_runners, group)
else: else:
print(group, f"({len(group_runners)})") print(group, f"({len(group_runners)})")
for runner in group_runners: for runner in group_runners:
@ -408,4 +313,6 @@ if __name__ == "__main__":
print("Attempt to get key and id from AWS secret manager") print("Attempt to get key and id from AWS secret manager")
private_key, args.app_id = get_key_and_app_from_aws() private_key, args.app_id = get_key_and_app_from_aws()
main(private_key, args.app_id, args.push_to_cloudwatch, args.delete_offline) token = get_access_token_by_key_app(private_key, args.app_id)
main(token, args.push_to_cloudwatch, args.delete_offline)

View File

@ -0,0 +1 @@
../lambda_shared_package/lambda_shared

View File

@ -1,3 +1,2 @@
requests<2.30 ../lambda_shared_package
PyJWT ../lambda_shared_package[token]
cryptography<38

View File

@ -0,0 +1,2 @@
build
*.egg-info

View File

@ -0,0 +1,221 @@
"""The shared code and types for all our CI lambdas
It exists as __init__.py and lambda_shared/__init__.py to work both in local and venv"""
import json
import logging
import time
from collections import namedtuple
from typing import Any, Dict, Iterable, List, Optional
import boto3 # type: ignore
import requests # type: ignore
RUNNER_TYPE_LABELS = [
"builder",
"func-tester",
"func-tester-aarch64",
"fuzzer-unit-tester",
"limited-tester",
"stress-tester",
"style-checker",
"style-checker-aarch64",
]
### VENDORING
def get_parameter_from_ssm(
name: str, decrypt: bool = True, client: Optional[Any] = None
) -> str:
if not client:
client = boto3.client("ssm", region_name="us-east-1")
return client.get_parameter(Name=name, WithDecryption=decrypt)[ # type: ignore
"Parameter"
]["Value"]
class CHException(Exception):
pass
class InsertException(CHException):
pass
class ClickHouseHelper:
def __init__(
self,
url: str,
user: Optional[str] = None,
password: Optional[str] = None,
):
self.url = url
self.auth = {}
if user:
self.auth["X-ClickHouse-User"] = user
if password:
self.auth["X-ClickHouse-Key"] = password
@staticmethod
def _insert_json_str_info_impl(
url: str, auth: Dict[str, str], db: str, table: str, json_str: str
) -> None:
params = {
"database": db,
"query": f"INSERT INTO {table} FORMAT JSONEachRow",
"date_time_input_format": "best_effort",
"send_logs_level": "warning",
}
for i in range(5):
try:
response = requests.post(
url, params=params, data=json_str, headers=auth
)
except Exception as e:
error = f"Received exception while sending data to {url} on {i} attempt: {e}"
logging.warning(error)
continue
logging.info("Response content '%s'", response.content)
if response.ok:
break
error = (
"Cannot insert data into clickhouse at try "
+ str(i)
+ ": HTTP code "
+ str(response.status_code)
+ ": '"
+ str(response.text)
+ "'"
)
if response.status_code >= 500:
# A retriable error
time.sleep(1)
continue
logging.info(
"Request headers '%s', body '%s'",
response.request.headers,
response.request.body,
)
raise InsertException(error)
else:
raise InsertException(error)
def _insert_json_str_info(self, db: str, table: str, json_str: str) -> None:
self._insert_json_str_info_impl(self.url, self.auth, db, table, json_str)
def insert_event_into(
self, db: str, table: str, event: object, safe: bool = True
) -> None:
event_str = json.dumps(event)
try:
self._insert_json_str_info(db, table, event_str)
except InsertException as e:
logging.error(
"Exception happened during inserting data into clickhouse: %s", e
)
if not safe:
raise
def insert_events_into(
self, db: str, table: str, events: Iterable[object], safe: bool = True
) -> None:
jsons = []
for event in events:
jsons.append(json.dumps(event))
try:
self._insert_json_str_info(db, table, ",".join(jsons))
except InsertException as e:
logging.error(
"Exception happened during inserting data into clickhouse: %s", e
)
if not safe:
raise
def _select_and_get_json_each_row(self, db: str, query: str) -> str:
params = {
"database": db,
"query": query,
"default_format": "JSONEachRow",
}
for i in range(5):
response = None
try:
response = requests.get(self.url, params=params, headers=self.auth)
response.raise_for_status()
return response.text # type: ignore
except Exception as ex:
logging.warning("Cannot fetch data with exception %s", str(ex))
if response:
logging.warning("Reponse text %s", response.text)
time.sleep(0.1 * i)
raise CHException("Cannot fetch data from clickhouse")
def select_json_each_row(self, db: str, query: str) -> List[dict]:
text = self._select_and_get_json_each_row(db, query)
result = []
for line in text.split("\n"):
if line:
result.append(json.loads(line))
return result
### Runners
RunnerDescription = namedtuple(
"RunnerDescription", ["id", "name", "tags", "offline", "busy"]
)
RunnerDescriptions = List[RunnerDescription]
def list_runners(access_token: str) -> RunnerDescriptions:
headers = {
"Authorization": f"token {access_token}",
"Accept": "application/vnd.github.v3+json",
}
per_page = 100
response = requests.get(
f"https://api.github.com/orgs/ClickHouse/actions/runners?per_page={per_page}",
headers=headers,
)
response.raise_for_status()
data = response.json()
total_runners = data["total_count"]
print("Expected total runners", total_runners)
runners = data["runners"]
# round to 0 for 0, 1 for 1..100, but to 2 for 101..200
total_pages = (total_runners - 1) // per_page + 1
print("Total pages", total_pages)
for i in range(2, total_pages + 1):
response = requests.get(
"https://api.github.com/orgs/ClickHouse/actions/runners"
f"?page={i}&per_page={per_page}",
headers=headers,
)
response.raise_for_status()
data = response.json()
runners += data["runners"]
print("Total runners", len(runners))
result = []
for runner in runners:
tags = [tag["name"] for tag in runner["labels"]]
desc = RunnerDescription(
id=runner["id"],
name=runner["name"],
tags=tags,
offline=runner["status"] == "offline",
busy=runner["busy"],
)
result.append(desc)
return result

View File

@ -0,0 +1,184 @@
#!/usr/bin/env python
import re
from typing import Tuple
# Individual trusted contirbutors who are not in any trusted organization.
# Can be changed in runtime: we will append users that we learned to be in
# a trusted org, to save GitHub API calls.
TRUSTED_CONTRIBUTORS = {
e.lower()
for e in [
"achimbab",
"adevyatova ", # DOCSUP
"Algunenano", # Raúl Marín, Tinybird
"amosbird",
"AnaUvarova", # DOCSUP
"anauvarova", # technical writer, Yandex
"annvsh", # technical writer, Yandex
"atereh", # DOCSUP
"azat",
"bharatnc", # Newbie, but already with many contributions.
"bobrik", # Seasoned contributor, CloudFlare
"BohuTANG",
"codyrobert", # Flickerbox engineer
"cwurm", # Employee
"damozhaeva", # DOCSUP
"den-crane",
"flickerbox-tom", # Flickerbox
"gyuton", # DOCSUP
"hagen1778", # Roman Khavronenko, seasoned contributor
"hczhcz",
"hexiaoting", # Seasoned contributor
"ildus", # adjust, ex-pgpro
"javisantana", # a Spanish ClickHouse enthusiast, ex-Carto
"ka1bi4", # DOCSUP
"kirillikoff", # DOCSUP
"kreuzerkrieg",
"lehasm", # DOCSUP
"michon470", # DOCSUP
"nikvas0",
"nvartolomei",
"olgarev", # DOCSUP
"otrazhenia", # Yandex docs contractor
"pdv-ru", # DOCSUP
"podshumok", # cmake expert from QRator Labs
"s-mx", # Maxim Sabyanin, former employee, present contributor
"sevirov", # technical writer, Yandex
"spongedu", # Seasoned contributor
"taiyang-li",
"ucasFL", # Amos Bird's friend
"vdimir", # Employee
"vzakaznikov",
"YiuRULE",
"zlobober", # Developer of YT
"ilejn", # Arenadata, responsible for Kerberized Kafka
"thomoco", # ClickHouse
"BoloniniD", # Seasoned contributor, HSE
"tonickkozlov", # Cloudflare
"tylerhannan", # ClickHouse Employee
"myrrc", # Mike Kot, DoubleCloud
"thevar1able", # ClickHouse Employee
"aalexfvk",
"MikhailBurdukov",
"tsolodov", # ClickHouse Employee
"kitaisreal",
]
}
# Descriptions are used in .github/PULL_REQUEST_TEMPLATE.md, keep comments there
# updated accordingly
# The following lists are append only, try to avoid editing them
# They still could be cleaned out after the decent time though.
LABELS = {
"pr-backward-incompatible": ["Backward Incompatible Change"],
"pr-bugfix": [
"Bug Fix",
"Bug Fix (user-visible misbehavior in an official stable release)",
"Bug Fix (user-visible misbehaviour in official stable or prestable release)",
"Bug Fix (user-visible misbehavior in official stable or prestable release)",
],
"pr-build": [
"Build/Testing/Packaging Improvement",
"Build Improvement",
"Build/Testing Improvement",
"Build",
"Packaging Improvement",
],
"pr-documentation": [
"Documentation (changelog entry is not required)",
"Documentation",
],
"pr-feature": ["New Feature"],
"pr-improvement": ["Improvement"],
"pr-not-for-changelog": [
"Not for changelog (changelog entry is not required)",
"Not for changelog",
],
"pr-performance": ["Performance Improvement"],
}
CATEGORY_TO_LABEL = {c: lb for lb, categories in LABELS.items() for c in categories}
def check_pr_description(pr_body: str) -> Tuple[str, str]:
"""The function checks the body to being properly formatted according to
.github/PULL_REQUEST_TEMPLATE.md, if the first returned string is not empty,
then there is an error."""
lines = list(map(lambda x: x.strip(), pr_body.split("\n") if pr_body else []))
lines = [re.sub(r"\s+", " ", line) for line in lines]
# Check if body contains "Reverts ClickHouse/ClickHouse#36337"
if [
True
for line in lines
if re.match(r"\AReverts {GITHUB_REPOSITORY}#[\d]+\Z", line)
]:
return "", LABELS["pr-not-for-changelog"][0]
category = ""
entry = ""
description_error = ""
i = 0
while i < len(lines):
if re.match(r"(?i)^[#>*_ ]*change\s*log\s*category", lines[i]):
i += 1
if i >= len(lines):
break
# Can have one empty line between header and the category
# itself. Filter it out.
if not lines[i]:
i += 1
if i >= len(lines):
break
category = re.sub(r"^[-*\s]*", "", lines[i])
i += 1
# Should not have more than one category. Require empty line
# after the first found category.
if i >= len(lines):
break
if lines[i]:
second_category = re.sub(r"^[-*\s]*", "", lines[i])
description_error = (
"More than one changelog category specified: "
f"'{category}', '{second_category}'"
)
return description_error, category
elif re.match(
r"(?i)^[#>*_ ]*(short\s*description|change\s*log\s*entry)", lines[i]
):
i += 1
# Can have one empty line between header and the entry itself.
# Filter it out.
if i < len(lines) and not lines[i]:
i += 1
# All following lines until empty one are the changelog entry.
entry_lines = []
while i < len(lines) and lines[i]:
entry_lines.append(lines[i])
i += 1
entry = " ".join(entry_lines)
# Don't accept changelog entries like '...'.
entry = re.sub(r"[#>*_.\- ]", "", entry)
# Don't accept changelog entries like 'Close #12345'.
entry = re.sub(r"^[\w\-\s]{0,10}#?\d{5,6}\.?$", "", entry)
else:
i += 1
if not category:
description_error = "Changelog category is empty"
# Filter out the PR categories that are not for changelog.
elif re.match(
r"(?i)doc|((non|in|not|un)[-\s]*significant)|(not[ ]*for[ ]*changelog)",
category,
):
pass # to not check the rest of the conditions
elif category not in CATEGORY_TO_LABEL:
description_error, category = f"Category '{category}' is not valid", ""
elif not entry:
description_error = f"Changelog entry required for category '{category}'"
return description_error, category

View File

@ -0,0 +1,90 @@
"""Module to get the token for GitHub"""
from dataclasses import dataclass
import json
import time
from typing import Tuple
import boto3 # type: ignore
import jwt
import requests # type: ignore
def get_key_and_app_from_aws() -> Tuple[str, int]:
secret_name = "clickhouse_github_secret_key"
session = boto3.session.Session()
client = session.client(
service_name="secretsmanager",
)
get_secret_value_response = client.get_secret_value(SecretId=secret_name)
data = json.loads(get_secret_value_response["SecretString"])
return data["clickhouse-app-key"], int(data["clickhouse-app-id"])
def get_installation_id(jwt_token: str) -> int:
headers = {
"Authorization": f"Bearer {jwt_token}",
"Accept": "application/vnd.github.v3+json",
}
response = requests.get("https://api.github.com/app/installations", headers=headers)
response.raise_for_status()
data = response.json()
for installation in data:
if installation["account"]["login"] == "ClickHouse":
installation_id = installation["id"]
return installation_id # type: ignore
def get_access_token_by_jwt(jwt_token: str, installation_id: int) -> str:
headers = {
"Authorization": f"Bearer {jwt_token}",
"Accept": "application/vnd.github.v3+json",
}
response = requests.post(
f"https://api.github.com/app/installations/{installation_id}/access_tokens",
headers=headers,
)
response.raise_for_status()
data = response.json()
return data["token"] # type: ignore
def get_token_from_aws() -> str:
private_key, app_id = get_key_and_app_from_aws()
return get_access_token_by_key_app(private_key, app_id)
def get_access_token_by_key_app(private_key: str, app_id: int) -> str:
payload = {
"iat": int(time.time()) - 60,
"exp": int(time.time()) + (10 * 60),
"iss": app_id,
}
encoded_jwt = jwt.encode(payload, private_key, algorithm="RS256")
installation_id = get_installation_id(encoded_jwt)
return get_access_token_by_jwt(encoded_jwt, installation_id)
@dataclass
class CachedToken:
time: int
value: str
updating: bool = False
_cached_token = CachedToken(0, "")
def get_cached_access_token() -> str:
if time.time() - 550 < _cached_token.time or _cached_token.updating:
return _cached_token.value
# Indicate that the value is updating now, so the cached value can be
# used. The first setting and close-to-ttl are not counted as update
if _cached_token.time != 0 or time.time() - 590 < _cached_token.time:
_cached_token.updating = True
private_key, app_id = get_key_and_app_from_aws()
_cached_token.time = int(time.time())
_cached_token.value = get_access_token_by_key_app(private_key, app_id)
_cached_token.updating = False
return _cached_token.value

View File

@ -0,0 +1,24 @@
[build-system]
requires = ["setuptools"]
build-backend = "setuptools.build_meta"
[project]
name = "lambda_shared"
version = "0.0.1"
dependencies = [
"requests",
"urllib3 < 2"
]
[project.optional-dependencies]
token = [
"PyJWT",
"cryptography",
]
dev = [
"boto3",
"lambda_shared[token]",
]
[tool.distutils.bdist_wheel]
universal = true

View File

@ -0,0 +1,8 @@
### This file exists for clear builds in docker ###
# without it the `build` directory wouldn't be #
# updated on the fly and will require manual clean #
[build]
build_base = /tmp/lambda_shared
[egg_info]
egg_base = /tmp/

View File

@ -219,6 +219,12 @@ if __name__ == "__main__":
except Exception: except Exception:
traceback.print_exc() traceback.print_exc()
def too_many_slow(msg):
match = re.search(r"(|.* )(\d+) slower.*", msg)
# This threshold should be synchronized with the value in https://github.com/ClickHouse/ClickHouse/blob/master/docker/test/performance-comparison/report.py#L629
threshold = 5
return int(match.group(2).strip()) > threshold if match else False
# Try to fetch status from the report. # Try to fetch status from the report.
status = "" status = ""
message = "" message = ""
@ -236,7 +242,7 @@ if __name__ == "__main__":
# TODO: Remove me, always green mode for the first time, unless errors # TODO: Remove me, always green mode for the first time, unless errors
status = "success" status = "success"
if "errors" in message.lower(): if "errors" in message.lower() or too_many_slow(message.lower()):
status = "failure" status = "failure"
# TODO: Remove until here # TODO: Remove until here
except Exception: except Exception:

View File

@ -20,9 +20,11 @@ from docs_check import NAME as DOCS_NAME
from env_helper import GITHUB_REPOSITORY, GITHUB_SERVER_URL from env_helper import GITHUB_REPOSITORY, GITHUB_SERVER_URL
from get_robot_token import get_best_robot_token from get_robot_token import get_best_robot_token
from pr_info import FORCE_TESTS_LABEL, PRInfo from pr_info import FORCE_TESTS_LABEL, PRInfo
from lambda_shared_package.lambda_shared.pr import (
from cancel_and_rerun_workflow_lambda.app import CATEGORY_TO_LABEL, check_pr_description CATEGORY_TO_LABEL,
from workflow_approve_rerun_lambda.app import TRUSTED_CONTRIBUTORS TRUSTED_CONTRIBUTORS,
check_pr_description,
)
TRUSTED_ORG_IDS = { TRUSTED_ORG_IDS = {
54801242, # clickhouse 54801242, # clickhouse

View File

@ -2,40 +2,11 @@
import argparse import argparse
import sys import sys
import json
import time
import boto3 # type: ignore import boto3 # type: ignore
import jwt
import requests # type: ignore import requests # type: ignore
from lambda_shared.token import get_cached_access_token, get_access_token_by_key_app
def get_installation_id(jwt_token):
headers = {
"Authorization": f"Bearer {jwt_token}",
"Accept": "application/vnd.github.v3+json",
}
response = requests.get("https://api.github.com/app/installations", headers=headers)
response.raise_for_status()
data = response.json()
for installation in data:
if installation["account"]["login"] == "ClickHouse":
installation_id = installation["id"]
return installation_id
def get_access_token(jwt_token, installation_id):
headers = {
"Authorization": f"Bearer {jwt_token}",
"Accept": "application/vnd.github.v3+json",
}
response = requests.post(
f"https://api.github.com/app/installations/{installation_id}/access_tokens",
headers=headers,
)
response.raise_for_status()
data = response.json()
return data["token"]
def get_runner_registration_token(access_token): def get_runner_registration_token(access_token):
@ -52,32 +23,10 @@ def get_runner_registration_token(access_token):
return data["token"] return data["token"]
def get_key_and_app_from_aws(): def main(access_token, push_to_ssm, ssm_parameter_name):
secret_name = "clickhouse_github_secret_key"
session = boto3.session.Session()
client = session.client(
service_name="secretsmanager",
)
get_secret_value_response = client.get_secret_value(SecretId=secret_name)
data = json.loads(get_secret_value_response["SecretString"])
return data["clickhouse-app-key"], int(data["clickhouse-app-id"])
def main(github_secret_key, github_app_id, push_to_ssm, ssm_parameter_name):
payload = {
"iat": int(time.time()) - 60,
"exp": int(time.time()) + (10 * 60),
"iss": github_app_id,
}
encoded_jwt = jwt.encode(payload, github_secret_key, algorithm="RS256")
installation_id = get_installation_id(encoded_jwt)
access_token = get_access_token(encoded_jwt, installation_id)
runner_registration_token = get_runner_registration_token(access_token) runner_registration_token = get_runner_registration_token(access_token)
if push_to_ssm: if push_to_ssm:
import boto3
print("Trying to put params into ssm manager") print("Trying to put params into ssm manager")
client = boto3.client("ssm") client = boto3.client("ssm")
client.put_parameter( client.put_parameter(
@ -94,8 +43,7 @@ def main(github_secret_key, github_app_id, push_to_ssm, ssm_parameter_name):
def handler(event, context): def handler(event, context):
private_key, app_id = get_key_and_app_from_aws() main(get_cached_access_token(), True, "github_runner_registration_token")
main(private_key, app_id, True, "github_runner_registration_token")
if __name__ == "__main__": if __name__ == "__main__":
@ -140,4 +88,5 @@ if __name__ == "__main__":
with open(args.private_key_path, "r") as key_file: with open(args.private_key_path, "r") as key_file:
private_key = key_file.read() private_key = key_file.read()
main(private_key, args.app_id, args.push_to_ssm, args.ssm_parameter_name) token = get_access_token_by_key_app(private_key, args.app_id)
main(token, args.push_to_ssm, args.ssm_parameter_name)

View File

@ -0,0 +1 @@
../lambda_shared_package/lambda_shared

View File

@ -1,3 +1 @@
requests<2.30 ../lambda_shared_package[token]
PyJWT
cryptography<38

View File

@ -81,6 +81,8 @@ def get_cached_members_keys(members: set) -> Keys:
def get_token_from_aws() -> str: def get_token_from_aws() -> str:
# We need a separate token, since the clickhouse-ci app does not have
# access to the organization members' endpoint
secret_name = "clickhouse_robot_token" secret_name = "clickhouse_robot_token"
session = boto3.session.Session() session = boto3.session.Session()
client = session.client( client = session.client(
@ -130,4 +132,4 @@ if __name__ == "__main__":
args = parser.parse_args() args = parser.parse_args()
output = main(args.token, args.organization, args.team) output = main(args.token, args.organization, args.team)
print(f"# Just shoing off the keys:\n{output}") print(f"# Just showing off the keys:\n{output}")

View File

@ -3,13 +3,19 @@ set -xeo pipefail
WORKDIR=$(dirname "$0") WORKDIR=$(dirname "$0")
WORKDIR=$(readlink -f "${WORKDIR}") WORKDIR=$(readlink -f "${WORKDIR}")
DIR_NAME=$(basename "$WORKDIR")
cd "$WORKDIR" cd "$WORKDIR"
PY_VERSION=3.10 # Do not deploy the lambda to AWS
DRY_RUN=${DRY_RUN:-}
# Python runtime to install dependencies
PY_VERSION=${PY_VERSION:-3.10}
PY_EXEC="python${PY_VERSION}" PY_EXEC="python${PY_VERSION}"
DOCKER_IMAGE="python:${PY_VERSION}-slim" # Image to build the lambda zip package
LAMBDA_NAME=$(basename "$WORKDIR") DOCKER_IMAGE="public.ecr.aws/lambda/python:${PY_VERSION}"
LAMBDA_NAME=${LAMBDA_NAME//_/-} # Rename the_lambda_name directory to the-lambda-name lambda in AWS
LAMBDA_NAME=${DIR_NAME//_/-}
# The name of directory with lambda code
PACKAGE=lambda-package PACKAGE=lambda-package
rm -rf "$PACKAGE" "$PACKAGE".zip rm -rf "$PACKAGE" "$PACKAGE".zip
mkdir "$PACKAGE" mkdir "$PACKAGE"
@ -17,8 +23,9 @@ cp app.py "$PACKAGE"
if [ -f requirements.txt ]; then if [ -f requirements.txt ]; then
VENV=lambda-venv VENV=lambda-venv
rm -rf "$VENV" lambda-package.zip rm -rf "$VENV" lambda-package.zip
docker run --rm --user="${UID}" --volume="${WORKDIR}:/lambda" --workdir="/lambda" "${DOCKER_IMAGE}" \ docker run --rm --user="${UID}" -e HOME=/tmp --entrypoint=/bin/bash \
/bin/bash -c " --volume="${WORKDIR}/..:/ci" --workdir="/ci/${DIR_NAME}" "${DOCKER_IMAGE}" \
-exc "
'$PY_EXEC' -m venv '$VENV' && '$PY_EXEC' -m venv '$VENV' &&
source '$VENV/bin/activate' && source '$VENV/bin/activate' &&
pip install -r requirements.txt pip install -r requirements.txt
@ -28,4 +35,6 @@ if [ -f requirements.txt ]; then
fi fi
( cd "$PACKAGE" && zip -9 -r ../"$PACKAGE".zip . ) ( cd "$PACKAGE" && zip -9 -r ../"$PACKAGE".zip . )
aws lambda update-function-code --function-name "$LAMBDA_NAME" --zip-file fileb://"$PACKAGE".zip if [ -z "$DRY_RUN" ]; then
aws lambda update-function-code --function-name "$LAMBDA_NAME" --zip-file fileb://"$WORKDIR/$PACKAGE".zip
fi

View File

@ -0,0 +1 @@
../lambda_shared_package/lambda_shared

View File

@ -1 +1 @@
requests<2.30 ../lambda_shared_package

View File

@ -4,132 +4,45 @@ import argparse
import json import json
import sys import sys
import time import time
from collections import namedtuple
from dataclasses import dataclass from dataclasses import dataclass
from typing import Any, Dict, List, Tuple from typing import Any, Dict, List
import boto3 # type: ignore import boto3 # type: ignore
import requests # type: ignore
import jwt
from lambda_shared import RunnerDescriptions, list_runners
def get_key_and_app_from_aws() -> Tuple[str, int]: from lambda_shared.token import get_access_token_by_key_app, get_cached_access_token
secret_name = "clickhouse_github_secret_key"
session = boto3.session.Session()
client = session.client(
service_name="secretsmanager",
)
get_secret_value_response = client.get_secret_value(SecretId=secret_name)
data = json.loads(get_secret_value_response["SecretString"])
return data["clickhouse-app-key"], int(data["clickhouse-app-id"])
def get_installation_id(jwt_token: str) -> int:
headers = {
"Authorization": f"Bearer {jwt_token}",
"Accept": "application/vnd.github.v3+json",
}
response = requests.get("https://api.github.com/app/installations", headers=headers)
response.raise_for_status()
data = response.json()
for installation in data:
if installation["account"]["login"] == "ClickHouse":
installation_id = installation["id"]
break
return installation_id # type: ignore
def get_access_token(jwt_token: str, installation_id: int) -> str:
headers = {
"Authorization": f"Bearer {jwt_token}",
"Accept": "application/vnd.github.v3+json",
}
response = requests.post(
f"https://api.github.com/app/installations/{installation_id}/access_tokens",
headers=headers,
)
response.raise_for_status()
data = response.json()
return data["token"] # type: ignore
@dataclass @dataclass
class CachedToken: class CachedInstances:
time: int time: int
value: str value: dict
updating: bool = False
cached_token = CachedToken(0, "") cached_instances = CachedInstances(0, {})
def get_cached_access_token() -> str: def get_cached_instances() -> dict:
if time.time() - 500 < cached_token.time: """return cached instances description with updating it once per five minutes"""
return cached_token.value if time.time() - 250 < cached_instances.time or cached_instances.updating:
private_key, app_id = get_key_and_app_from_aws() return cached_instances.value
payload = { # Indicate that the value is updating now, so the cached value can be
"iat": int(time.time()) - 60, # used. The first setting and close-to-ttl are not counted as update
"exp": int(time.time()) + (10 * 60), if cached_instances.time != 0 or time.time() - 300 < cached_instances.time:
"iss": app_id, cached_instances.updating = True
} ec2_client = boto3.client("ec2")
instances_response = ec2_client.describe_instances(
encoded_jwt = jwt.encode(payload, private_key, algorithm="RS256") Filters=[{"Name": "instance-state-name", "Values": ["running"]}]
installation_id = get_installation_id(encoded_jwt)
cached_token.time = int(time.time())
cached_token.value = get_access_token(encoded_jwt, installation_id)
return cached_token.value
RunnerDescription = namedtuple(
"RunnerDescription", ["id", "name", "tags", "offline", "busy"]
)
RunnerDescriptions = List[RunnerDescription]
def list_runners(access_token: str) -> RunnerDescriptions:
headers = {
"Authorization": f"token {access_token}",
"Accept": "application/vnd.github.v3+json",
}
per_page = 100
response = requests.get(
f"https://api.github.com/orgs/ClickHouse/actions/runners?per_page={per_page}",
headers=headers,
) )
response.raise_for_status() cached_instances.time = int(time.time())
data = response.json() cached_instances.value = {
total_runners = data["total_count"] instance["InstanceId"]: instance
print("Expected total runners", total_runners) for reservation in instances_response["Reservations"]
runners = data["runners"] for instance in reservation["Instances"]
}
# round to 0 for 0, 1 for 1..100, but to 2 for 101..200 cached_instances.updating = False
total_pages = (total_runners - 1) // per_page + 1 return cached_instances.value
print("Total pages", total_pages)
for i in range(2, total_pages + 1):
response = requests.get(
"https://api.github.com/orgs/ClickHouse/actions/runners"
f"?page={i}&per_page={per_page}",
headers=headers,
)
response.raise_for_status()
data = response.json()
runners += data["runners"]
print("Total runners", len(runners))
result = []
for runner in runners:
tags = [tag["name"] for tag in runner["labels"]]
desc = RunnerDescription(
id=runner["id"],
name=runner["name"],
tags=tags,
offline=runner["status"] == "offline",
busy=runner["busy"],
)
result.append(desc)
return result
def how_many_instances_to_kill(event_data: dict) -> Dict[str, int]: def how_many_instances_to_kill(event_data: dict) -> Dict[str, int]:
@ -160,11 +73,37 @@ def get_candidates_to_be_killed(event_data: dict) -> Dict[str, List[str]]:
def main(access_token: str, event: dict) -> Dict[str, List[str]]: def main(access_token: str, event: dict) -> Dict[str, List[str]]:
print("Got event", json.dumps(event, sort_keys=True, indent=4)) start = time.time()
print("Got event", json.dumps(event, sort_keys=True).replace("\n", ""))
to_kill_by_zone = how_many_instances_to_kill(event) to_kill_by_zone = how_many_instances_to_kill(event)
instances_by_zone = get_candidates_to_be_killed(event) instances_by_zone = get_candidates_to_be_killed(event)
# Getting ASG and instances' descriptions from the API
# We don't kill instances that alive for less than 10 minutes, since they
# could be not in the GH active runners yet
print(f"Check other hosts from the same ASG {event['AutoScalingGroupName']}")
asg_client = boto3.client("autoscaling")
as_groups_response = asg_client.describe_auto_scaling_groups(
AutoScalingGroupNames=[event["AutoScalingGroupName"]]
)
assert len(as_groups_response["AutoScalingGroups"]) == 1
asg = as_groups_response["AutoScalingGroups"][0]
asg_instance_ids = [instance["InstanceId"] for instance in asg["Instances"]]
instance_descriptions = get_cached_instances()
# The instances launched less than 10 minutes ago
immune_ids = [
instance["InstanceId"]
for instance in instance_descriptions.values()
if start - instance["LaunchTime"].timestamp() < 600
]
# if the ASG's instance ID not in instance_descriptions, it's most probably
# is not cached yet, so we must mark it as immuned
immune_ids.extend(
iid for iid in asg_instance_ids if iid not in instance_descriptions
)
print("Time spent on the requests to AWS: ", time.time() - start)
runners = list_runners(access_token) runners = list_runners(access_token)
runner_ids = set(runner.name for runner in runners)
# We used to delete potential hosts to terminate from GitHub runners pool, # We used to delete potential hosts to terminate from GitHub runners pool,
# but the documentation states: # but the documentation states:
# --- Returning an instance first in the response data does not guarantee its termination # --- Returning an instance first in the response data does not guarantee its termination
@ -177,18 +116,23 @@ def main(access_token: str, event: dict) -> Dict[str, List[str]]:
total_to_kill += num_to_kill total_to_kill += num_to_kill
if num_to_kill > len(candidates): if num_to_kill > len(candidates):
raise Exception( raise Exception(
f"Required to kill {num_to_kill}, but have only {len(candidates)} candidates in AV {zone}" f"Required to kill {num_to_kill}, but have only {len(candidates)}"
f" candidates in AV {zone}"
) )
delete_for_av = [] # type: RunnerDescriptions delete_for_av = [] # type: RunnerDescriptions
for candidate in candidates: for candidate in candidates:
if candidate not in set(runner.name for runner in runners): if candidate in immune_ids:
print(
f"Candidate {candidate} started less than 10 minutes ago, won't touch a child"
)
break
if candidate not in runner_ids:
print( print(
f"Candidate {candidate} was not in runners list, simply delete it" f"Candidate {candidate} was not in runners list, simply delete it"
) )
instances_to_kill.append(candidate) instances_to_kill.append(candidate)
break
for candidate in candidates:
if len(delete_for_av) + len(instances_to_kill) == num_to_kill: if len(delete_for_av) + len(instances_to_kill) == num_to_kill:
break break
if candidate in instances_to_kill: if candidate in instances_to_kill:
@ -207,22 +151,18 @@ def main(access_token: str, event: dict) -> Dict[str, List[str]]:
if len(delete_for_av) < num_to_kill: if len(delete_for_av) < num_to_kill:
print( print(
f"Checked all candidates for av {zone}, get to delete {len(delete_for_av)}, but still cannot get required {num_to_kill}" f"Checked all candidates for av {zone}, get to delete "
f"{len(delete_for_av)}, but still cannot get required {num_to_kill}"
) )
instances_to_kill += [runner.name for runner in delete_for_av] instances_to_kill += [runner.name for runner in delete_for_av]
if len(instances_to_kill) < total_to_kill: if len(instances_to_kill) < total_to_kill:
print(f"Check other hosts from the same ASG {event['AutoScalingGroupName']}") for instance in asg_instance_ids:
client = boto3.client("autoscaling") if instance in immune_ids:
as_groups = client.describe_auto_scaling_groups( continue
AutoScalingGroupNames=[event["AutoScalingGroupName"]]
)
assert len(as_groups["AutoScalingGroups"]) == 1
asg = as_groups["AutoScalingGroups"][0]
for instance in asg["Instances"]:
for runner in runners: for runner in runners:
if runner.name == instance["InstanceId"] and not runner.busy: if runner.name == instance and not runner.busy:
print(f"Runner {runner.name} is not busy and can be deleted") print(f"Runner {runner.name} is not busy and can be deleted")
instances_to_kill.append(runner.name) instances_to_kill.append(runner.name)
@ -230,9 +170,9 @@ def main(access_token: str, event: dict) -> Dict[str, List[str]]:
print("Got enough instances to kill") print("Got enough instances to kill")
break break
print("Got instances to kill: ", ", ".join(instances_to_kill))
response = {"InstanceIDs": instances_to_kill} response = {"InstanceIDs": instances_to_kill}
print(response) print("Got instances to kill: ", response)
print("Time spent on the request: ", time.time() - start)
return response return response
@ -270,6 +210,8 @@ if __name__ == "__main__":
with open(args.private_key_path, "r") as key_file: with open(args.private_key_path, "r") as key_file:
private_key = key_file.read() private_key = key_file.read()
token = get_access_token_by_key_app(private_key, args.app_id)
sample_event = { sample_event = {
"AutoScalingGroupARN": "arn:aws:autoscaling:us-east-1:<account-id>:autoScalingGroup:d4738357-2d40-4038-ae7e-b00ae0227003:autoScalingGroupName/my-asg", "AutoScalingGroupARN": "arn:aws:autoscaling:us-east-1:<account-id>:autoScalingGroup:d4738357-2d40-4038-ae7e-b00ae0227003:autoScalingGroupName/my-asg",
"AutoScalingGroupName": "my-asg", "AutoScalingGroupName": "my-asg",
@ -314,14 +256,4 @@ if __name__ == "__main__":
"Cause": "SCALE_IN", "Cause": "SCALE_IN",
} }
payload = { main(token, sample_event)
"iat": int(time.time()) - 60,
"exp": int(time.time()) + (10 * 60),
"iss": args.app_id,
}
encoded_jwt = jwt.encode(payload, private_key, algorithm="RS256")
installation_id = get_installation_id(encoded_jwt)
access_token = get_access_token(encoded_jwt, args.app_id)
main(access_token, sample_event)

View File

@ -0,0 +1 @@
../lambda_shared_package/lambda_shared

View File

@ -1,3 +1 @@
requests<2.30 ../lambda_shared_package[token]
PyJWT
cryptography<38

View File

@ -1 +0,0 @@
#!/usr/bin/env python

View File

@ -5,9 +5,10 @@ import fnmatch
import json import json
import time import time
import jwt
import requests # type: ignore import requests # type: ignore
import boto3 # type: ignore
from lambda_shared.pr import TRUSTED_CONTRIBUTORS
from lambda_shared.token import get_cached_access_token
SUSPICIOUS_CHANGED_FILES_NUMBER = 200 SUSPICIOUS_CHANGED_FILES_NUMBER = 200
@ -67,108 +68,6 @@ NEED_RERUN_WORKFLOWS = {
"ReleaseBranchCI", "ReleaseBranchCI",
} }
# Individual trusted contirbutors who are not in any trusted organization.
# Can be changed in runtime: we will append users that we learned to be in
# a trusted org, to save GitHub API calls.
TRUSTED_CONTRIBUTORS = {
e.lower()
for e in [
"achimbab",
"adevyatova ", # DOCSUP
"Algunenano", # Raúl Marín, Tinybird
"amosbird",
"AnaUvarova", # DOCSUP
"anauvarova", # technical writer, Yandex
"annvsh", # technical writer, Yandex
"atereh", # DOCSUP
"azat",
"bharatnc", # Newbie, but already with many contributions.
"bobrik", # Seasoned contributor, CloudFlare
"BohuTANG",
"codyrobert", # Flickerbox engineer
"cwurm", # Employee
"damozhaeva", # DOCSUP
"den-crane",
"flickerbox-tom", # Flickerbox
"gyuton", # DOCSUP
"hagen1778", # Roman Khavronenko, seasoned contributor
"hczhcz",
"hexiaoting", # Seasoned contributor
"ildus", # adjust, ex-pgpro
"javisantana", # a Spanish ClickHouse enthusiast, ex-Carto
"ka1bi4", # DOCSUP
"kirillikoff", # DOCSUP
"kreuzerkrieg",
"lehasm", # DOCSUP
"michon470", # DOCSUP
"nikvas0",
"nvartolomei",
"olgarev", # DOCSUP
"otrazhenia", # Yandex docs contractor
"pdv-ru", # DOCSUP
"podshumok", # cmake expert from QRator Labs
"s-mx", # Maxim Sabyanin, former employee, present contributor
"sevirov", # technical writer, Yandex
"spongedu", # Seasoned contributor
"taiyang-li",
"ucasFL", # Amos Bird's friend
"vdimir", # Employee
"vzakaznikov",
"YiuRULE",
"zlobober", # Developer of YT
"ilejn", # Arenadata, responsible for Kerberized Kafka
"thomoco", # ClickHouse
"BoloniniD", # Seasoned contributor, HSE
"tonickkozlov", # Cloudflare
"tylerhannan", # ClickHouse Employee
"myrrc", # Mike Kot, DoubleCloud
"thevar1able", # ClickHouse Employee
"aalexfvk",
"MikhailBurdukov",
"tsolodov", # ClickHouse Employee
"kitaisreal",
]
}
def get_installation_id(jwt_token):
headers = {
"Authorization": f"Bearer {jwt_token}",
"Accept": "application/vnd.github.v3+json",
}
response = requests.get("https://api.github.com/app/installations", headers=headers)
response.raise_for_status()
data = response.json()
for installation in data:
if installation["account"]["login"] == "ClickHouse":
installation_id = installation["id"]
return installation_id
def get_access_token(jwt_token, installation_id):
headers = {
"Authorization": f"Bearer {jwt_token}",
"Accept": "application/vnd.github.v3+json",
}
response = requests.post(
f"https://api.github.com/app/installations/{installation_id}/access_tokens",
headers=headers,
)
response.raise_for_status()
data = response.json()
return data["token"]
def get_key_and_app_from_aws():
secret_name = "clickhouse_github_secret_key"
session = boto3.session.Session()
client = session.client(
service_name="secretsmanager",
)
get_secret_value_response = client.get_secret_value(SecretId=secret_name)
data = json.loads(get_secret_value_response["SecretString"])
return data["clickhouse-app-key"], int(data["clickhouse-app-id"])
def is_trusted_contributor(pr_user_login, pr_user_orgs): def is_trusted_contributor(pr_user_login, pr_user_orgs):
if pr_user_login.lower() in TRUSTED_CONTRIBUTORS: if pr_user_login.lower() in TRUSTED_CONTRIBUTORS:
@ -331,19 +230,6 @@ def label_manual_approve(pull_request, token):
_exec_post_with_retry(url, token, data) _exec_post_with_retry(url, token, data)
def get_token_from_aws():
private_key, app_id = get_key_and_app_from_aws()
payload = {
"iat": int(time.time()) - 60,
"exp": int(time.time()) + (10 * 60),
"iss": app_id,
}
encoded_jwt = jwt.encode(payload, private_key, algorithm="RS256")
installation_id = get_installation_id(encoded_jwt)
return get_access_token(encoded_jwt, installation_id)
def get_workflow_jobs(workflow_description, token): def get_workflow_jobs(workflow_description, token):
jobs_url = ( jobs_url = (
workflow_description.api_url + f"/attempts/{workflow_description.attempt}/jobs" workflow_description.api_url + f"/attempts/{workflow_description.attempt}/jobs"
@ -443,7 +329,7 @@ def check_workflow_completed(
def main(event): def main(event):
token = get_token_from_aws() token = get_cached_access_token()
event_data = json.loads(event["body"]) event_data = json.loads(event["body"])
print("The body received:", event["body"]) print("The body received:", event["body"])
workflow_description = get_workflow_description_from_event(event_data) workflow_description = get_workflow_description_from_event(event_data)

View File

@ -0,0 +1 @@
../lambda_shared_package/lambda_shared

View File

@ -1,3 +1 @@
requests<2.30 ../lambda_shared_package[token]
PyJWT
cryptography<38

View File

@ -10,13 +10,11 @@ fields for private repositories
from base64 import b64decode from base64 import b64decode
from dataclasses import dataclass from dataclasses import dataclass
from typing import Any, List from typing import Any, List, Optional
import json import json
import logging import logging
import time
import boto3 # type: ignore from lambda_shared import ClickHouseHelper, InsertException, get_parameter_from_ssm
import requests # type: ignore
logging.getLogger().setLevel(logging.INFO) logging.getLogger().setLevel(logging.INFO)
@ -66,137 +64,7 @@ class WorkflowJob:
return self.__dict__ return self.__dict__
### VENDORING CH_CLIENT = None # type: Optional[ClickHouseHelper]
def get_parameter_from_ssm(name, decrypt=True, client=None):
if not client:
client = boto3.client("ssm", region_name="us-east-1")
return client.get_parameter(Name=name, WithDecryption=decrypt)["Parameter"]["Value"]
class InsertException(Exception):
pass
class ClickHouseHelper:
def __init__(self, url=None):
if url is None:
url = get_parameter_from_ssm("clickhouse-test-stat-url")
self.url = url
self.auth = {
"X-ClickHouse-User": get_parameter_from_ssm("clickhouse-test-stat-login"),
"X-ClickHouse-Key": get_parameter_from_ssm("clickhouse-test-stat-password"),
}
@staticmethod
def _insert_json_str_info_impl(url, auth, db, table, json_str):
params = {
"database": db,
"query": f"INSERT INTO {table} FORMAT JSONEachRow",
"date_time_input_format": "best_effort",
"send_logs_level": "warning",
}
for i in range(5):
try:
response = requests.post(
url, params=params, data=json_str, headers=auth
)
except Exception as e:
error = f"Received exception while sending data to {url} on {i} attempt: {e}"
logging.warning(error)
continue
logging.info("Response content '%s'", response.content)
if response.ok:
break
error = (
"Cannot insert data into clickhouse at try "
+ str(i)
+ ": HTTP code "
+ str(response.status_code)
+ ": '"
+ str(response.text)
+ "'"
)
if response.status_code >= 500:
# A retriable error
time.sleep(1)
continue
logging.info(
"Request headers '%s', body '%s'",
response.request.headers,
response.request.body,
)
raise InsertException(error)
else:
raise InsertException(error)
def _insert_json_str_info(self, db, table, json_str):
self._insert_json_str_info_impl(self.url, self.auth, db, table, json_str)
def insert_event_into(self, db, table, event, safe=True):
event_str = json.dumps(event)
try:
self._insert_json_str_info(db, table, event_str)
except InsertException as e:
logging.error(
"Exception happened during inserting data into clickhouse: %s", e
)
if not safe:
raise
def insert_events_into(self, db, table, events, safe=True):
jsons = []
for event in events:
jsons.append(json.dumps(event))
try:
self._insert_json_str_info(db, table, ",".join(jsons))
except InsertException as e:
logging.error(
"Exception happened during inserting data into clickhouse: %s", e
)
if not safe:
raise
def _select_and_get_json_each_row(self, db, query):
params = {
"database": db,
"query": query,
"default_format": "JSONEachRow",
}
for i in range(5):
response = None
try:
response = requests.get(self.url, params=params, headers=self.auth)
response.raise_for_status()
return response.text
except Exception as ex:
logging.warning("Cannot insert with exception %s", str(ex))
if response:
logging.warning("Reponse text %s", response.text)
time.sleep(0.1 * i)
raise Exception("Cannot fetch data from clickhouse")
def select_json_each_row(self, db, query):
text = self._select_and_get_json_each_row(db, query)
result = []
for line in text.split("\n"):
if line:
result.append(json.loads(line))
return result
### VENDORING END
clickhouse_client = ClickHouseHelper()
def send_event_workflow_job(workflow_job: WorkflowJob) -> None: def send_event_workflow_job(workflow_job: WorkflowJob) -> None:
@ -232,23 +100,30 @@ def send_event_workflow_job(workflow_job: WorkflowJob) -> None:
# PARTITION BY toStartOfMonth(started_at) # PARTITION BY toStartOfMonth(started_at)
# ORDER BY (id, updated_at) # ORDER BY (id, updated_at)
# SETTINGS index_granularity = 8192 # SETTINGS index_granularity = 8192
global clickhouse_client global CH_CLIENT
kwargs = { CH_CLIENT = CH_CLIENT or ClickHouseHelper(
"db": "default", get_parameter_from_ssm("clickhouse-test-stat-url"),
"table": "workflow_jobs", get_parameter_from_ssm("clickhouse-test-stat-login"),
"event": workflow_job.as_dict(), get_parameter_from_ssm("clickhouse-test-stat-password"),
"safe": False, )
}
try: try:
clickhouse_client.insert_event_into(**kwargs) CH_CLIENT.insert_event_into(
"default", "workflow_jobs", workflow_job.as_dict(), False
)
except InsertException as ex: except InsertException as ex:
logging.exception( logging.exception(
"Got an exception on insert, tryuing to update the client " "Got an exception on insert, tryuing to update the client "
"credentials and repeat", "credentials and repeat",
exc_info=ex, exc_info=ex,
) )
clickhouse_client = ClickHouseHelper() CH_CLIENT = ClickHouseHelper(
clickhouse_client.insert_event_into(**kwargs) get_parameter_from_ssm("clickhouse-test-stat-url"),
get_parameter_from_ssm("clickhouse-test-stat-login"),
get_parameter_from_ssm("clickhouse-test-stat-password"),
)
CH_CLIENT.insert_event_into(
"default", "workflow_jobs", workflow_job.as_dict(), False
)
def handler(event: dict, context: Any) -> dict: def handler(event: dict, context: Any) -> dict:
@ -257,6 +132,7 @@ def handler(event: dict, context: Any) -> dict:
else: else:
event_data = json.loads(event["body"]) event_data = json.loads(event["body"])
logging.info("Got the next raw event from the github hook: %s", event_data)
repo = event_data["repository"] repo = event_data["repository"]
try: try:
wf_job = event_data["workflow_job"] wf_job = event_data["workflow_job"]
@ -265,6 +141,9 @@ def handler(event: dict, context: Any) -> dict:
logging.error("The event data: %s", event) logging.error("The event data: %s", event)
logging.error("The context data: %s", context) logging.error("The context data: %s", context)
# We record only finished steps
steps = len([step for step in wf_job["steps"] if step["conclusion"] is not None])
workflow_job = WorkflowJob( workflow_job = WorkflowJob(
wf_job["id"], wf_job["id"],
wf_job["run_id"], wf_job["run_id"],
@ -281,7 +160,7 @@ def handler(event: dict, context: Any) -> dict:
wf_job["started_at"], wf_job["started_at"],
wf_job["completed_at"] or "1970-01-01T00:00:00", # nullable date wf_job["completed_at"] or "1970-01-01T00:00:00", # nullable date
wf_job["name"], wf_job["name"],
len(wf_job["steps"]), steps,
wf_job["check_run_url"], wf_job["check_run_url"],
wf_job["labels"], wf_job["labels"],
wf_job["runner_id"] or 0, # nullable wf_job["runner_id"] or 0, # nullable

View File

@ -0,0 +1 @@
../lambda_shared_package/lambda_shared

View File

@ -1 +1 @@
requests<2.30 ../lambda_shared_package

View File

@ -12,6 +12,7 @@
<disk>s3</disk> <disk>s3</disk>
<max_size>100000000</max_size> <max_size>100000000</max_size>
<path>./cache_s3/</path> <path>./cache_s3/</path>
<cache_on_write_operations>1</cache_on_write_operations>
</cache_s3> </cache_s3>
</disks> </disks>
<policies> <policies>

View File

@ -0,0 +1,7 @@
<clickhouse>
<profiles>
<default>
<enable_filesystem_cache_on_write_operations>1</enable_filesystem_cache_on_write_operations>
</default>
</profiles>
</clickhouse>

View File

@ -19,6 +19,7 @@ def cluster():
cluster.add_instance( cluster.add_instance(
"node1", "node1",
main_configs=["configs/config.d/storage_conf.xml"], main_configs=["configs/config.d/storage_conf.xml"],
user_configs=["configs/config.d/users.xml"],
macros={"replica": "1"}, macros={"replica": "1"},
with_minio=True, with_minio=True,
with_zookeeper=True, with_zookeeper=True,
@ -26,12 +27,14 @@ def cluster():
cluster.add_instance( cluster.add_instance(
"node2", "node2",
main_configs=["configs/config.d/storage_conf.xml"], main_configs=["configs/config.d/storage_conf.xml"],
user_configs=["configs/config.d/users.xml"],
macros={"replica": "2"}, macros={"replica": "2"},
with_zookeeper=True, with_zookeeper=True,
) )
cluster.add_instance( cluster.add_instance(
"node3", "node3",
main_configs=["configs/config.d/storage_conf.xml"], main_configs=["configs/config.d/storage_conf.xml"],
user_configs=["configs/config.d/users.xml"],
macros={"replica": "3"}, macros={"replica": "3"},
with_zookeeper=True, with_zookeeper=True,
) )
@ -74,7 +77,7 @@ def generate_values(date_str, count, sign=1):
def create_table(cluster, additional_settings=None): def create_table(cluster, additional_settings=None):
create_table_statement = """ create_table_statement = """
CREATE TABLE s3_test ON CLUSTER cluster( CREATE TABLE s3_test ON CLUSTER cluster (
dt Date, dt Date,
id Int64, id Int64,
data String, data String,
@ -95,7 +98,8 @@ def create_table(cluster, additional_settings=None):
def drop_table(cluster): def drop_table(cluster):
yield yield
for node in list(cluster.instances.values()): for node in list(cluster.instances.values()):
node.query("DROP TABLE IF EXISTS s3_test") node.query("DROP TABLE IF EXISTS s3_test SYNC")
node.query("DROP TABLE IF EXISTS test_drop_table SYNC")
minio = cluster.minio_client minio = cluster.minio_client
# Remove extra objects to prevent tests cascade failing # Remove extra objects to prevent tests cascade failing

View File

@ -1,6 +1,6 @@
DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS mt;
CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS max_part_removal_threads = 16, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, old_parts_lifetime = 1, parts_to_delay_insert = 100000, parts_to_throw_insert = 100000; CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, old_parts_lifetime = 1, parts_to_delay_insert = 100000, parts_to_throw_insert = 100000;
SYSTEM STOP MERGES mt; SYSTEM STOP MERGES mt;

View File

@ -2,7 +2,7 @@
DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS mt;
CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS max_part_loading_threads = 16, parts_to_delay_insert = 100000, parts_to_throw_insert = 100000; CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS parts_to_delay_insert = 100000, parts_to_throw_insert = 100000;
SYSTEM STOP MERGES mt; SYSTEM STOP MERGES mt;

View File

@ -6,6 +6,7 @@ Code: 516
1 1
Code: 516 Code: 516
processes processes
processes
Code: 81 Code: 81
[1] [1]
Code: 73 Code: 73

View File

@ -11,6 +11,9 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh . "$CUR_DIR"/../shell_config.sh
# The number of threads removing data parts should be between 1 and 129.
# Because max_parts_cleaning_thread_pool_size is 128 by default
$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -nm -q "create database ordinary_$CLICKHOUSE_DATABASE engine=Ordinary" $CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -nm -q "create database ordinary_$CLICKHOUSE_DATABASE engine=Ordinary"
# MergeTree # MergeTree
@ -22,7 +25,7 @@ $CLICKHOUSE_CLIENT -nm -q """
Engine=MergeTree() Engine=MergeTree()
order by key order by key
partition by key%100 partition by key%100
settings max_part_removal_threads=10, concurrent_part_removal_threshold=99, min_bytes_for_wide_part=0; settings concurrent_part_removal_threshold=99, min_bytes_for_wide_part=0;
insert into data_01810 select * from numbers(100); insert into data_01810 select * from numbers(100);
drop table data_01810 settings log_queries=1; drop table data_01810 settings log_queries=1;
@ -30,7 +33,7 @@ $CLICKHOUSE_CLIENT -nm -q """
-- sometimes the same thread can be used to remove part, due to ThreadPool, -- sometimes the same thread can be used to remove part, due to ThreadPool,
-- hence we cannot compare strictly. -- hence we cannot compare strictly.
select throwIf(not(length(thread_ids) between 1 and 11)) select throwIf(not(length(thread_ids) between 1 and 129))
from system.query_log from system.query_log
where where
event_date >= yesterday() and event_date >= yesterday() and
@ -49,7 +52,7 @@ $CLICKHOUSE_CLIENT -nm -q """
Engine=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rep_data_01810', '1') Engine=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rep_data_01810', '1')
order by key order by key
partition by key%100 partition by key%100
settings max_part_removal_threads=10, concurrent_part_removal_threshold=99, min_bytes_for_wide_part=0; settings concurrent_part_removal_threshold=99, min_bytes_for_wide_part=0;
SET insert_keeper_max_retries=1000; SET insert_keeper_max_retries=1000;
SET insert_keeper_retry_max_backoff_ms=10; SET insert_keeper_retry_max_backoff_ms=10;
@ -60,7 +63,7 @@ $CLICKHOUSE_CLIENT -nm -q """
-- sometimes the same thread can be used to remove part, due to ThreadPool, -- sometimes the same thread can be used to remove part, due to ThreadPool,
-- hence we cannot compare strictly. -- hence we cannot compare strictly.
select throwIf(not(length(thread_ids) between 1 and 11)) select throwIf(not(length(thread_ids) between 1 and 129))
from system.query_log from system.query_log
where where
event_date >= yesterday() and event_date >= yesterday() and

View File

@ -8,7 +8,7 @@ drop table if exists rmt2;
-- Disable compact parts, because we need hardlinks in mutations. -- Disable compact parts, because we need hardlinks in mutations.
create table rmt (n int, m int, k int) engine=ReplicatedMergeTree('/test/02432/{database}', '1') order by tuple() create table rmt (n int, m int, k int) engine=ReplicatedMergeTree('/test/02432/{database}', '1') order by tuple()
settings storage_policy = 's3_cache', allow_remote_fs_zero_copy_replication=1, settings storage_policy = 's3_cache', allow_remote_fs_zero_copy_replication=1,
max_part_removal_threads=10, concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1, concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1,
max_replicated_merges_in_queue=0, max_replicated_mutations_in_queue=0, min_bytes_for_wide_part=0, min_rows_for_wide_part=0; max_replicated_merges_in_queue=0, max_replicated_mutations_in_queue=0, min_bytes_for_wide_part=0, min_rows_for_wide_part=0;
insert into rmt(n, m) values (1, 42); insert into rmt(n, m) values (1, 42);
@ -38,7 +38,7 @@ select count(), sum(n), sum(m) from rmt;
-- New table can assign merges/mutations and can remove old parts -- New table can assign merges/mutations and can remove old parts
create table rmt2 (n int, m int, k String) engine=ReplicatedMergeTree('/test/02432/{database}', '2') order by tuple() create table rmt2 (n int, m int, k String) engine=ReplicatedMergeTree('/test/02432/{database}', '2') order by tuple()
settings storage_policy = 's3_cache', allow_remote_fs_zero_copy_replication=1, settings storage_policy = 's3_cache', allow_remote_fs_zero_copy_replication=1,
max_part_removal_threads=10, concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1, concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1,
min_bytes_for_wide_part=0, min_rows_for_wide_part=0, max_replicated_merges_in_queue=1, min_bytes_for_wide_part=0, min_rows_for_wide_part=0, max_replicated_merges_in_queue=1,
old_parts_lifetime=0; old_parts_lifetime=0;
@ -66,4 +66,3 @@ drop table rmt2;
system flush logs; system flush logs;
select count() > 0 from system.text_log where yesterday() <= event_date and logger_name like '%' || currentDatabase() || '%' and message like '%Removing % parts from filesystem (concurrently): Parts:%'; select count() > 0 from system.text_log where yesterday() <= event_date and logger_name like '%' || currentDatabase() || '%' and message like '%Removing % parts from filesystem (concurrently): Parts:%';
select count() > 1, countDistinct(thread_id) > 1 from system.text_log where yesterday() <= event_date and logger_name like '%' || currentDatabase() || '%' and message like '%Removing % parts in blocks range%'; select count() > 1, countDistinct(thread_id) > 1 from system.text_log where yesterday() <= event_date and logger_name like '%' || currentDatabase() || '%' and message like '%Removing % parts in blocks range%';

View File

@ -0,0 +1,4 @@
0
0
default true true
2 2

View File

@ -0,0 +1,18 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
USER_POSTFIX=`random_str 10`
USER="test_user_02771_$USER_POSTFIX"
$CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS $USER"
$CLICKHOUSE_CLIENT -q "CREATE USER $USER"
$CLICKHOUSE_CLIENT -q "GRANT SELECT ON system.* TO $USER"
$CLICKHOUSE_CLIENT -u "$USER" -q "SELECT * FROM system.numbers LIMIT 1"
$CLICKHOUSE_CLIENT -u "$USER" -q "SELECT * FROM system.numbers LIMIT 1"
$CLICKHOUSE_CLIENT -q "SELECT user, toBool(ProfileEvents['SelectQuery'] > 0), toBool(ProfileEvents['Query'] > 0) FROM system.user_processes WHERE user='default'"
$CLICKHOUSE_CLIENT -q "SELECT ProfileEvents['SelectQuery'], ProfileEvents['Query'] FROM system.user_processes WHERE user='$USER'"
$CLICKHOUSE_CLIENT -q "DROP USER $USER"