Merge branch 'master' into signal_handlers_in_client

This commit is contained in:
Alexander Tokmakov 2024-07-16 13:15:06 +02:00
commit ee9b08a2b6
14 changed files with 121 additions and 38 deletions

View File

@ -3,8 +3,9 @@
#include <base/defines.h>
#include <fstream>
#include <sstream>
#include <string>
namespace fs = std::filesystem;
bool cgroupsV2Enabled()
{
@ -13,11 +14,11 @@ bool cgroupsV2Enabled()
{
/// This file exists iff the host has cgroups v2 enabled.
auto controllers_file = default_cgroups_mount / "cgroup.controllers";
if (!std::filesystem::exists(controllers_file))
if (!fs::exists(controllers_file))
return false;
return true;
}
catch (const std::filesystem::filesystem_error &) /// all "underlying OS API errors", typically: permission denied
catch (const fs::filesystem_error &) /// all "underlying OS API errors", typically: permission denied
{
return false; /// not logging the exception as most callers fall back to cgroups v1
}
@ -33,8 +34,9 @@ bool cgroupsV2MemoryControllerEnabled()
/// According to https://docs.kernel.org/admin-guide/cgroup-v2.html, file "cgroup.controllers" defines which controllers are available
/// for the current + child cgroups. The set of available controllers can be restricted from level to level using file
/// "cgroups.subtree_control". It is therefore sufficient to check the bottom-most nested "cgroup.controllers" file.
std::string cgroup = cgroupV2OfProcess();
auto cgroup_dir = cgroup.empty() ? default_cgroups_mount : (default_cgroups_mount / cgroup);
fs::path cgroup_dir = cgroupV2PathOfProcess();
if (cgroup_dir.empty())
return false;
std::ifstream controllers_file(cgroup_dir / "cgroup.controllers");
if (!controllers_file.is_open())
return false;
@ -46,7 +48,7 @@ bool cgroupsV2MemoryControllerEnabled()
#endif
}
std::string cgroupV2OfProcess()
fs::path cgroupV2PathOfProcess()
{
#if defined(OS_LINUX)
chassert(cgroupsV2Enabled());
@ -54,17 +56,18 @@ std::string cgroupV2OfProcess()
/// A simpler way to get the membership is:
std::ifstream cgroup_name_file("/proc/self/cgroup");
if (!cgroup_name_file.is_open())
return "";
return {};
/// With cgroups v2, there will be a *single* line with prefix "0::/"
/// (see https://docs.kernel.org/admin-guide/cgroup-v2.html)
std::string cgroup;
std::getline(cgroup_name_file, cgroup);
static const std::string v2_prefix = "0::/";
if (!cgroup.starts_with(v2_prefix))
return "";
return {};
cgroup = cgroup.substr(v2_prefix.length());
return cgroup;
/// Note: The 'root' cgroup can have an empty cgroup name, this is valid
return default_cgroups_mount / cgroup;
#else
return "";
return {};
#endif
}

View File

@ -1,7 +1,6 @@
#pragma once
#include <filesystem>
#include <string>
#if defined(OS_LINUX)
/// I think it is possible to mount the cgroups hierarchy somewhere else (e.g. when in containers).
@ -16,7 +15,7 @@ bool cgroupsV2Enabled();
/// Assumes that cgroupsV2Enabled() is enabled.
bool cgroupsV2MemoryControllerEnabled();
/// Which cgroup does the process belong to?
/// Returns an empty string if the cgroup cannot be determined.
/// Detects which cgroup v2 the process belongs to and returns the filesystem path to the cgroup.
/// Returns an empty path the cgroup cannot be determined.
/// Assumes that cgroupsV2Enabled() is enabled.
std::string cgroupV2OfProcess();
std::filesystem::path cgroupV2PathOfProcess();

View File

@ -23,8 +23,9 @@ std::optional<uint64_t> getCgroupsV2MemoryLimit()
if (!cgroupsV2MemoryControllerEnabled())
return {};
std::string cgroup = cgroupV2OfProcess();
auto current_cgroup = cgroup.empty() ? default_cgroups_mount : (default_cgroups_mount / cgroup);
std::filesystem::path current_cgroup = cgroupV2PathOfProcess();
if (current_cgroup.empty())
return {};
/// Open the bottom-most nested memory limit setting file. If there is no such file at the current
/// level, try again at the parent level as memory settings are inherited.

View File

@ -49,6 +49,7 @@ void logAboutProgress(LoggerPtr log, size_t processed, size_t total, AtomicStopw
AsyncLoader::Pool::Pool(const AsyncLoader::PoolInitializer & init)
: name(init.name)
, priority(init.priority)
, max_threads(init.max_threads > 0 ? init.max_threads : getNumberOfPhysicalCPUCores())
, thread_pool(std::make_unique<ThreadPool>(
init.metric_threads,
init.metric_active_threads,
@ -56,17 +57,16 @@ AsyncLoader::Pool::Pool(const AsyncLoader::PoolInitializer & init)
/* max_threads = */ std::numeric_limits<size_t>::max(), // Unlimited number of threads, we do worker management ourselves
/* max_free_threads = */ 0, // We do not require free threads
/* queue_size = */0)) // Unlimited queue to avoid blocking during worker spawning
, max_threads(init.max_threads > 0 ? init.max_threads : getNumberOfPhysicalCPUCores())
{}
AsyncLoader::Pool::Pool(Pool&& o) noexcept
: name(o.name)
, priority(o.priority)
, thread_pool(std::move(o.thread_pool))
, ready_queue(std::move(o.ready_queue))
, max_threads(o.max_threads)
, workers(o.workers)
, suspended_workers(o.suspended_workers.load()) // All these constructors are needed because std::atomic is neither copy-constructible, nor move-constructible. We never move pools after init, so it is safe.
, thread_pool(std::move(o.thread_pool))
{}
void cancelOnDependencyFailure(const LoadJobPtr & self, const LoadJobPtr & dependency, std::exception_ptr & cancel)

View File

@ -365,11 +365,11 @@ private:
{
const String name;
const Priority priority;
std::unique_ptr<ThreadPool> thread_pool; // NOTE: we avoid using a `ThreadPool` queue to be able to move jobs between pools.
std::map<UInt64, LoadJobPtr> ready_queue; // FIFO queue of jobs to be executed in this pool. Map is used for faster erasing. Key is `ready_seqno`
size_t max_threads; // Max number of workers to be spawn
size_t workers = 0; // Number of currently executing workers
std::atomic<size_t> suspended_workers{0}; // Number of workers that are blocked by `wait()` call on a job executing in the same pool (for deadlock resolution)
std::unique_ptr<ThreadPool> thread_pool; // NOTE: we avoid using a `ThreadPool` queue to be able to move jobs between pools.
explicit Pool(const PoolInitializer & init);
Pool(Pool&& o) noexcept;

View File

@ -25,6 +25,7 @@
#endif
using namespace DB;
namespace fs = std::filesystem;
namespace DB
{
@ -69,7 +70,7 @@ uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, const std::string & ke
struct CgroupsV1Reader : ICgroupsReader
{
explicit CgroupsV1Reader(const std::filesystem::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { }
explicit CgroupsV1Reader(const fs::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { }
uint64_t readMemoryUsage() override
{
@ -85,7 +86,7 @@ private:
struct CgroupsV2Reader : ICgroupsReader
{
explicit CgroupsV2Reader(const std::filesystem::path & stat_file_dir)
explicit CgroupsV2Reader(const fs::path & stat_file_dir)
: current_buf(stat_file_dir / "memory.current"), stat_buf(stat_file_dir / "memory.stat")
{
}
@ -129,8 +130,9 @@ std::optional<std::string> getCgroupsV2Path()
if (!cgroupsV2MemoryControllerEnabled())
return {};
String cgroup = cgroupV2OfProcess();
auto current_cgroup = cgroup.empty() ? default_cgroups_mount : (default_cgroups_mount / cgroup);
fs::path current_cgroup = cgroupV2PathOfProcess();
if (current_cgroup.empty())
return {};
/// Return the bottom-most nested current memory file. If there is no such file at the current
/// level, try again at the parent level as memory settings are inherited.
@ -138,7 +140,7 @@ std::optional<std::string> getCgroupsV2Path()
{
const auto current_path = current_cgroup / "memory.current";
const auto stat_path = current_cgroup / "memory.stat";
if (std::filesystem::exists(current_path) && std::filesystem::exists(stat_path))
if (fs::exists(current_path) && fs::exists(stat_path))
return {current_cgroup};
current_cgroup = current_cgroup.parent_path();
}
@ -148,7 +150,7 @@ std::optional<std::string> getCgroupsV2Path()
std::optional<std::string> getCgroupsV1Path()
{
auto path = default_cgroups_mount / "memory/memory.stat";
if (!std::filesystem::exists(path))
if (!fs::exists(path))
return {};
return {default_cgroups_mount / "memory"};
}

View File

@ -37,12 +37,12 @@ uint32_t getCGroupLimitedCPUCores(unsigned default_cpu_count)
/// cgroupsv2
if (cgroupsV2Enabled())
{
/// First, we identify the cgroup the process belongs
std::string cgroup = cgroupV2OfProcess();
if (cgroup.empty())
/// First, we identify the path of the cgroup the process belongs
std::filesystem::path cgroup_path = cgroupV2PathOfProcess();
if (cgroup_path.empty())
return default_cpu_count;
auto current_cgroup = cgroup.empty() ? default_cgroups_mount : (default_cgroups_mount / cgroup);
auto current_cgroup = cgroup_path;
// Looking for cpu.max in directories from the current cgroup to the top level
// It does not stop on the first time since the child could have a greater value than parent
@ -62,7 +62,7 @@ uint32_t getCGroupLimitedCPUCores(unsigned default_cpu_count)
}
current_cgroup = current_cgroup.parent_path();
}
current_cgroup = default_cgroups_mount / cgroup;
current_cgroup = cgroup_path;
// Looking for cpuset.cpus.effective in directories from the current cgroup to the top level
while (current_cgroup != default_cgroups_mount.parent_path())
{

View File

@ -226,10 +226,11 @@ void EmbeddedRocksDBBulkSink::consume(Chunk chunk_)
if (chunks_to_write.empty())
return;
size_t num_chunks = chunks_to_write.size();
auto [serialized_key_column, serialized_value_column]
= storage.ttl > 0 ? serializeChunks<true>(std::move(chunks_to_write)) : serializeChunks<false>(std::move(chunks_to_write));
auto sst_file_path = getTemporarySSTFilePath();
LOG_DEBUG(getLogger("EmbeddedRocksDBBulkSink"), "Writing {} rows to SST file {}", serialized_key_column->size(), sst_file_path);
LOG_DEBUG(getLogger("EmbeddedRocksDBBulkSink"), "Writing {} rows from {} chunks to SST file {}", serialized_key_column->size(), num_chunks, sst_file_path);
if (auto status = buildSSTFile(sst_file_path, *serialized_key_column, *serialized_value_column); !status.ok())
throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString());

View File

@ -420,7 +420,7 @@ String transformQueryForExternalDatabase(
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "No column names for query '{}' to external table '{}.{}'",
query_info.query_tree->formatASTForErrorMessage(), database, table);
auto clone_query = getASTForExternalDatabaseFromQueryTree(query_info.query_tree);
auto clone_query = getASTForExternalDatabaseFromQueryTree(query_info.query_tree, query_info.table_expression);
return transformQueryForExternalDatabaseImpl(
clone_query,

View File

@ -3,6 +3,7 @@
#include <Storages/transformQueryForExternalDatabaseAnalyzer.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Columns/ColumnConst.h>
@ -10,7 +11,7 @@
#include <Analyzer/QueryNode.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/ConstantValue.h>
#include <Analyzer/JoinNode.h>
#include <DataTypes/DataTypesNumber.h>
@ -20,6 +21,7 @@ namespace DB
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
extern const int LOGICAL_ERROR;
}
namespace
@ -55,7 +57,7 @@ public:
}
ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tree)
ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tree, const QueryTreeNodePtr & table_expression)
{
auto new_tree = query_tree->clone();
@ -63,6 +65,21 @@ ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tre
visitor.visit(new_tree);
const auto * query_node = new_tree->as<QueryNode>();
const auto & join_tree = query_node->getJoinTree();
bool allow_where = true;
if (const auto * join_node = join_tree->as<JoinNode>())
{
if (join_node->getStrictness() != JoinStrictness::All)
allow_where = false;
if (join_node->getKind() == JoinKind::Left)
allow_where = join_node->getLeftTableExpression()->isEqual(*table_expression);
else if (join_node->getKind() == JoinKind::Right)
allow_where = join_node->getRightTableExpression()->isEqual(*table_expression);
else
allow_where = (join_node->getKind() == JoinKind::Inner);
}
auto query_node_ast = query_node->toAST({ .add_cast_for_constants = false, .fully_qualified_identifiers = false });
const IAST * ast = query_node_ast.get();
@ -76,7 +93,13 @@ ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tre
if (union_ast->list_of_selects->children.size() != 1)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "QueryNode AST is not a single ASTSelectQuery, got {}", union_ast->list_of_selects->children.size());
return union_ast->list_of_selects->children.at(0);
ASTPtr select_query = union_ast->list_of_selects->children.at(0);
auto * select_query_typed = select_query->as<ASTSelectQuery>();
if (!select_query_typed)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected ASTSelectQuery, got {}", select_query ? select_query->formatForErrorMessage() : "nullptr");
if (!allow_where)
select_query_typed->setExpression(ASTSelectQuery::Expression::WHERE, nullptr);
return select_query;
}
}

View File

@ -6,6 +6,6 @@
namespace DB
{
ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tree);
ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tree, const QueryTreeNodePtr & table_expression);
}

View File

@ -834,6 +834,60 @@ def test_literal_escaping(started_cluster):
cursor.execute(f"DROP TABLE escaping")
def test_filter_pushdown(started_cluster):
cursor = started_cluster.postgres_conn.cursor()
cursor.execute("CREATE SCHEMA test_filter_pushdown")
cursor.execute(
"CREATE TABLE test_filter_pushdown.test_table (id integer, value integer)"
)
cursor.execute(
"INSERT INTO test_filter_pushdown.test_table VALUES (1, 10), (1, 110), (2, 0), (3, 33), (4, 0)"
)
node1.query(
"""
CREATE TABLE test_filter_pushdown_pg_table (id UInt32, value UInt32)
ENGINE PostgreSQL('postgres1:5432', 'postgres', 'test_table', 'postgres', 'mysecretpassword', 'test_filter_pushdown');
"""
)
node1.query(
"""
CREATE TABLE test_filter_pushdown_local_table (id UInt32, value UInt32) ENGINE Memory AS SELECT * FROM test_filter_pushdown_pg_table
"""
)
node1.query(
"CREATE TABLE ch_table (id UInt32, pg_id UInt32) ENGINE MergeTree ORDER BY id"
)
node1.query("INSERT INTO ch_table VALUES (1, 1), (2, 2), (3, 1), (4, 2), (5, 999)")
def compare_results(query, **kwargs):
result1 = node1.query(
query.format(pg_table="test_filter_pushdown_pg_table", **kwargs)
)
result2 = node1.query(
query.format(pg_table="test_filter_pushdown_local_table", **kwargs)
)
assert result1 == result2
for kind in ["INNER", "LEFT", "RIGHT", "FULL"]:
for value in [0, 10]:
compare_results(
"SELECT * FROM ch_table {kind} JOIN {pg_table} as p ON ch_table.pg_id = p.id WHERE value = {value} ORDER BY ALL",
kind=kind,
value=value,
)
compare_results(
"SELECT * FROM {pg_table} as p {kind} JOIN ch_table ON ch_table.pg_id = p.id WHERE value = {value} ORDER BY ALL",
kind=kind,
value=value,
)
cursor.execute("DROP SCHEMA test_filter_pushdown CASCADE")
if __name__ == "__main__":
cluster.start()
input("Cluster created, press any key to destroy...")

View File

@ -3,7 +3,7 @@
1000
1
1000
2
1
1000000
1000
0 999001

View File

@ -29,7 +29,7 @@ ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;"
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;"
# Must set both max_threads and max_insert_threads to 2 to make sure there is only two sinks
${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000) SETTINGS max_threads = 2, max_insert_threads = 2, max_block_size = 10000, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, insert_deduplication_token = '', optimize_trivial_insert_select = 1;"
${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 2 because default bulk sink size is ~1M rows / SST file
${CLICKHOUSE_CLIENT} --query "SELECT sum(value) IN (1, 2) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be not more than 2 because default bulk sink size is ~1M rows / SST file.
${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;"
# Testing insert with duplicated keys