Parallel replicas. Part [2] (#43772)

This commit is contained in:
Nikita Mikhaylov 2023-02-03 14:34:18 +01:00 committed by GitHub
parent d5117f2aa6
commit 33877b5e00
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
119 changed files with 3019 additions and 1668 deletions

View File

@ -2813,6 +2813,217 @@ jobs:
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
sudo rm -fr "$TEMP_PATH"
# Parallel replicas
FunctionalStatefulTestDebugParallelReplicas:
needs: [BuilderDebDebug]
runs-on: [self-hosted, func-tester]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/stateful_debug
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=Stateful tests (debug, ParallelReplicas)
REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse
KILL_TIMEOUT=3600
EOF
- name: Download json reports
uses: actions/download-artifact@v2
with:
path: ${{ env.REPORTS_PATH }}
- name: Clear repository
run: |
sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE"
- name: Check out repository code
uses: actions/checkout@v2
- name: Functional test
run: |
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci"
python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT"
- name: Cleanup
if: always()
run: |
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
sudo rm -fr "$TEMP_PATH"
FunctionalStatefulTestUBsanParallelReplicas:
needs: [BuilderDebUBsan]
runs-on: [self-hosted, func-tester]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/stateful_ubsan
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=Stateful tests (ubsan, ParallelReplicas)
REPO_COPY=${{runner.temp}}/stateful_ubsan/ClickHouse
KILL_TIMEOUT=3600
EOF
- name: Download json reports
uses: actions/download-artifact@v2
with:
path: ${{ env.REPORTS_PATH }}
- name: Clear repository
run: |
sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE"
- name: Check out repository code
uses: actions/checkout@v2
- name: Functional test
run: |
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci"
python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT"
- name: Cleanup
if: always()
run: |
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
sudo rm -fr "$TEMP_PATH"
FunctionalStatefulTestMsanParallelReplicas:
needs: [BuilderDebMsan]
runs-on: [self-hosted, func-tester]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/stateful_msan
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=Stateful tests (msan, ParallelReplicas)
REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse
KILL_TIMEOUT=3600
EOF
- name: Download json reports
uses: actions/download-artifact@v2
with:
path: ${{ env.REPORTS_PATH }}
- name: Clear repository
run: |
sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE"
- name: Check out repository code
uses: actions/checkout@v2
- name: Functional test
run: |
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci"
python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT"
- name: Cleanup
if: always()
run: |
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
sudo rm -fr "$TEMP_PATH"
FunctionalStatefulTestTsanParallelReplicas:
needs: [BuilderDebTsan]
runs-on: [self-hosted, func-tester]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/stateful_tsan
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=Stateful tests (tsan, ParallelReplicas)
REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse
KILL_TIMEOUT=3600
EOF
- name: Download json reports
uses: actions/download-artifact@v2
with:
path: ${{ env.REPORTS_PATH }}
- name: Clear repository
run: |
sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE"
- name: Check out repository code
uses: actions/checkout@v2
- name: Functional test
run: |
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci"
python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT"
- name: Cleanup
if: always()
run: |
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
sudo rm -fr "$TEMP_PATH"
FunctionalStatefulTestAsanParallelReplicas:
needs: [BuilderDebAsan]
runs-on: [self-hosted, func-tester]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/stateful_debug
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=Stateful tests (asan, ParallelReplicas)
REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse
KILL_TIMEOUT=3600
EOF
- name: Download json reports
uses: actions/download-artifact@v2
with:
path: ${{ env.REPORTS_PATH }}
- name: Clear repository
run: |
sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE"
- name: Check out repository code
uses: actions/checkout@v2
- name: Functional test
run: |
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci"
python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT"
- name: Cleanup
if: always()
run: |
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
sudo rm -fr "$TEMP_PATH"
FunctionalStatefulTestReleaseParallelReplicas:
needs: [BuilderDebRelease]
runs-on: [self-hosted, func-tester]
steps:
- name: Set envs
run: |
cat >> "$GITHUB_ENV" << 'EOF'
TEMP_PATH=${{runner.temp}}/stateful_release
REPORTS_PATH=${{runner.temp}}/reports_dir
CHECK_NAME=Stateful tests (release, ParallelReplicas)
REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse
KILL_TIMEOUT=3600
EOF
- name: Download json reports
uses: actions/download-artifact@v2
with:
path: ${{ env.REPORTS_PATH }}
- name: Clear repository
run: |
sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE"
- name: Check out repository code
uses: actions/checkout@v2
- name: Functional test
run: |
sudo rm -fr "$TEMP_PATH"
mkdir -p "$TEMP_PATH"
cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH"
cd "$REPO_COPY/tests/ci"
python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT"
- name: Cleanup
if: always()
run: |
docker ps --quiet | xargs --no-run-if-empty docker kill ||:
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
sudo rm -fr "$TEMP_PATH"
##############################################################################################
######################################### STRESS TESTS #######################################
##############################################################################################

View File

@ -126,13 +126,16 @@ function run_tests()
fi
set +e
clickhouse-test -j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --hung-check --print-time \
--skip 00168_parallel_processing_on_replicas "${ADDITIONAL_OPTIONS[@]}" \
if [[ -n "$USE_PARALLEL_REPLICAS" ]] && [[ "$USE_PARALLEL_REPLICAS" -eq 1 ]]; then
clickhouse-test --client="clickhouse-client --use_hedged_requests=0 --allow_experimental_parallel_reading_from_replicas=1 \
--max_parallel_replicas=100 --cluster_for_parallel_replicas='parallel_replicas'" \
-j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --no-parallel-replicas --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \
"$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt
clickhouse-test --timeout 1200 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --hung-check --print-time \
00168_parallel_processing_on_replicas "${ADDITIONAL_OPTIONS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee -a test_output/test_result.txt
else
clickhouse-test -j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \
"$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt
fi
set -e
}

View File

@ -134,9 +134,9 @@ function run_tests()
set +e
clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \
--test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \
| ts '%Y-%m-%d %H:%M:%S' \
| tee -a test_output/test_result.txt
--test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \
| ts '%Y-%m-%d %H:%M:%S' \
| tee -a test_output/test_result.txt
set -e
}

View File

@ -474,7 +474,7 @@ private:
executor.sendQuery(ClientInfo::QueryKind::INITIAL_QUERY);
ProfileInfo info;
while (Block block = executor.read())
while (Block block = executor.readBlock())
info.update(block);
executor.finish();

View File

@ -2040,7 +2040,7 @@ UInt64 ClusterCopier::executeQueryOnCluster(
while (true)
{
auto block = remote_query_executor->read();
auto block = remote_query_executor->readBlock();
if (!block)
break;
}

View File

@ -854,6 +854,51 @@
</replica>
</shard-->
</test_cluster_one_shard_three_replicas_localhost>
<parallel_replicas>
<shard>
<internal_replication>false</internal_replication>
<replica>
<host>127.0.0.1</host>
<port>9000</port>
</replica>
<replica>
<host>127.0.0.2</host>
<port>9000</port>
</replica>
<replica>
<host>127.0.0.3</host>
<port>9000</port>
</replica>
<replica>
<host>127.0.0.4</host>
<port>9000</port>
</replica>
<replica>
<host>127.0.0.5</host>
<port>9000</port>
</replica>
<replica>
<host>127.0.0.6</host>
<port>9000</port>
</replica>
<replica>
<host>127.0.0.7</host>
<port>9000</port>
</replica>
<replica>
<host>127.0.0.8</host>
<port>9000</port>
</replica>
<replica>
<host>127.0.0.9</host>
<port>9000</port>
</replica>
<replica>
<host>127.0.0.10</host>
<port>9000</port>
</replica>
</shard>
</parallel_replicas>
<test_cluster_two_shards_localhost>
<shard>
<replica>

View File

@ -343,8 +343,8 @@ set_source_files_properties(
PROPERTIES COMPILE_FLAGS "-mwaitpkg")
endif ()
target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::re2_st)
target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::re2)
target_link_libraries(common PUBLIC ch_contrib::re2_st)
target_link_libraries(common PUBLIC ch_contrib::re2)
target_link_libraries(clickhouse_common_io
PUBLIC

View File

@ -686,7 +686,7 @@ void Connection::sendReadTaskResponse(const String & response)
}
void Connection::sendMergeTreeReadTaskResponse(const PartitionReadResponse & response)
void Connection::sendMergeTreeReadTaskResponse(const ParallelReadResponse & response)
{
writeVarUInt(Protocol::Client::MergeTreeReadTaskResponse, *out);
response.serialize(*out);
@ -960,8 +960,12 @@ Packet Connection::receivePacket()
case Protocol::Server::ReadTaskRequest:
return res;
case Protocol::Server::MergeTreeAllRangesAnnounecement:
res.announcement = receiveInitialParallelReadAnnounecement();
return res;
case Protocol::Server::MergeTreeReadTaskRequest:
res.request = receivePartitionReadRequest();
res.request = receiveParallelReadRequest();
return res;
case Protocol::Server::ProfileEvents:
@ -1114,13 +1118,20 @@ ProfileInfo Connection::receiveProfileInfo() const
return profile_info;
}
PartitionReadRequest Connection::receivePartitionReadRequest() const
ParallelReadRequest Connection::receiveParallelReadRequest() const
{
PartitionReadRequest request;
ParallelReadRequest request;
request.deserialize(*in);
return request;
}
InitialAllRangesAnnouncement Connection::receiveInitialParallelReadAnnounecement() const
{
InitialAllRangesAnnouncement announcement;
announcement.deserialize(*in);
return announcement;
}
void Connection::throwUnexpectedPacket(UInt64 packet_type, const char * expected) const
{

View File

@ -110,7 +110,7 @@ public:
void sendData(const Block & block, const String & name/* = "" */, bool scalar/* = false */) override;
void sendMergeTreeReadTaskResponse(const PartitionReadResponse & response) override;
void sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) override;
void sendExternalTablesData(ExternalTablesData & data) override;
@ -265,7 +265,8 @@ private:
std::vector<String> receiveMultistringMessage(UInt64 msg_type) const;
std::unique_ptr<Exception> receiveException() const;
Progress receiveProgress() const;
PartitionReadRequest receivePartitionReadRequest() const;
ParallelReadRequest receiveParallelReadRequest() const;
InitialAllRangesAnnouncement receiveInitialParallelReadAnnounecement() const;
ProfileInfo receiveProfileInfo() const;
void initInputBuffers();

View File

@ -94,7 +94,7 @@ public:
throw Exception(ErrorCodes::LOGICAL_ERROR, "sendReadTaskResponse in not supported with HedgedConnections");
}
void sendMergeTreeReadTaskResponse(PartitionReadResponse) override
void sendMergeTreeReadTaskResponse(const ParallelReadResponse &) override
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "sendMergeTreeReadTaskResponse in not supported with HedgedConnections");
}

View File

@ -34,7 +34,7 @@ public:
bool with_pending_data) = 0;
virtual void sendReadTaskResponse(const String &) = 0;
virtual void sendMergeTreeReadTaskResponse(PartitionReadResponse response) = 0;
virtual void sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) = 0;
/// Get packet from any replica.
virtual Packet receivePacket() = 0;
@ -60,9 +60,9 @@ public:
/// Get the replica addresses as a string.
virtual std::string dumpAddresses() const = 0;
struct ReplicaInfo
{
bool collaborate_with_initiator{false};
size_t all_replicas_count{0};
size_t number_of_current_replica{0};
};

View File

@ -33,8 +33,10 @@ struct Packet
Progress progress;
ProfileInfo profile_info;
std::vector<UUID> part_uuids;
PartitionReadRequest request;
PartitionReadResponse response;
InitialAllRangesAnnouncement announcement;
ParallelReadRequest request;
ParallelReadResponse response;
Packet() : type(Protocol::Server::Hello) {}
};
@ -104,7 +106,7 @@ public:
/// Send all contents of external (temporary) tables.
virtual void sendExternalTablesData(ExternalTablesData & data) = 0;
virtual void sendMergeTreeReadTaskResponse(const PartitionReadResponse & response) = 0;
virtual void sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) = 0;
/// Check, if has data to read.
virtual bool poll(size_t timeout_microseconds) = 0;

View File

@ -508,7 +508,7 @@ void LocalConnection::sendExternalTablesData(ExternalTablesData &)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented");
}
void LocalConnection::sendMergeTreeReadTaskResponse(const PartitionReadResponse &)
void LocalConnection::sendMergeTreeReadTaskResponse(const ParallelReadResponse &)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented");
}

View File

@ -110,7 +110,7 @@ public:
void sendExternalTablesData(ExternalTablesData &) override;
void sendMergeTreeReadTaskResponse(const PartitionReadResponse & response) override;
void sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) override;
bool poll(size_t timeout_microseconds/* = 0 */) override;

View File

@ -133,16 +133,11 @@ void MultiplexedConnections::sendQuery(
modified_settings.group_by_two_level_threshold_bytes = 0;
}
bool parallel_reading_from_replicas = settings.max_parallel_replicas > 1
&& settings.allow_experimental_parallel_reading_from_replicas
/// To avoid trying to coordinate with clickhouse-benchmark,
/// since it uses the same code.
&& client_info.query_kind != ClientInfo::QueryKind::INITIAL_QUERY;
if (parallel_reading_from_replicas)
if (replica_info)
{
client_info.collaborate_with_initiator = true;
client_info.count_participating_replicas = replica_info.all_replicas_count;
client_info.number_of_current_replica = replica_info.number_of_current_replica;
client_info.count_participating_replicas = replica_info->all_replicas_count;
client_info.number_of_current_replica = replica_info->number_of_current_replica;
}
}
@ -199,7 +194,7 @@ void MultiplexedConnections::sendReadTaskResponse(const String & response)
}
void MultiplexedConnections::sendMergeTreeReadTaskResponse(PartitionReadResponse response)
void MultiplexedConnections::sendMergeTreeReadTaskResponse(const ParallelReadResponse & response)
{
std::lock_guard lock(cancel_mutex);
if (cancelled)
@ -263,6 +258,7 @@ Packet MultiplexedConnections::drain()
switch (packet.type)
{
case Protocol::Server::MergeTreeAllRangesAnnounecement:
case Protocol::Server::MergeTreeReadTaskRequest:
case Protocol::Server::ReadTaskRequest:
case Protocol::Server::PartUUIDs:
@ -343,6 +339,7 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac
switch (packet.type)
{
case Protocol::Server::MergeTreeAllRangesAnnounecement:
case Protocol::Server::MergeTreeReadTaskRequest:
case Protocol::Server::ReadTaskRequest:
case Protocol::Server::PartUUIDs:

View File

@ -42,7 +42,7 @@ public:
bool with_pending_data) override;
void sendReadTaskResponse(const String &) override;
void sendMergeTreeReadTaskResponse(PartitionReadResponse response) override;
void sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) override;
Packet receivePacket() override;
@ -104,7 +104,8 @@ private:
bool sent_query = false;
bool cancelled = false;
ReplicaInfo replica_info;
/// std::nullopt if parallel reading from replicas is not used
std::optional<ReplicaInfo> replica_info;
/// A mutex for the sendCancel function to execute safely
/// in separate thread.

View File

@ -81,7 +81,8 @@ namespace Protocol
/// This is such an inverted logic, where server sends requests
/// And client returns back response
ProfileEvents = 14, /// Packet with profile events from server.
MergeTreeReadTaskRequest = 15, /// Request from a MergeTree replica to a coordinator
MergeTreeAllRangesAnnounecement = 15,
MergeTreeReadTaskRequest = 16, /// Request from a MergeTree replica to a coordinator
MAX = MergeTreeReadTaskRequest,
};
@ -108,6 +109,7 @@ namespace Protocol
"PartUUIDs",
"ReadTaskRequest",
"ProfileEvents",
"MergeTreeAllRangesAnnounecement",
"MergeTreeReadTaskRequest",
};
return packet <= MAX

View File

@ -33,6 +33,8 @@
#define DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION 1
#define DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS 54453
#define DBMS_MERGE_TREE_PART_INFO_VERSION 1
/// Minimum revision supporting interserver secret.
#define DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET 54441

View File

@ -151,7 +151,9 @@ class IColumn;
M(UInt64, parallel_replicas_count, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the number of parallel replicas participating in query processing.", 0) \
M(UInt64, parallel_replica_offset, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas.", 0) \
\
M(String, cluster_for_parallel_replicas, "default", "Cluster for a shard in which current server is located", 0) \
M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table. It will work for any kind on MergeTree table.", 0) \
M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \
\
M(Bool, skip_unavailable_shards, false, "If true, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \
\

View File

@ -28,6 +28,7 @@ namespace Poco
{
namespace Util
{
/// NOLINTNEXTLINE(cppcoreguidelines-virtual-class-destructor)
class AbstractConfiguration;
}
}

View File

@ -7,7 +7,10 @@
#include <Common/ProfileEvents.h>
#include <Common/checkStackSize.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <IO/ConnectionTimeoutsContext.h>
#include <Interpreters/AddDefaultDatabaseVisitor.h>
#include <Interpreters/RequiredSourceColumnsVisitor.h>
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
#include <DataTypes/ObjectUtils.h>
#include <Client/IConnections.h>
@ -36,6 +39,53 @@ namespace ErrorCodes
namespace ClusterProxy
{
/// select query has database, table and table function names as AST pointers
/// Creates a copy of query, changes database, table and table function names.
ASTPtr rewriteSelectQuery(
ContextPtr context,
const ASTPtr & query,
const std::string & remote_database,
const std::string & remote_table,
ASTPtr table_function_ptr)
{
auto modified_query_ast = query->clone();
ASTSelectQuery & select_query = modified_query_ast->as<ASTSelectQuery &>();
// Get rid of the settings clause so we don't send them to remote. Thus newly non-important
// settings won't break any remote parser. It's also more reasonable since the query settings
// are written into the query context and will be sent by the query pipeline.
select_query.setExpression(ASTSelectQuery::Expression::SETTINGS, {});
if (table_function_ptr)
select_query.addTableFunction(table_function_ptr);
else
select_query.replaceDatabaseAndTable(remote_database, remote_table);
/// Restore long column names (cause our short names are ambiguous).
/// TODO: aliased table functions & CREATE TABLE AS table function cases
if (!table_function_ptr)
{
RestoreQualifiedNamesVisitor::Data data;
data.distributed_table = DatabaseAndTableWithAlias(*getTableExpression(query->as<ASTSelectQuery &>(), 0));
data.remote_table.database = remote_database;
data.remote_table.table = remote_table;
RestoreQualifiedNamesVisitor(data).visit(modified_query_ast);
}
/// To make local JOIN works, default database should be added to table names.
/// But only for JOIN section, since the following should work using default_database:
/// - SELECT * FROM d WHERE value IN (SELECT l.value FROM l) ORDER BY value
/// (see 01487_distributed_in_not_default_db)
AddDefaultDatabaseVisitor visitor(context, context->getCurrentDatabase(),
/* only_replace_current_database_function_= */false,
/* only_replace_in_join_= */true);
visitor.visit(modified_query_ast);
return modified_query_ast;
}
SelectStreamFactory::SelectStreamFactory(
const Block & header_,
const ColumnsDescriptionByShardNum & objects_by_shard_,
@ -171,67 +221,5 @@ void SelectStreamFactory::createForShard(
}
void SelectStreamFactory::createForShardWithParallelReplicas(
const Cluster::ShardInfo & shard_info,
const ASTPtr & query_ast,
const StorageID & main_table,
ContextPtr context,
UInt32 shard_count,
std::vector<QueryPlanPtr> & local_plans,
Shards & remote_shards)
{
if (auto it = objects_by_shard.find(shard_info.shard_num); it != objects_by_shard.end())
replaceMissedSubcolumnsByConstants(storage_snapshot->object_columns, it->second, query_ast);
const auto & settings = context->getSettingsRef();
auto is_local_replica_obsolete = [&]()
{
auto resolved_id = context->resolveStorageID(main_table);
auto main_table_storage = DatabaseCatalog::instance().tryGetTable(resolved_id, context);
const auto * replicated_storage = dynamic_cast<const StorageReplicatedMergeTree *>(main_table_storage.get());
if (!replicated_storage)
return false;
UInt64 max_allowed_delay = settings.max_replica_delay_for_distributed_queries;
if (!max_allowed_delay)
return false;
UInt64 local_delay = replicated_storage->getAbsoluteDelay();
return local_delay >= max_allowed_delay;
};
size_t next_replica_number = 0;
size_t all_replicas_count = shard_info.getRemoteNodeCount();
auto coordinator = std::make_shared<ParallelReplicasReadingCoordinator>();
if (settings.prefer_localhost_replica && shard_info.isLocal())
{
/// We don't need more than one local replica in parallel reading
if (!is_local_replica_obsolete())
{
++all_replicas_count;
local_plans.emplace_back(createLocalPlan(
query_ast, header, context, processed_stage, shard_info.shard_num, shard_count, next_replica_number, all_replicas_count, coordinator));
++next_replica_number;
}
}
if (shard_info.hasRemoteConnections())
remote_shards.emplace_back(Shard{
.query = query_ast,
.header = header,
.shard_info = shard_info,
.lazy = false,
.local_delay = 0,
.coordinator = coordinator,
});
}
}
}

View File

@ -29,6 +29,14 @@ struct StorageID;
namespace ClusterProxy
{
/// select query has database, table and table function names as AST pointers
/// Creates a copy of query, changes database, table and table function names.
ASTPtr rewriteSelectQuery(
ContextPtr context,
const ASTPtr & query,
const std::string & remote_database,
const std::string & remote_table,
ASTPtr table_function_ptr = nullptr);
using ColumnsDescriptionByShardNum = std::unordered_map<UInt32, ColumnsDescription>;
@ -80,16 +88,6 @@ public:
std::unique_ptr<QueryPlan> remote_plan;
};
void createForShardWithParallelReplicas(
const Cluster::ShardInfo & shard_info,
const ASTPtr & query_ast,
const StorageID & main_table,
ContextPtr context,
UInt32 shard_count,
std::vector<QueryPlanPtr> & local_plans,
Shards & remote_shards);
private:
const Block header;
const ColumnsDescriptionByShardNum objects_by_shard;
const StorageSnapshotPtr storage_snapshot;

View File

@ -1,6 +1,8 @@
#include <Core/QueryProcessingStage.h>
#include <Core/Settings.h>
#include <Core/UUID.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/ObjectUtils.h>
#include <Interpreters/Cluster.h>
#include <Interpreters/ClusterProxy/SelectStreamFactory.h>
#include <Interpreters/ClusterProxy/executeQuery.h>
@ -13,8 +15,11 @@
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/ReadFromRemote.h>
#include <Processors/QueryPlan/UnionStep.h>
#include <Processors/QueryPlan/DistributedCreateLocalPlan.h>
#include <Processors/ResizeProcessor.h>
#include <QueryPipeline/Pipe.h>
#include <Storages/SelectQueryInfo.h>
#include <Storages/StorageReplicatedMergeTree.h>
namespace DB
{
@ -23,6 +28,7 @@ namespace ErrorCodes
{
extern const int TOO_LARGE_DISTRIBUTED_DEPTH;
extern const int LOGICAL_ERROR;
extern const int SUPPORT_IS_DISABLED;
}
namespace ClusterProxy
@ -117,6 +123,31 @@ ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr c
return new_context;
}
static ThrottlerPtr getThrottler(const ContextPtr & context)
{
const Settings & settings = context->getSettingsRef();
ThrottlerPtr user_level_throttler;
if (auto process_list_element = context->getProcessListElement())
user_level_throttler = process_list_element->getUserNetworkThrottler();
/// Network bandwidth limit, if needed.
ThrottlerPtr throttler;
if (settings.max_network_bandwidth || settings.max_network_bytes)
{
throttler = std::make_shared<Throttler>(
settings.max_network_bandwidth,
settings.max_network_bytes,
"Limit for bytes to send or receive over network exceeded.",
user_level_throttler);
}
else
throttler = user_level_throttler;
return throttler;
}
void executeQuery(
QueryPlan & query_plan,
const Block & header,
@ -138,26 +169,8 @@ void executeQuery(
SelectStreamFactory::Shards remote_shards;
auto new_context = updateSettingsForCluster(*query_info.getCluster(), context, settings, main_table, &query_info, log);
new_context->getClientInfo().distributed_depth += 1;
ThrottlerPtr user_level_throttler;
if (auto process_list_element = context->getProcessListElement())
user_level_throttler = process_list_element->getUserNetworkThrottler();
/// Network bandwidth limit, if needed.
ThrottlerPtr throttler;
if (settings.max_network_bandwidth || settings.max_network_bytes)
{
throttler = std::make_shared<Throttler>(
settings.max_network_bandwidth,
settings.max_network_bytes,
"Limit for bytes to send or receive over network exceeded.",
user_level_throttler);
}
else
throttler = user_level_throttler;
size_t shards = query_info.getCluster()->getShardCount();
for (const auto & shard_info : query_info.getCluster()->getShardsInfo())
{
@ -199,7 +212,7 @@ void executeQuery(
main_table,
table_func_ptr,
new_context,
throttler,
getThrottler(context),
std::move(scalars),
std::move(external_tables),
log,
@ -236,103 +249,76 @@ void executeQueryWithParallelReplicas(
const StorageID & main_table,
const ASTPtr & table_func_ptr,
SelectStreamFactory & stream_factory,
const ASTPtr & query_ast,
ContextPtr context,
const SelectQueryInfo & query_info,
const ExpressionActionsPtr & sharding_key_expr,
const std::string & sharding_key_column_name,
const ClusterPtr & not_optimized_cluster,
QueryProcessingStage::Enum processed_stage)
const ASTPtr & query_ast, ContextPtr context, const SelectQueryInfo & query_info,
const ClusterPtr & not_optimized_cluster)
{
const Settings & settings = context->getSettingsRef();
if (not_optimized_cluster->getShardsInfo().size() != 1)
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Cluster for parallel replicas should consist only from one shard");
ThrottlerPtr user_level_throttler;
if (auto process_list_element = context->getProcessListElement())
user_level_throttler = process_list_element->getUserNetworkThrottler();
auto shard_info = not_optimized_cluster->getShardsInfo().front();
/// Network bandwidth limit, if needed.
ThrottlerPtr throttler;
if (settings.max_network_bandwidth || settings.max_network_bytes)
{
throttler = std::make_shared<Throttler>(
settings.max_network_bandwidth,
settings.max_network_bytes,
"Limit for bytes to send or receive over network exceeded.",
user_level_throttler);
}
else
throttler = user_level_throttler;
std::vector<QueryPlanPtr> plans;
SelectStreamFactory::Shards remote_shards;
size_t shards = query_info.getCluster()->getShardCount();
for (const auto & shard_info : query_info.getCluster()->getShardsInfo())
{
ASTPtr query_ast_for_shard;
if (query_info.optimized_cluster && settings.optimize_skip_unused_shards_rewrite_in && shards > 1)
{
query_ast_for_shard = query_ast->clone();
OptimizeShardingKeyRewriteInVisitor::Data visitor_data{
sharding_key_expr,
sharding_key_expr->getSampleBlock().getByPosition(0).type,
sharding_key_column_name,
shard_info,
not_optimized_cluster->getSlotToShard(),
};
OptimizeShardingKeyRewriteInVisitor visitor(visitor_data);
visitor.visit(query_ast_for_shard);
}
else
query_ast_for_shard = query_ast;
stream_factory.createForShardWithParallelReplicas(
shard_info, query_ast_for_shard, main_table, context, static_cast<UInt32>(shards), plans, remote_shards);
}
Scalars scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{};
scalars.emplace(
"_shard_count", Block{{DataTypeUInt32().createColumnConst(1, shards), std::make_shared<DataTypeUInt32>(), "_shard_count"}});
auto external_tables = context->getExternalTables();
if (!remote_shards.empty())
{
auto new_context = Context::createCopy(context);
for (const auto & shard : remote_shards)
{
auto read_from_remote = std::make_unique<ReadFromParallelRemoteReplicasStep>(
shard.coordinator,
shard,
shard.header,
processed_stage,
main_table,
table_func_ptr,
new_context,
throttler,
scalars,
external_tables,
&Poco::Logger::get("ReadFromParallelRemoteReplicasStep"),
query_info.storage_limits);
auto remote_plan = std::make_unique<QueryPlan>();
remote_plan->addStep(std::move(read_from_remote));
remote_plan->addInterpreterContext(new_context);
plans.emplace_back(std::move(remote_plan));
}
}
if (plans.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "No plans were generated for reading from Distributed. This is a bug");
if (plans.size() == 1)
const auto & settings = context->getSettingsRef();
auto all_replicas_count = std::min(static_cast<size_t>(settings.max_parallel_replicas), shard_info.all_addresses.size());
auto coordinator = std::make_shared<ParallelReplicasReadingCoordinator>(all_replicas_count);
auto remote_plan = std::make_unique<QueryPlan>();
auto plans = std::vector<QueryPlanPtr>();
/// This is a little bit weird, but we construct an "empty" coordinator without
/// any specified reading/coordination method (like Default, InOrder, InReverseOrder)
/// Because we will understand it later during QueryPlan optimization
/// So we place a reference to the coordinator to some common plane like QueryInfo
/// to then tell it about the reading method we chose.
query_info.coordinator = coordinator;
UUID parallel_group_id = UUIDHelpers::generateV4();
plans.emplace_back(createLocalPlan(
query_ast,
stream_factory.header,
context,
stream_factory.processed_stage,
shard_info.shard_num,
/*shard_count*/1,
0,
all_replicas_count,
coordinator,
parallel_group_id));
if (!shard_info.hasRemoteConnections())
{
if (!plans.front())
throw Exception(ErrorCodes::LOGICAL_ERROR, "An empty plan was generated to read from local shard and there is no remote connections. This is a bug");
query_plan = std::move(*plans.front());
return;
}
auto new_context = Context::createCopy(context);
auto scalars = new_context->hasQueryContext() ? new_context->getQueryContext()->getScalars() : Scalars{};
auto external_tables = new_context->getExternalTables();
auto read_from_remote = std::make_unique<ReadFromParallelRemoteReplicasStep>(
query_ast,
std::move(shard_info),
coordinator,
stream_factory.header,
stream_factory.processed_stage,
main_table,
table_func_ptr,
new_context,
getThrottler(new_context),
std::move(scalars),
std::move(external_tables),
&Poco::Logger::get("ReadFromParallelRemoteReplicasStep"),
query_info.storage_limits,
parallel_group_id);
remote_plan->addStep(std::move(read_from_remote));
remote_plan->addInterpreterContext(context);
plans.emplace_back(std::move(remote_plan));
if (std::all_of(plans.begin(), plans.end(), [](const QueryPlanPtr & plan) { return !plan; }))
throw Exception(ErrorCodes::LOGICAL_ERROR, "No plans were generated for reading from shard. This is a bug");
DataStreams input_streams;
input_streams.reserve(plans.size());
for (const auto & plan : plans)

View File

@ -61,10 +61,7 @@ void executeQueryWithParallelReplicas(
const ASTPtr & query_ast,
ContextPtr context,
const SelectQueryInfo & query_info,
const ExpressionActionsPtr & sharding_key_expr,
const std::string & sharding_key_column_name,
const ClusterPtr & not_optimized_cluster,
QueryProcessingStage::Enum processed_stage);
const ClusterPtr & not_optimized_cluster);
}
}

View File

@ -3621,6 +3621,32 @@ void Context::setMergeTreeReadTaskCallback(MergeTreeReadTaskCallback && callback
merge_tree_read_task_callback = callback;
}
MergeTreeAllRangesCallback Context::getMergeTreeAllRangesCallback() const
{
if (!merge_tree_all_ranges_callback.has_value())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Next task callback is not set for query with id: {}", getInitialQueryId());
return merge_tree_all_ranges_callback.value();
}
void Context::setMergeTreeAllRangesCallback(MergeTreeAllRangesCallback && callback)
{
merge_tree_all_ranges_callback = callback;
}
void Context::setParallelReplicasGroupUUID(UUID uuid)
{
parallel_replicas_group_uuid = uuid;
}
UUID Context::getParallelReplicasGroupUUID() const
{
return parallel_replicas_group_uuid;
}
PartUUIDsPtr Context::getIgnoredPartUUIDs() const
{
auto lock = getLock();
@ -3886,4 +3912,22 @@ WriteSettings Context::getWriteSettings() const
return res;
}
bool Context::canUseParallelReplicasOnInitiator() const
{
const auto & settings = getSettingsRef();
return settings.allow_experimental_parallel_reading_from_replicas
&& settings.max_parallel_replicas > 1
&& !settings.use_hedged_requests
&& !getClientInfo().collaborate_with_initiator;
}
bool Context::canUseParallelReplicasOnFollower() const
{
const auto & settings = getSettingsRef();
return settings.allow_experimental_parallel_reading_from_replicas
&& settings.max_parallel_replicas > 1
&& !settings.use_hedged_requests
&& getClientInfo().collaborate_with_initiator;
}
}

View File

@ -1,5 +1,11 @@
#pragma once
#include <base/types.h>
#include <Common/isLocalAddress.h>
#include <Common/MultiVersion.h>
#include <Common/OpenTelemetryTraceContext.h>
#include <Common/RemoteHostFilter.h>
#include <Common/ThreadPool.h>
#include <Core/Block.h>
#include <Core/NamesAndTypes.h>
#include <Core/Settings.h>
@ -8,32 +14,24 @@
#include <Interpreters/Context_fwd.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/MergeTreeTransactionHolder.h>
#include <Parsers/IAST_fwd.h>
#include <Parsers/ASTSelectQuery.h>
#include <Storages/IStorage_fwd.h>
#include <Common/MultiVersion.h>
#include <Common/OpenTelemetryTraceContext.h>
#include <Common/RemoteHostFilter.h>
#include <Common/ThreadPool.h>
#include <Common/isLocalAddress.h>
#include <base/types.h>
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
#include <Storages/ColumnsDescription.h>
#include <IO/IResourceManager.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/IAST_fwd.h>
#include <Processors/ResizeProcessor.h>
#include <Processors/Transforms/ReadFromMergeTreeDependencyTransform.h>
#include <Server/HTTP/HTTPContext.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/IStorage_fwd.h>
#include "config.h"
#include <boost/container/flat_set.hpp>
#include <exception>
#include <functional>
#include <memory>
#include <mutex>
#include <optional>
#include <thread>
#include <exception>
namespace Poco::Net { class IPAddress; }
@ -98,7 +96,11 @@ class TransactionsInfoLog;
class ProcessorsProfileLog;
class FilesystemCacheLog;
class AsynchronousInsertLog;
class IAsynchronousReader;
struct MergeTreeSettings;
struct InitialAllRangesAnnouncement;
struct ParallelReadRequest;
struct ParallelReadResponse;
class StorageS3Settings;
class IDatabase;
class DDLWorker;
@ -172,11 +174,15 @@ using InputBlocksReader = std::function<Block(ContextPtr)>;
/// Used in distributed task processing
using ReadTaskCallback = std::function<String()>;
using MergeTreeReadTaskCallback = std::function<std::optional<PartitionReadResponse>(PartitionReadRequest)>;
using MergeTreeAllRangesCallback = std::function<void(InitialAllRangesAnnouncement)>;
using MergeTreeReadTaskCallback = std::function<std::optional<ParallelReadResponse>(ParallelReadRequest)>;
class TemporaryDataOnDiskScope;
using TemporaryDataOnDiskScopePtr = std::shared_ptr<TemporaryDataOnDiskScope>;
class ParallelReplicasReadingCoordinator;
using ParallelReplicasReadingCoordinatorPtr = std::shared_ptr<ParallelReplicasReadingCoordinator>;
#if USE_ROCKSDB
class MergeTreeMetadataCache;
using MergeTreeMetadataCachePtr = std::shared_ptr<MergeTreeMetadataCache>;
@ -262,6 +268,8 @@ private:
/// Used in parallel reading from replicas. A replica tells about its intentions to read
/// some ranges from some part and initiator will tell the replica about whether it is accepted or denied.
std::optional<MergeTreeReadTaskCallback> merge_tree_read_task_callback;
std::optional<MergeTreeAllRangesCallback> merge_tree_all_ranges_callback;
UUID parallel_replicas_group_uuid{UUIDHelpers::Nil};
/// Record entities accessed by current query, and store this information in system.query_log.
struct QueryAccessInfo
@ -380,6 +388,7 @@ private:
/// Temporary data for query execution accounting.
TemporaryDataOnDiskScopePtr temp_data_on_disk;
public:
/// Some counters for current query execution.
/// Most of them are workarounds and should be removed in the future.
@ -402,6 +411,8 @@ public:
KitchenSink kitchen_sink;
ParallelReplicasReadingCoordinatorPtr parallel_reading_coordinator;
private:
using SampleBlockCache = std::unordered_map<std::string, Block>;
mutable SampleBlockCache sample_block_cache;
@ -1045,6 +1056,12 @@ public:
MergeTreeReadTaskCallback getMergeTreeReadTaskCallback() const;
void setMergeTreeReadTaskCallback(MergeTreeReadTaskCallback && callback);
MergeTreeAllRangesCallback getMergeTreeAllRangesCallback() const;
void setMergeTreeAllRangesCallback(MergeTreeAllRangesCallback && callback);
UUID getParallelReplicasGroupUUID() const;
void setParallelReplicasGroupUUID(UUID uuid);
/// Background executors related methods
void initializeBackgroundExecutorsIfNeeded();
bool areBackgroundExecutorsInitialized();
@ -1071,6 +1088,10 @@ public:
/** Get settings for writing to filesystem. */
WriteSettings getWriteSettings() const;
/** There are multiple conditions that have to be met to be able to use parallel replicas */
bool canUseParallelReplicasOnInitiator() const;
bool canUseParallelReplicasOnFollower() const;
private:
std::unique_lock<std::recursive_mutex> getLock() const;

View File

@ -448,6 +448,16 @@ InterpreterSelectQuery::InterpreterSelectQuery(
}
}
/// FIXME: Memory bound aggregation may cause another reading algorithm to be used on remote replicas
if (settings.allow_experimental_parallel_reading_from_replicas && settings.enable_memory_bound_merging_of_aggregation_results)
context->setSetting("enable_memory_bound_merging_of_aggregation_results", false);
if (joined_tables.tablesCount() > 1 && settings.allow_experimental_parallel_reading_from_replicas)
{
LOG_WARNING(log, "Joins are not supported with parallel replicas. Query will be executed without using them.");
context->setSetting("allow_experimental_parallel_reading_from_replicas", false);
}
/// Rewrite JOINs
if (!has_input && joined_tables.tablesCount() > 1)
{
@ -543,6 +553,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
parameter_values,
parameter_types);
query_info.syntax_analyzer_result = syntax_analyzer_result;
context->setDistributed(syntax_analyzer_result->is_remote_storage);
@ -1902,22 +1913,6 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(
}
}
void InterpreterSelectQuery::setMergeTreeReadTaskCallbackAndClientInfo(MergeTreeReadTaskCallback && callback)
{
context->getClientInfo().collaborate_with_initiator = true;
context->setMergeTreeReadTaskCallback(std::move(callback));
}
void InterpreterSelectQuery::setProperClientInfo(size_t replica_num, size_t replica_count)
{
context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
context->getClientInfo().count_participating_replicas = replica_count;
context->getClientInfo().number_of_current_replica = replica_num;
context->getClientInfo().connection_client_version_major = DBMS_VERSION_MAJOR;
context->getClientInfo().connection_client_version_minor = DBMS_VERSION_MINOR;
context->getClientInfo().connection_tcp_protocol_version = DBMS_TCP_PROTOCOL_VERSION;
}
RowPolicyFilterPtr InterpreterSelectQuery::getRowPolicyFilter() const
{
return row_policy_filter;
@ -2572,12 +2567,13 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool
const bool should_produce_results_in_order_of_bucket_number = options.to_stage == QueryProcessingStage::WithMergeableState
&& (settings.distributed_aggregation_memory_efficient || settings.enable_memory_bound_merging_of_aggregation_results);
const bool parallel_replicas_from_merge_tree = storage->isMergeTree() && context->canUseParallelReplicasOnInitiator();
executeMergeAggregatedImpl(
query_plan,
overflow_row,
final,
storage && storage->isRemote(),
storage && (storage->isRemote() || parallel_replicas_from_merge_tree),
has_grouping_sets,
context->getSettingsRef(),
query_analyzer->aggregationKeys(),

View File

@ -122,16 +122,6 @@ public:
bool supportsTransactions() const override { return true; }
/// This is tiny crutch to support reading from localhost replica during distributed query
/// Replica need to talk to the initiator through a connection to ask for a next task
/// but there will be no connection if we create Interpreter explicitly.
/// The other problem is that context is copied inside Interpreter's constructor
/// And with this method we can change the internals of cloned one
void setMergeTreeReadTaskCallbackAndClientInfo(MergeTreeReadTaskCallback && callback);
/// It will set shard_num and shard_count to the client_info
void setProperClientInfo(size_t replica_num, size_t replica_count);
FilterDAGInfoPtr getAdditionalQueryInfo() const { return additional_filter_info; }
RowPolicyFilterPtr getRowPolicyFilter() const;

View File

@ -135,20 +135,4 @@ void InterpreterSelectQueryAnalyzer::addStorageLimits(const StorageLimitsList &
planner.addStorageLimits(storage_limits);
}
void InterpreterSelectQueryAnalyzer::setMergeTreeReadTaskCallbackAndClientInfo(MergeTreeReadTaskCallback && callback)
{
context->getClientInfo().collaborate_with_initiator = true;
context->setMergeTreeReadTaskCallback(std::move(callback));
}
void InterpreterSelectQueryAnalyzer::setProperClientInfo(size_t replica_number, size_t count_participating_replicas)
{
context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
context->getClientInfo().number_of_current_replica = replica_number;
context->getClientInfo().count_participating_replicas = count_participating_replicas;
context->getClientInfo().connection_client_version_major = DBMS_VERSION_MAJOR;
context->getClientInfo().connection_client_version_minor = DBMS_VERSION_MINOR;
context->getClientInfo().connection_tcp_protocol_version = DBMS_TCP_PROTOCOL_VERSION;
}
}

View File

@ -112,6 +112,8 @@ std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
subquery_options.removeDuplicates();
}
/// We don't want to execute reading for subqueries in parallel
subquery_context->setSetting("allow_experimental_parallel_reading_from_replicas", false);
return std::make_shared<InterpreterSelectWithUnionQuery>(query, subquery_context, subquery_options, required_source_columns);
}

View File

@ -1,9 +1,12 @@
#include <Processors/QueryPlan/DistributedCreateLocalPlan.h>
#include "config_version.h"
#include <Common/checkStackSize.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Core/ProtocolDefines.h>
#include <Interpreters/ActionsDAG.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Processors/QueryPlan/ExpressionStep.h>
namespace DB
{
@ -40,48 +43,58 @@ std::unique_ptr<QueryPlan> createLocalPlan(
const Block & header,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
UInt32 shard_num,
UInt32 shard_count,
size_t shard_num,
size_t shard_count,
size_t replica_num,
size_t replica_count,
std::shared_ptr<ParallelReplicasReadingCoordinator> coordinator)
std::shared_ptr<ParallelReplicasReadingCoordinator> coordinator,
UUID group_uuid)
{
checkStackSize();
auto query_plan = std::make_unique<QueryPlan>();
auto new_context = Context::createCopy(context);
/// Do not apply AST optimizations, because query
/// is already optimized and some optimizations
/// can be applied only for non-distributed tables
/// and we can produce query, inconsistent with remote plans.
auto select_query_options = SelectQueryOptions(processed_stage)
.setShardInfo(shard_num, shard_count)
.setShardInfo(static_cast<UInt32>(shard_num), static_cast<UInt32>(shard_count))
.ignoreASTOptimizations();
auto update_interpreter = [&](auto & interpreter)
/// There are much things that are needed for coordination
/// during reading with parallel replicas
if (coordinator)
{
interpreter.setProperClientInfo(replica_num, replica_count);
if (coordinator)
new_context->parallel_reading_coordinator = coordinator;
new_context->getClientInfo().interface = ClientInfo::Interface::LOCAL;
new_context->getClientInfo().collaborate_with_initiator = true;
new_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
new_context->getClientInfo().count_participating_replicas = replica_count;
new_context->getClientInfo().number_of_current_replica = replica_num;
new_context->getClientInfo().connection_client_version_major = DBMS_VERSION_MAJOR;
new_context->getClientInfo().connection_client_version_minor = DBMS_VERSION_MINOR;
new_context->getClientInfo().connection_tcp_protocol_version = DBMS_TCP_PROTOCOL_VERSION;
new_context->setParallelReplicasGroupUUID(group_uuid);
new_context->setMergeTreeAllRangesCallback([coordinator](InitialAllRangesAnnouncement announcement)
{
interpreter.setMergeTreeReadTaskCallbackAndClientInfo([coordinator](PartitionReadRequest request) -> std::optional<PartitionReadResponse>
{
return coordinator->handleRequest(request);
});
}
};
coordinator->handleInitialAllRangesAnnouncement(announcement);
});
new_context->setMergeTreeReadTaskCallback([coordinator](ParallelReadRequest request) -> std::optional<ParallelReadResponse>
{
return coordinator->handleRequest(request);
});
}
if (context->getSettingsRef().allow_experimental_analyzer)
{
auto interpreter = InterpreterSelectQueryAnalyzer(query_ast, context, select_query_options);
update_interpreter(interpreter);
auto interpreter = InterpreterSelectQueryAnalyzer(query_ast, new_context, select_query_options);
query_plan = std::make_unique<QueryPlan>(std::move(interpreter).extractQueryPlan());
}
else
{
auto interpreter = InterpreterSelectQuery(
query_ast, context,
select_query_options);
update_interpreter(interpreter);
auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options);
interpreter.buildQueryPlan(*query_plan);
}

View File

@ -3,6 +3,7 @@
#include <Core/QueryProcessingStage.h>
#include <Parsers/IAST_fwd.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/ResizeProcessor.h>
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
namespace DB
@ -13,10 +14,11 @@ std::unique_ptr<QueryPlan> createLocalPlan(
const Block & header,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
UInt32 shard_num,
UInt32 shard_count,
size_t shard_num,
size_t shard_count,
size_t replica_num,
size_t replica_count,
std::shared_ptr<ParallelReplicasReadingCoordinator> coordinator);
std::shared_ptr<ParallelReplicasReadingCoordinator> coordinator,
UUID group_uuid = UUIDHelpers::Nil);
}

View File

@ -1005,8 +1005,6 @@ InputOrderInfoPtr buildInputOrderInfo(SortingStep & sorting, QueryPlan::Node & n
if (auto * reading = typeid_cast<ReadFromMergeTree *>(reading_node->step.get()))
{
//std::cerr << "---- optimizeReadInOrder found mt" << std::endl;
auto order_info = buildInputOrderInfo(
reading,
fixed_columns,

View File

@ -1,8 +1,9 @@
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <Common/Exception.h>
#include <Processors/QueryPlan/MergingAggregatedStep.h>
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <Processors/QueryPlan/UnionStep.h>
#include <stack>
namespace DB

View File

@ -13,6 +13,7 @@
#include <Processors/QueryPlan/PartsSplitter.h>
#include <Processors/Transforms/FilterSortedStreamByRange.h>
#include <Storages/MergeTree/RangesInDataPart.h>
#include <Storages/MergeTree/IMergeTreeDataPart.h>
using namespace DB;
@ -77,7 +78,7 @@ std::pair<std::vector<Values>, std::vector<RangesInDataParts>> split(RangesInDat
RangeEnd,
};
bool operator<(const PartsRangesIterator & other) const { return std::tie(value, event) > std::tie(other.value, other.event); }
[[ maybe_unused ]] bool operator<(const PartsRangesIterator & other) const { return std::tie(value, event) > std::tie(other.value, other.event); }
Values value;
MarkRangeWithPartIdx range;

View File

@ -166,6 +166,7 @@ QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline(
QueryPipelineBuilderPtr last_pipeline;
std::stack<Frame> stack;
stack.push(Frame{.node = root});
@ -198,6 +199,13 @@ QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline(
last_pipeline->setProcessListElement(build_pipeline_settings.process_list_element);
last_pipeline->addResources(std::move(resources));
/// This is related to parallel replicas.
/// Not to let the remote sources starve for CPU we create an
/// explicit dependency between processors which read from local replica
/// and ones that receive data from remote replicas and constantly answer
/// to coordination packets.
last_pipeline->connectDependencies();
return last_pipeline;
}

View File

@ -1,16 +1,18 @@
#include <algorithm>
#include <functional>
#include <memory>
#include <numeric>
#include <queue>
#include <stdexcept>
#include <IO/Operators.h>
#include <Storages/MergeTree/MergeTreeReadPool.h>
#include <base/sort.h>
#include <Common/JSONBuilder.h>
#include <Common/logger_useful.h>
#include <Common/isLocalAddress.h>
#include "Storages/MergeTree/RequestResponse.h"
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/Context.h>
#include <IO/Operators.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSelectQuery.h>
#include <Poco/Logger.h>
#include <Processors/ConcatProcessor.h>
#include <Processors/Merges/AggregatingSortedTransform.h>
#include <Processors/Merges/CollapsingSortedTransform.h>
@ -25,18 +27,22 @@
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/FilterTransform.h>
#include <Processors/Transforms/ReverseTransform.h>
#include <Processors/Transforms/ReadFromMergeTreeDependencyTransform.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
#include <Storages/MergeTree/MergeTreeInOrderSelectProcessor.h>
#include <Storages/MergeTree/MergeTreeReadPool.h>
#include <Storages/MergeTree/MergeTreeReverseSelectProcessor.h>
#include <Storages/MergeTree/MergeTreeThreadSelectProcessor.h>
#include <Storages/MergeTree/MergeTreeSource.h>
#include <Storages/MergeTree/RangesInDataPart.h>
#include <Storages/VirtualColumnUtils.h>
#include <Common/logger_useful.h>
#include <base/sort.h>
#include <Poco/Logger.h>
#include <Common/JSONBuilder.h>
#include <algorithm>
#include <functional>
#include <memory>
#include <numeric>
#include <queue>
#include <stdexcept>
namespace ProfileEvents
{
@ -114,6 +120,7 @@ ReadFromMergeTree::ReadFromMergeTree(
, max_block_numbers_to_read(std::move(max_block_numbers_to_read_))
, log(log_)
, analyzed_result_ptr(analyzed_result_ptr_)
, is_parallel_reading_from_replicas(enable_parallel_reading)
{
if (sample_factor_column_queried)
{
@ -123,8 +130,11 @@ ReadFromMergeTree::ReadFromMergeTree(
output_stream->header.insert({type->createColumn(), type, "_sample_factor"});
}
if (enable_parallel_reading)
if (is_parallel_reading_from_replicas)
{
all_ranges_callback = context->getMergeTreeAllRangesCallback();
read_task_callback = context->getMergeTreeReadTaskCallback();
}
const auto & settings = context->getSettingsRef();
if (settings.max_streams_for_merge_tree_reading)
@ -173,6 +183,80 @@ ReadFromMergeTree::ReadFromMergeTree(
}
}
Pipe ReadFromMergeTree::readFromPoolParallelReplicas(
RangesInDataParts parts_with_range,
Names required_columns,
size_t max_streams,
size_t min_marks_for_concurrent_read,
bool use_uncompressed_cache
)
{
const auto & client_info = context->getClientInfo();
auto extension = ParallelReadingExtension
{
.all_callback = all_ranges_callback.value(),
.callback = read_task_callback.value(),
.count_participating_replicas = client_info.count_participating_replicas,
.number_of_current_replica = client_info.number_of_current_replica,
.colums_to_read = required_columns
};
/// We have a special logic for local replica. It has to read less data, because in some cases it should
/// merge states of aggregate functions or do some other important stuff other than reading from Disk.
auto is_local_replica = context->getClientInfo().interface == ClientInfo::Interface::LOCAL;
if (!is_local_replica)
min_marks_for_concurrent_read = static_cast<size_t>(min_marks_for_concurrent_read * context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier);
auto pool = std::make_shared<MergeTreeReadPoolParallelReplicas>(
storage_snapshot,
max_streams,
extension,
parts_with_range,
prewhere_info,
required_columns,
virt_column_names,
min_marks_for_concurrent_read
);
Pipes pipes;
const auto & settings = context->getSettingsRef();
size_t total_rows = parts_with_range.getRowsCountAllParts();
for (size_t i = 0; i < max_streams; ++i)
{
auto algorithm = std::make_unique<MergeTreeThreadSelectAlgorithm>(
i, pool, min_marks_for_concurrent_read, max_block_size,
settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes,
data, storage_snapshot, use_uncompressed_cache,
prewhere_info, actions_settings, reader_settings, virt_column_names);
auto source = std::make_shared<MergeTreeSource>(std::move(algorithm));
/// Set the approximate number of rows for the first source only
/// In case of parallel processing on replicas do not set approximate rows at all.
/// Because the value will be identical on every replicas and will be accounted
/// multiple times (settings.max_parallel_replicas times more)
if (i == 0 && !client_info.collaborate_with_initiator)
source->addTotalRowsApprox(total_rows);
pipes.emplace_back(std::move(source));
/// Add a special dependency transform which will be connected later with
/// all RemoteSources through a simple scheduler (ResizeProcessor)
if (context->getClientInfo().interface == ClientInfo::Interface::LOCAL)
{
pipes.back().addSimpleTransform([&](const Block & header) -> ProcessorPtr
{
return std::make_shared<ReadFromMergeTreeDependencyTransform>(header, context->getParallelReplicasGroupUUID());
});
}
}
return Pipe::unitePipes(std::move(pipes));
}
Pipe ReadFromMergeTree::readFromPool(
RangesInDataParts parts_with_range,
Names required_columns,
@ -181,22 +265,25 @@ Pipe ReadFromMergeTree::readFromPool(
bool use_uncompressed_cache)
{
Pipes pipes;
size_t sum_marks = 0;
size_t total_rows = 0;
for (const auto & part : parts_with_range)
{
sum_marks += part.getMarksCount();
total_rows += part.getRowsCount();
}
size_t sum_marks = parts_with_range.getMarksCountAllParts();
size_t total_rows = parts_with_range.getRowsCountAllParts();
if (query_info.limit > 0 && query_info.limit < total_rows)
total_rows = query_info.limit;
const auto & settings = context->getSettingsRef();
const auto & client_info = context->getClientInfo();
MergeTreeReadPool::BackoffSettings backoff_settings(settings);
/// round min_marks_to_read up to nearest multiple of block_size expressed in marks
/// If granularity is adaptive it doesn't make sense
/// Maybe it will make sense to add settings `max_block_size_bytes`
if (max_block_size && !data.canUseAdaptiveGranularity())
{
size_t fixed_index_granularity = data.getSettings()->index_granularity;
min_marks_for_concurrent_read = (min_marks_for_concurrent_read * fixed_index_granularity + max_block_size - 1)
/ max_block_size * max_block_size / fixed_index_granularity;
}
auto pool = std::make_shared<MergeTreeReadPool>(
max_streams,
sum_marks,
@ -215,34 +302,17 @@ Pipe ReadFromMergeTree::readFromPool(
for (size_t i = 0; i < max_streams; ++i)
{
std::optional<ParallelReadingExtension> extension;
if (read_task_callback)
{
extension = ParallelReadingExtension
{
.callback = read_task_callback.value(),
.count_participating_replicas = client_info.count_participating_replicas,
.number_of_current_replica = client_info.number_of_current_replica,
.colums_to_read = required_columns
};
}
auto algorithm = std::make_unique<MergeTreeThreadSelectAlgorithm>(
i, pool, min_marks_for_concurrent_read, max_block_size,
settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes,
data, storage_snapshot, use_uncompressed_cache,
prewhere_info, actions_settings, reader_settings, virt_column_names, std::move(extension));
prewhere_info, actions_settings, reader_settings, virt_column_names);
auto source = std::make_shared<MergeTreeSource>(std::move(algorithm));
/// Set the approximate number of rows for the first source only
/// In case of parallel processing on replicas do not set approximate rows at all.
/// Because the value will be identical on every replicas and will be accounted
/// multiple times (settings.max_parallel_replicas times more)
if (i == 0 && !client_info.collaborate_with_initiator)
if (i == 0)
source->addTotalRowsApprox(total_rows);
pipes.emplace_back(std::move(source));
}
@ -257,21 +327,9 @@ ProcessorPtr ReadFromMergeTree::createSource(
const RangesInDataPart & part,
const Names & required_columns,
bool use_uncompressed_cache,
bool has_limit_below_one_block)
bool has_limit_below_one_block,
MergeTreeInOrderReadPoolParallelReplicasPtr pool)
{
const auto & client_info = context->getClientInfo();
std::optional<ParallelReadingExtension> extension;
if (read_task_callback)
{
extension = ParallelReadingExtension
{
.callback = read_task_callback.value(),
.count_participating_replicas = client_info.count_participating_replicas,
.number_of_current_replica = client_info.number_of_current_replica,
.colums_to_read = required_columns
};
}
auto total_rows = part.getRowsCount();
if (query_info.limit > 0 && query_info.limit < total_rows)
total_rows = query_info.limit;
@ -281,12 +339,12 @@ ProcessorPtr ReadFromMergeTree::createSource(
/// In this case we won't set approximate rows, because it will be accounted multiple times.
/// Also do not count amount of read rows if we read in order of sorting key,
/// because we don't know actual amount of read rows in case when limit is set.
bool set_rows_approx = !extension.has_value() && !reader_settings.read_in_order;
bool set_rows_approx = !is_parallel_reading_from_replicas && !reader_settings.read_in_order;
auto algorithm = std::make_unique<Algorithm>(
data, storage_snapshot, part.data_part, max_block_size, preferred_block_size_bytes,
preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, prewhere_info,
actions_settings, reader_settings, virt_column_names, part.part_index_in_query, has_limit_below_one_block, std::move(extension));
actions_settings, reader_settings, pool, virt_column_names, part.part_index_in_query, has_limit_below_one_block);
auto source = std::make_shared<MergeTreeSource>(std::move(algorithm));
@ -301,7 +359,8 @@ Pipe ReadFromMergeTree::readInOrder(
Names required_columns,
ReadType read_type,
bool use_uncompressed_cache,
UInt64 limit)
UInt64 limit,
MergeTreeInOrderReadPoolParallelReplicasPtr pool)
{
Pipes pipes;
/// For reading in order it makes sense to read only
@ -311,8 +370,8 @@ Pipe ReadFromMergeTree::readInOrder(
for (const auto & part : parts_with_range)
{
auto source = read_type == ReadType::InReverseOrder
? createSource<MergeTreeReverseSelectAlgorithm>(part, required_columns, use_uncompressed_cache, has_limit_below_one_block)
: createSource<MergeTreeInOrderSelectAlgorithm>(part, required_columns, use_uncompressed_cache, has_limit_below_one_block);
? createSource<MergeTreeReverseSelectAlgorithm>(part, required_columns, use_uncompressed_cache, has_limit_below_one_block, pool)
: createSource<MergeTreeInOrderSelectAlgorithm>(part, required_columns, use_uncompressed_cache, has_limit_below_one_block, pool);
pipes.emplace_back(std::move(source));
}
@ -334,11 +393,14 @@ Pipe ReadFromMergeTree::read(
RangesInDataParts parts_with_range, Names required_columns, ReadType read_type,
size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache)
{
if (read_type == ReadType::ParallelReplicas)
return readFromPoolParallelReplicas(parts_with_range, required_columns, max_streams, min_marks_for_concurrent_read, use_uncompressed_cache);
if (read_type == ReadType::Default && max_streams > 1)
return readFromPool(parts_with_range, required_columns, max_streams,
min_marks_for_concurrent_read, use_uncompressed_cache);
auto pipe = readInOrder(parts_with_range, required_columns, read_type, use_uncompressed_cache, 0);
auto pipe = readInOrder(parts_with_range, required_columns, read_type, use_uncompressed_cache, /*limit */0, /*pool*/nullptr);
/// Use ConcatProcessor to concat sources together.
/// It is needed to read in parts order (and so in PK order) if single thread is used.
@ -425,6 +487,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(
const auto & settings = context->getSettingsRef();
const auto data_settings = data.getSettings();
LOG_TRACE(log, "Spreading mark ranges among streams (default reading)");
PartRangesReadInfo info(parts_with_ranges, settings, *data_settings);
if (0 == info.sum_marks)
@ -438,7 +502,9 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(
num_streams = std::max((info.sum_marks + info.min_marks_for_concurrent_read - 1) / info.min_marks_for_concurrent_read, parts_with_ranges.size());
}
return read(std::move(parts_with_ranges), column_names, ReadType::Default,
auto read_type = is_parallel_reading_from_replicas ? ReadType::ParallelReplicas : ReadType::Default;
return read(std::move(parts_with_ranges), column_names, read_type,
num_streams, info.min_marks_for_concurrent_read, info.use_uncompressed_cache);
}
@ -459,6 +525,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
const auto & settings = context->getSettingsRef();
const auto data_settings = data.getSettings();
LOG_TRACE(log, "Spreading ranges among streams with order");
PartRangesReadInfo info(parts_with_ranges, settings, *data_settings);
Pipes res;
@ -534,7 +602,41 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
const size_t min_marks_per_stream = (info.sum_marks - 1) / requested_num_streams + 1;
bool need_preliminary_merge = (parts_with_ranges.size() > settings.read_in_order_two_level_merge_threshold);
Pipes pipes;
std::vector<RangesInDataParts> splitted_parts_and_ranges;
splitted_parts_and_ranges.reserve(requested_num_streams);
const auto read_type = input_order_info->direction == 1
? ReadFromMergeTree::ReadType::InOrder
: ReadFromMergeTree::ReadType::InReverseOrder;
MergeTreeInOrderReadPoolParallelReplicasPtr pool;
if (is_parallel_reading_from_replicas)
{
const auto & client_info = context->getClientInfo();
auto extension = ParallelReadingExtension
{
.all_callback = all_ranges_callback.value(),
.callback = read_task_callback.value(),
.count_participating_replicas = client_info.count_participating_replicas,
.number_of_current_replica = client_info.number_of_current_replica,
.colums_to_read = column_names
};
/// We have a special logic for local replica. It has to read less data, because in some cases it should
/// merge states of aggregate functions or do some other important stuff other than reading from Disk.
auto is_local_replica = context->getClientInfo().interface == ClientInfo::Interface::LOCAL;
auto min_marks_for_concurrent_read = info.min_marks_for_concurrent_read;
if (!is_local_replica)
min_marks_for_concurrent_read = static_cast<size_t>(min_marks_for_concurrent_read * settings.parallel_replicas_single_task_marks_count_multiplier);
pool = std::make_shared<MergeTreeInOrderReadPoolParallelReplicas>(
parts_with_ranges,
extension,
read_type == ReadFromMergeTree::ReadType::InOrder ? CoordinationMode::WithOrder : CoordinationMode::ReverseOrder,
min_marks_for_concurrent_read);
}
for (size_t i = 0; i < requested_num_streams && !parts_with_ranges.empty(); ++i)
{
@ -602,12 +704,14 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
new_parts.emplace_back(part.data_part, part.part_index_in_query, std::move(ranges_to_get_from_part));
}
auto read_type = input_order_info->direction == 1
? ReadFromMergeTree::ReadType::InOrder
: ReadFromMergeTree::ReadType::InReverseOrder;
splitted_parts_and_ranges.emplace_back(std::move(new_parts));
}
pipes.emplace_back(readInOrder(std::move(new_parts), column_names, read_type,
info.use_uncompressed_cache, input_order_info->limit));
Pipes pipes;
for (auto & item : splitted_parts_and_ranges)
{
pipes.emplace_back(readInOrder(std::move(item), column_names, read_type,
info.use_uncompressed_cache, input_order_info->limit, pool));
}
Block pipe_header;
@ -758,7 +862,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
/// If do_not_merge_across_partitions_select_final is true and num_streams > 1
/// we will store lonely parts with level > 0 to use parallel select on them.
std::vector<RangesInDataPart> lonely_parts;
RangesInDataParts lonely_parts;
size_t sum_marks_in_lonely_parts = 0;
for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index)
@ -1265,6 +1369,17 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
const auto & input_order_info = query_info.getInputOrderInfo();
/// Construct a proper coordinator
if (input_order_info && is_parallel_reading_from_replicas && context->getClientInfo().interface == ClientInfo::Interface::LOCAL)
{
assert(context->parallel_reading_coordinator);
auto mode = input_order_info->direction == 1 ? CoordinationMode::WithOrder : CoordinationMode::ReverseOrder;
context->parallel_reading_coordinator->setMode(mode);
}
if (final && is_parallel_reading_from_replicas)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Final modifier is not supported with parallel replicas");
if (final)
{
/// Add columns needed to calculate the sorting expression and the sign.
@ -1406,6 +1521,8 @@ static const char * readTypeToString(ReadFromMergeTree::ReadType type)
return "InOrder";
case ReadFromMergeTree::ReadType::InReverseOrder:
return "InReverseOrder";
case ReadFromMergeTree::ReadType::ParallelReplicas:
return "Parallel";
}
UNREACHABLE();

View File

@ -3,6 +3,8 @@
#include <Storages/MergeTree/RangesInDataPart.h>
#include <Storages/MergeTree/RequestResponse.h>
#include <Storages/SelectQueryInfo.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeReadPool.h>
namespace DB
{
@ -11,7 +13,7 @@ using PartitionIdToMaxBlock = std::unordered_map<String, Int64>;
class Pipe;
using MergeTreeReadTaskCallback = std::function<std::optional<PartitionReadResponse>(PartitionReadRequest)>;
using MergeTreeReadTaskCallback = std::function<std::optional<ParallelReadResponse>(ParallelReadRequest)>;
struct MergeTreeDataSelectSamplingData
{
@ -68,6 +70,10 @@ public:
/// The same as InOrder, but in reverse order.
/// For every part, read ranges and granules from end to begin. Also add ReverseTransform.
InReverseOrder,
/// A special type of reading where every replica
/// talks to a remote coordinator (which is located on the initiator node)
/// and who spreads marks and parts across them.
ParallelReplicas,
};
struct AnalysisResult
@ -212,10 +218,11 @@ private:
Pipe read(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache);
Pipe readFromPool(RangesInDataParts parts_with_ranges, Names required_columns, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache);
Pipe readInOrder(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, bool use_uncompressed_cache, UInt64 limit);
Pipe readFromPoolParallelReplicas(RangesInDataParts parts_with_ranges, Names required_columns, size_t max_streams, size_t min_marks_for_concurrent_read, bool use_uncompressed_cache);
Pipe readInOrder(RangesInDataParts parts_with_range, Names required_columns, ReadType read_type, bool use_uncompressed_cache, UInt64 limit, MergeTreeInOrderReadPoolParallelReplicasPtr pool);
template<typename TSource>
ProcessorPtr createSource(const RangesInDataPart & part, const Names & required_columns, bool use_uncompressed_cache, bool has_limit_below_one_block);
ProcessorPtr createSource(const RangesInDataPart & part, const Names & required_columns, bool use_uncompressed_cache, bool has_limit_below_one_block, MergeTreeInOrderReadPoolParallelReplicasPtr pool);
Pipe spreadMarkRangesAmongStreams(
RangesInDataParts && parts_with_ranges,
@ -236,6 +243,8 @@ private:
ReadFromMergeTree::AnalysisResult getAnalysisResult() const;
MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr;
bool is_parallel_reading_from_replicas;
std::optional<MergeTreeAllRangesCallback> all_ranges_callback;
std::optional<MergeTreeReadTaskCallback> read_task_callback;
};

View File

@ -9,10 +9,13 @@
#include <Processors/Sources/RemoteSource.h>
#include <Processors/Sources/DelayedSource.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/ReadFromMergeTreeDependencyTransform.h>
#include <Interpreters/ActionsDAG.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <IO/ConnectionTimeoutsContext.h>
#include "Common/logger_useful.h"
#include <Common/checkStackSize.h>
#include <Core/QueryProcessingStage.h>
#include <Client/ConnectionPool.h>
#include <Client/ConnectionPoolWithFailover.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
@ -180,7 +183,8 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream
if (try_results.empty() || local_delay < max_remote_delay)
{
auto plan = createLocalPlan(query, header, context, stage, shard.shard_info.shard_num, shard_count, 0, 0, /*coordinator=*/nullptr);
auto plan = createLocalPlan(
query, header, context, stage, shard.shard_info.shard_num, shard_count, 0, 0, /*coordinator=*/nullptr);
return std::move(*plan->buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(context),
@ -231,7 +235,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact
std::shared_ptr<RemoteQueryExecutor> remote_query_executor;
remote_query_executor = std::make_shared<RemoteQueryExecutor>(
shard.shard_info.pool, query_string, shard.header, context, throttler, scalars, external_tables, stage);
shard.shard_info.pool, query_string, output_stream->header, context, throttler, scalars, external_tables, stage);
remote_query_executor->setLogger(log);
remote_query_executor->setPoolMode(PoolMode::GET_MANY);
@ -265,8 +269,9 @@ void ReadFromRemote::initializePipeline(QueryPipelineBuilder & pipeline, const B
ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
ASTPtr query_ast_,
Cluster::ShardInfo shard_info_,
ParallelReplicasReadingCoordinatorPtr coordinator_,
ClusterProxy::SelectStreamFactory::Shard shard_,
Block header_,
QueryProcessingStage::Enum stage_,
StorageID main_table_,
@ -276,10 +281,12 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
Scalars scalars_,
Tables external_tables_,
Poco::Logger * log_,
std::shared_ptr<const StorageLimitsList> storage_limits_)
std::shared_ptr<const StorageLimitsList> storage_limits_,
UUID uuid_)
: ISourceStep(DataStream{.header = std::move(header_)})
, shard_info(shard_info_)
, query_ast(query_ast_)
, coordinator(std::move(coordinator_))
, shard(std::move(shard_))
, stage(std::move(stage_))
, main_table(std::move(main_table_))
, table_func_ptr(table_func_ptr_)
@ -289,10 +296,11 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
, external_tables{external_tables_}
, storage_limits(std::move(storage_limits_))
, log(log_)
, uuid(uuid_)
{
std::vector<String> description;
for (const auto & address : shard.shard_info.all_addresses)
for (const auto & address : shard_info.all_addresses)
if (!address.is_local)
description.push_back(fmt::format("Replica: {}", address.host_name));
@ -312,28 +320,46 @@ void ReadFromParallelRemoteReplicasStep::enforceAggregationInOrder()
void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
Pipes pipes;
const Settings & current_settings = context->getSettingsRef();
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings);
for (size_t replica_num = 0; replica_num < shard.shard_info.getAllNodeCount(); ++replica_num)
size_t all_replicas_count = current_settings.max_parallel_replicas;
if (all_replicas_count > shard_info.all_addresses.size())
{
if (shard.shard_info.all_addresses[replica_num].is_local)
LOG_INFO(&Poco::Logger::get("ReadFromParallelRemoteReplicasStep"),
"The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). "\
"Will use the latter number to execute the query.", current_settings.max_parallel_replicas, shard_info.all_addresses.size());
all_replicas_count = shard_info.all_addresses.size();
}
/// The requested number of replicas to read from could be less
/// than the total number of replicas in the shard
/// And we have to pick only "remote" ones
/// So, that's why this loop looks like this.
size_t replica_num = 0;
while (pipes.size() != all_replicas_count - 1)
{
if (shard_info.all_addresses[replica_num].is_local)
{
++replica_num;
continue;
}
IConnections::ReplicaInfo replica_info
{
.all_replicas_count = shard.shard_info.getAllNodeCount(),
.number_of_current_replica = replica_num
.all_replicas_count = all_replicas_count,
/// Replica 0 is threated as local always
.number_of_current_replica = pipes.size() + 1
};
auto pool = shard.shard_info.per_replica_pools[replica_num];
auto pool = shard_info.per_replica_pools[replica_num];
assert(pool);
auto pool_with_failover = std::make_shared<ConnectionPoolWithFailover>(
ConnectionPoolPtrs{pool}, current_settings.load_balancing);
addPipeForSingeReplica(pipes, std::move(pool_with_failover), replica_info);
++replica_num;
}
auto pipe = Pipe::unitePipes(std::move(pipes));
@ -355,25 +381,22 @@ void ReadFromParallelRemoteReplicasStep::addPipeForSingeReplica(Pipes & pipes, s
if (stage == QueryProcessingStage::Complete)
{
add_totals = shard.query->as<ASTSelectQuery &>().group_by_with_totals;
add_totals = query_ast->as<ASTSelectQuery &>().group_by_with_totals;
add_extremes = context->getSettingsRef().extremes;
}
String query_string = formattedAST(shard.query);
String query_string = formattedAST(query_ast);
scalars["_shard_num"]
= Block{{DataTypeUInt32().createColumnConst(1, shard.shard_info.shard_num), std::make_shared<DataTypeUInt32>(), "_shard_num"}};
assert(stage != QueryProcessingStage::Complete);
assert(output_stream);
auto remote_query_executor = std::make_shared<RemoteQueryExecutor>(
pool, query_string, shard.header, context, throttler, scalars, external_tables, stage,
pool, query_string, output_stream->header, context, throttler, scalars, external_tables, stage,
RemoteQueryExecutor::Extension{.parallel_reading_coordinator = coordinator, .replica_info = std::move(replica_info)});
remote_query_executor->setLogger(log);
if (!table_func_ptr)
remote_query_executor->setMainTable(main_table);
pipes.emplace_back(createRemoteSourcePipe(std::move(remote_query_executor), add_agg_info, add_totals, add_extremes, async_read));
pipes.emplace_back(createRemoteSourcePipe(std::move(remote_query_executor), add_agg_info, add_totals, add_extremes, async_read, uuid));
addConvertingActions(pipes.back(), output_stream->header);
}

View File

@ -6,6 +6,7 @@
#include <Interpreters/StorageID.h>
#include <Interpreters/ClusterProxy/SelectStreamFactory.h>
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
#include "Core/UUID.h"
namespace DB
{
@ -45,18 +46,13 @@ public:
private:
ClusterProxy::SelectStreamFactory::Shards shards;
QueryProcessingStage::Enum stage;
StorageID main_table;
ASTPtr table_func_ptr;
ContextMutablePtr context;
ThrottlerPtr throttler;
Scalars scalars;
Tables external_tables;
std::shared_ptr<const StorageLimitsList> storage_limits;
Poco::Logger * log;
UInt32 shard_count;
@ -69,8 +65,9 @@ class ReadFromParallelRemoteReplicasStep : public ISourceStep
{
public:
ReadFromParallelRemoteReplicasStep(
ASTPtr query_ast_,
Cluster::ShardInfo shard_info,
ParallelReplicasReadingCoordinatorPtr coordinator_,
ClusterProxy::SelectStreamFactory::Shard shard,
Block header_,
QueryProcessingStage::Enum stage_,
StorageID main_table_,
@ -80,7 +77,8 @@ public:
Scalars scalars_,
Tables external_tables_,
Poco::Logger * log_,
std::shared_ptr<const StorageLimitsList> storage_limits_);
std::shared_ptr<const StorageLimitsList> storage_limits_,
UUID uuid);
String getName() const override { return "ReadFromRemoteParallelReplicas"; }
@ -93,22 +91,20 @@ private:
void addPipeForSingeReplica(Pipes & pipes, std::shared_ptr<ConnectionPoolWithFailover> pool, IConnections::ReplicaInfo replica_info);
Cluster::ShardInfo shard_info;
ASTPtr query_ast;
ParallelReplicasReadingCoordinatorPtr coordinator;
ClusterProxy::SelectStreamFactory::Shard shard;
QueryProcessingStage::Enum stage;
StorageID main_table;
ASTPtr table_func_ptr;
ContextMutablePtr context;
ThrottlerPtr throttler;
Scalars scalars;
Tables external_tables;
std::shared_ptr<const StorageLimitsList> storage_limits;
Poco::Logger * log;
UUID uuid;
};
}

View File

@ -1,3 +1,4 @@
#include <variant>
#include <Processors/Sources/RemoteSource.h>
#include <QueryPipeline/RemoteQueryExecutor.h>
#include <QueryPipeline/RemoteQueryExecutorReadContext.h>
@ -8,10 +9,16 @@
namespace DB
{
RemoteSource::RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation_info_, bool async_read_)
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
RemoteSource::RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation_info_, bool async_read_, UUID uuid_)
: ISource(executor->getHeader(), false)
, add_aggregation_info(add_aggregation_info_), query_executor(std::move(executor))
, async_read(async_read_)
, uuid(uuid_)
{
/// Add AggregatedChunkInfo if we expect DataTypeAggregateFunction as a result.
const auto & sample = getPort().getHeader();
@ -22,6 +29,18 @@ RemoteSource::RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation
RemoteSource::~RemoteSource() = default;
void RemoteSource::connectToScheduler(InputPort & input_port)
{
outputs.emplace_back(Block{}, this);
dependency_port = &outputs.back();
connect(*dependency_port, input_port);
}
UUID RemoteSource::getParallelReplicasGroupUUID()
{
return uuid;
}
void RemoteSource::setStorageLimits(const std::shared_ptr<const StorageLimitsList> & storage_limits_)
{
/// Remove leaf limits for remote source.
@ -50,8 +69,21 @@ ISource::Status RemoteSource::prepare()
if (status == Status::Finished)
{
query_executor->finish(&read_context);
if (dependency_port)
dependency_port->finish();
is_async_state = false;
return status;
}
if (status == Status::PortFull)
{
/// Also push empty chunk to dependency to signal that we read data from remote source
/// or answered to the incoming request from parallel replica
if (dependency_port && !dependency_port->isFinished() && dependency_port->canPush())
dependency_port->push(Chunk());
}
return status;
}
@ -88,19 +120,29 @@ std::optional<Chunk> RemoteSource::tryGenerate()
if (async_read)
{
auto res = query_executor->read(read_context);
if (std::holds_alternative<int>(res))
if (res.getType() == RemoteQueryExecutor::ReadResult::Type::Nothing)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Got an empty packet from the RemoteQueryExecutor. This is a bug");
if (res.getType() == RemoteQueryExecutor::ReadResult::Type::FileDescriptor)
{
fd = std::get<int>(res);
fd = res.getFileDescriptor();
is_async_state = true;
return Chunk();
}
if (res.getType() == RemoteQueryExecutor::ReadResult::Type::ParallelReplicasToken)
{
is_async_state = false;
return Chunk();
}
is_async_state = false;
block = std::get<Block>(std::move(res));
block = res.getBlock();
}
else
block = query_executor->read();
block = query_executor->readBlock();
if (!block)
{
@ -180,9 +222,9 @@ Chunk RemoteExtremesSource::generate()
Pipe createRemoteSourcePipe(
RemoteQueryExecutorPtr query_executor,
bool add_aggregation_info, bool add_totals, bool add_extremes, bool async_read)
bool add_aggregation_info, bool add_totals, bool add_extremes, bool async_read, UUID uuid)
{
Pipe pipe(std::make_shared<RemoteSource>(query_executor, add_aggregation_info, async_read));
Pipe pipe(std::make_shared<RemoteSource>(query_executor, add_aggregation_info, async_read, uuid));
if (add_totals)
pipe.addTotalsSource(std::make_shared<RemoteTotalsSource>(query_executor));

View File

@ -3,6 +3,7 @@
#include <Processors/ISource.h>
#include <Processors/RowsBeforeLimitCounter.h>
#include <QueryPipeline/Pipe.h>
#include "Core/UUID.h"
#include <atomic>
namespace DB
@ -14,20 +15,24 @@ using RemoteQueryExecutorPtr = std::shared_ptr<RemoteQueryExecutor>;
class RemoteQueryExecutorReadContext;
/// Source from RemoteQueryExecutor. Executes remote query and returns query result chunks.
class RemoteSource : public ISource
class RemoteSource final : public ISource
{
public:
/// Flag add_aggregation_info tells if AggregatedChunkInfo should be added to result chunk.
/// AggregatedChunkInfo stores the bucket number used for two-level aggregation.
/// This flag should be typically enabled for queries with GROUP BY which are executed till WithMergeableState.
RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation_info_, bool async_read_);
RemoteSource(RemoteQueryExecutorPtr executor, bool add_aggregation_info_, bool async_read_, UUID uuid = UUIDHelpers::Nil);
~RemoteSource() override;
Status prepare() override;
String getName() const override { return "Remote"; }
void connectToScheduler(InputPort & input_port);
void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) { rows_before_limit.swap(counter); }
UUID getParallelReplicasGroupUUID();
/// Stop reading from stream if output port is finished.
void onUpdatePorts() override;
@ -46,9 +51,12 @@ private:
RemoteQueryExecutorPtr query_executor;
RowsBeforeLimitCounterPtr rows_before_limit;
OutputPort * dependency_port{nullptr};
const bool async_read;
bool is_async_state = false;
std::unique_ptr<RemoteQueryExecutorReadContext> read_context;
UUID uuid;
int fd = -1;
};
@ -87,6 +95,6 @@ private:
/// Create pipe with remote sources.
Pipe createRemoteSourcePipe(
RemoteQueryExecutorPtr query_executor,
bool add_aggregation_info, bool add_totals, bool add_extremes, bool async_read);
bool add_aggregation_info, bool add_totals, bool add_extremes, bool async_read, UUID uuid = UUIDHelpers::Nil);
}

View File

@ -0,0 +1,103 @@
#include <Processors/Transforms/ReadFromMergeTreeDependencyTransform.h>
#include <QueryPipeline/RemoteQueryExecutor.h>
#include "Processors/Port.h"
namespace DB
{
ReadFromMergeTreeDependencyTransform::ReadFromMergeTreeDependencyTransform(const Block & header, UUID uuid_)
: IProcessor(InputPorts(1, header), OutputPorts(1, header))
, uuid(uuid_)
, data_port(&inputs.front())
{
}
void ReadFromMergeTreeDependencyTransform::connectToScheduler(OutputPort & output_port)
{
inputs.emplace_back(Block{}, this);
dependency_port = &inputs.back();
connect(output_port, *dependency_port);
}
UUID ReadFromMergeTreeDependencyTransform::getParallelReplicasGroupUUID()
{
return uuid;
}
IProcessor::Status ReadFromMergeTreeDependencyTransform::prepare()
{
Status status = Status::Ready;
while (status == Status::Ready)
{
status = !has_data ? prepareConsume()
: prepareGenerate();
}
return status;
}
IProcessor::Status ReadFromMergeTreeDependencyTransform::prepareConsume()
{
auto & output_port = getOutputPort();
/// Check all outputs are finished or ready to get data.
if (output_port.isFinished())
{
data_port->close();
dependency_port->close();
return Status::Finished;
}
/// Try get chunk from input.
if (data_port->isFinished())
{
if (dependency_port->hasData())
dependency_port->pull(true);
dependency_port->close();
output_port.finish();
return Status::Finished;
}
if (!dependency_port->isFinished())
{
dependency_port->setNeeded();
if (!dependency_port->hasData())
return Status::NeedData;
}
data_port->setNeeded();
if (!data_port->hasData())
return Status::NeedData;
if (!dependency_port->isFinished())
dependency_port->pull();
chunk = data_port->pull();
has_data = true;
return Status::Ready;
}
IProcessor::Status ReadFromMergeTreeDependencyTransform::prepareGenerate()
{
auto & output_port = getOutputPort();
if (!output_port.isFinished() && output_port.canPush())
{
output_port.push(std::move(chunk));
has_data = false;
return Status::Ready;
}
if (output_port.isFinished())
{
data_port->close();
dependency_port->close();
return Status::Finished;
}
return Status::PortFull;
}
}

View File

@ -0,0 +1,48 @@
#pragma once
#include <Processors/IProcessor.h>
namespace DB
{
class RemoteQueryExecutor;
using RemoteQueryExecutorPtr = std::shared_ptr<RemoteQueryExecutor>;
/// A tiny class which is used for reading with multiple replicas in parallel.
/// Motivation is that we don't have a full control on how
/// processors are scheduled across threads and there could be a situation
/// when all available threads will read from local replica and will just
/// forget about remote replicas existence. That is not what we want.
/// For parallel replicas we have to constantly answer to incoming requests
/// with a set of marks to read.
/// With the help of this class, we explicitly connect a "local" source with
/// all the remote ones. And thus achieve fairness somehow.
class ReadFromMergeTreeDependencyTransform : public IProcessor
{
public:
ReadFromMergeTreeDependencyTransform(const Block & header, UUID uuid_);
String getName() const override { return "ReadFromMergeTreeDependency"; }
Status prepare() override;
InputPort & getInputPort() { assert(data_port); return *data_port; }
InputPort & getDependencyPort() { assert(dependency_port); return *dependency_port; }
OutputPort & getOutputPort() { return outputs.front(); }
UUID getParallelReplicasGroupUUID();
void connectToScheduler(OutputPort & output_port);
private:
bool has_data{false};
Chunk chunk;
UUID uuid;
InputPort * data_port{nullptr};
InputPort * dependency_port{nullptr};
Status prepareGenerate();
Status prepareConsume();
};
}

View File

@ -102,6 +102,8 @@ public:
/// Get processors from Pipe without destroying pipe (used for EXPLAIN to keep QueryPlan).
const Processors & getProcessors() const { return *processors; }
std::shared_ptr<Processors> getProcessorsPtr() { return processors; }
private:
/// Header is common for all output below.
Block header;

View File

@ -1,34 +1,35 @@
#include <vector>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/ResizeProcessor.h>
#include <Processors/LimitTransform.h>
#include <Processors/Transforms/TotalsHavingTransform.h>
#include <Processors/Transforms/ExtremesTransform.h>
#include <Processors/Transforms/CreatingSetsTransform.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h>
#include <Processors/Transforms/JoiningTransform.h>
#include <Processors/Transforms/MergeJoinTransform.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <Processors/Transforms/PartialSortingTransform.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <IO/WriteHelpers.h>
#include <Common/CurrentThread.h>
#include <Common/typeid_cast.h>
#include "Core/UUID.h"
#include <Core/SortDescription.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/IJoin.h>
#include <Interpreters/TableJoin.h>
#include <Common/typeid_cast.h>
#include <Common/CurrentThread.h>
#include <IO/WriteHelpers.h>
#include <Processors/ConcatProcessor.h>
#include <Core/SortDescription.h>
#include <QueryPipeline/narrowPipe.h>
#include <Processors/DelayedPortsProcessor.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Processors/LimitTransform.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/ResizeProcessor.h>
#include <Processors/RowsBeforeLimitCounter.h>
#include <Processors/Sources/RemoteSource.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <Processors/Transforms/CreatingSetsTransform.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/ExtremesTransform.h>
#include <Processors/Transforms/JoiningTransform.h>
#include <Processors/Transforms/MergeJoinTransform.h>
#include <Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h>
#include <Processors/Transforms/PartialSortingTransform.h>
#include <Processors/Transforms/ReadFromMergeTreeDependencyTransform.h>
#include <Processors/Transforms/TotalsHavingTransform.h>
#include <QueryPipeline/narrowPipe.h>
namespace DB
{
@ -620,6 +621,65 @@ void QueryPipelineBuilder::setProgressCallback(ProgressCallback callback)
progress_callback = callback;
}
void QueryPipelineBuilder::connectDependencies()
{
/**
* This is needed because among all RemoteSources there could be
* one or several that don't belong to the parallel replicas reading process.
* It could happen for example if we read through distributed table + prefer_localhost_replica=1 + parallel replicas
* SELECT * FROM remote('127.0.0.{1,2}', table.merge_tree)
* Will generate a local pipeline and a remote source. For local pipeline because of parallel replicas we will create
* several processors to read and several remote sources.
*/
std::set<UUID> all_parallel_replicas_groups;
for (auto & processor : *pipe.getProcessorsPtr())
{
if (auto * remote_dependency = typeid_cast<RemoteSource *>(processor.get()); remote_dependency)
if (auto uuid = remote_dependency->getParallelReplicasGroupUUID(); uuid != UUIDHelpers::Nil)
all_parallel_replicas_groups.insert(uuid);
if (auto * merge_tree_dependency = typeid_cast<ReadFromMergeTreeDependencyTransform *>(processor.get()); merge_tree_dependency)
if (auto uuid = merge_tree_dependency->getParallelReplicasGroupUUID(); uuid != UUIDHelpers::Nil)
all_parallel_replicas_groups.insert(uuid);
}
for (const auto & group_id : all_parallel_replicas_groups)
{
std::vector<RemoteSource *> input_dependencies;
std::vector<ReadFromMergeTreeDependencyTransform *> output_dependencies;
for (auto & processor : *pipe.getProcessorsPtr())
{
if (auto * remote_dependency = typeid_cast<RemoteSource *>(processor.get()); remote_dependency)
if (auto uuid = remote_dependency->getParallelReplicasGroupUUID(); uuid == group_id)
input_dependencies.emplace_back(remote_dependency);
if (auto * merge_tree_dependency = typeid_cast<ReadFromMergeTreeDependencyTransform *>(processor.get()); merge_tree_dependency)
if (auto uuid = merge_tree_dependency->getParallelReplicasGroupUUID(); uuid == group_id)
output_dependencies.emplace_back(merge_tree_dependency);
}
if (input_dependencies.empty() || output_dependencies.empty())
continue;
auto input_dependency_iter = input_dependencies.begin();
auto output_dependency_iter = output_dependencies.begin();
auto scheduler = std::make_shared<ResizeProcessor>(Block{}, input_dependencies.size(), output_dependencies.size());
for (auto & scheduler_input : scheduler->getInputs())
{
(*input_dependency_iter)->connectToScheduler(scheduler_input);
++input_dependency_iter;
}
for (auto & scheduler_output : scheduler->getOutputs())
{
(*output_dependency_iter)->connectToScheduler(scheduler_output);
++output_dependency_iter;
}
pipe.getProcessorsPtr()->emplace_back(std::move(scheduler));
}
}
PipelineExecutorPtr QueryPipelineBuilder::execute()
{
if (!isCompleted())

View File

@ -140,6 +140,12 @@ public:
void addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, ContextPtr context);
/// Finds all processors for reading from MergeTree
/// And explicitly connects them with all RemoteSources
/// using a ResizeProcessor. This is needed not to let
/// the RemoteSource to starve for CPU time
void connectDependencies();
PipelineExecutorPtr execute();
size_t getNumStreams() const { return pipe.numOutputPorts(); }

View File

@ -259,48 +259,62 @@ void RemoteQueryExecutor::sendQuery(ClientInfo::QueryKind query_kind)
sendExternalTables();
}
Block RemoteQueryExecutor::read()
Block RemoteQueryExecutor::readBlock()
{
while (true)
{
auto res = read();
if (res.getType() == ReadResult::Type::Data)
return res.getBlock();
}
}
RemoteQueryExecutor::ReadResult RemoteQueryExecutor::read()
{
if (!sent_query)
{
sendQuery();
if (context->getSettingsRef().skip_unavailable_shards && (0 == connections->size()))
return {};
return ReadResult(Block());
}
while (true)
{
std::lock_guard lock(was_cancelled_mutex);
if (was_cancelled)
return Block();
return ReadResult(Block());
Packet packet = connections->receivePacket();
auto packet = connections->receivePacket();
auto anything = processPacket(std::move(packet));
if (auto block = processPacket(std::move(packet)))
return *block;
else if (got_duplicated_part_uuids)
return std::get<Block>(restartQueryWithoutDuplicatedUUIDs());
if (anything.getType() == ReadResult::Type::Data || anything.getType() == ReadResult::Type::ParallelReplicasToken)
return anything;
if (got_duplicated_part_uuids)
return restartQueryWithoutDuplicatedUUIDs();
}
}
std::variant<Block, int> RemoteQueryExecutor::read(std::unique_ptr<ReadContext> & read_context [[maybe_unused]])
RemoteQueryExecutor::ReadResult RemoteQueryExecutor::read(std::unique_ptr<ReadContext> & read_context [[maybe_unused]])
{
#if defined(OS_LINUX)
if (!sent_query)
{
sendQuery();
if (context->getSettingsRef().skip_unavailable_shards && (0 == connections->size()))
return Block();
return ReadResult(Block());
}
if (!read_context || resent_query)
{
std::lock_guard lock(was_cancelled_mutex);
if (was_cancelled)
return Block();
return ReadResult(Block());
read_context = std::make_unique<ReadContext>(*connections);
}
@ -308,12 +322,12 @@ std::variant<Block, int> RemoteQueryExecutor::read(std::unique_ptr<ReadContext>
do
{
if (!read_context->resumeRoutine())
return Block();
return ReadResult(Block());
if (read_context->is_read_in_progress.load(std::memory_order_relaxed))
{
read_context->setTimer();
return read_context->epoll.getFileDescriptor();
return ReadResult(read_context->epoll.getFileDescriptor());
}
else
{
@ -321,11 +335,14 @@ std::variant<Block, int> RemoteQueryExecutor::read(std::unique_ptr<ReadContext>
/// to avoid the race between cancel() thread and read() thread.
/// (since cancel() thread will steal the fiber and may update the packet).
if (was_cancelled)
return Block();
return ReadResult(Block());
if (auto data = processPacket(std::move(read_context->packet)))
return std::move(*data);
else if (got_duplicated_part_uuids)
auto anything = processPacket(std::move(read_context->packet));
if (anything.getType() == ReadResult::Type::Data || anything.getType() == ReadResult::Type::ParallelReplicasToken)
return anything;
if (got_duplicated_part_uuids)
return restartQueryWithoutDuplicatedUUIDs(&read_context);
}
}
@ -336,7 +353,7 @@ std::variant<Block, int> RemoteQueryExecutor::read(std::unique_ptr<ReadContext>
}
std::variant<Block, int> RemoteQueryExecutor::restartQueryWithoutDuplicatedUUIDs(std::unique_ptr<ReadContext> * read_context)
RemoteQueryExecutor::ReadResult RemoteQueryExecutor::restartQueryWithoutDuplicatedUUIDs(std::unique_ptr<ReadContext> * read_context)
{
/// Cancel previous query and disconnect before retry.
cancel(read_context);
@ -360,13 +377,18 @@ std::variant<Block, int> RemoteQueryExecutor::restartQueryWithoutDuplicatedUUIDs
throw Exception(ErrorCodes::DUPLICATED_PART_UUIDS, "Found duplicate uuids while processing query");
}
std::optional<Block> RemoteQueryExecutor::processPacket(Packet packet)
RemoteQueryExecutor::ReadResult RemoteQueryExecutor::processPacket(Packet packet)
{
switch (packet.type)
{
case Protocol::Server::MergeTreeReadTaskRequest:
processMergeTreeReadTaskRequest(packet.request);
break;
return ReadResult(ReadResult::Type::ParallelReplicasToken);
case Protocol::Server::MergeTreeAllRangesAnnounecement:
processMergeTreeInitialReadAnnounecement(packet.announcement);
return ReadResult(ReadResult::Type::ParallelReplicasToken);
case Protocol::Server::ReadTaskRequest:
processReadTaskRequest();
break;
@ -379,7 +401,7 @@ std::optional<Block> RemoteQueryExecutor::processPacket(Packet packet)
/// We can actually return it, and the first call to RemoteQueryExecutor::read
/// will return earlier. We should consider doing it.
if (packet.block && (packet.block.rows() > 0))
return adaptBlockStructure(packet.block, header);
return ReadResult(adaptBlockStructure(packet.block, header));
break; /// If the block is empty - we will receive other packets before EndOfStream.
case Protocol::Server::Exception:
@ -391,7 +413,8 @@ std::optional<Block> RemoteQueryExecutor::processPacket(Packet packet)
if (!connections->hasActiveConnections())
{
finished = true;
return Block();
/// TODO: Replace with Type::Finished
return ReadResult(Block{});
}
break;
@ -446,7 +469,7 @@ std::optional<Block> RemoteQueryExecutor::processPacket(Packet packet)
connections->dumpAddresses());
}
return {};
return ReadResult(ReadResult::Type::Nothing);
}
bool RemoteQueryExecutor::setPartUUIDs(const std::vector<UUID> & uuids)
@ -471,7 +494,7 @@ void RemoteQueryExecutor::processReadTaskRequest()
connections->sendReadTaskResponse(response);
}
void RemoteQueryExecutor::processMergeTreeReadTaskRequest(PartitionReadRequest request)
void RemoteQueryExecutor::processMergeTreeReadTaskRequest(ParallelReadRequest request)
{
if (!parallel_reading_coordinator)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Coordinator for parallel reading from replicas is not initialized");
@ -480,6 +503,14 @@ void RemoteQueryExecutor::processMergeTreeReadTaskRequest(PartitionReadRequest r
connections->sendMergeTreeReadTaskResponse(response);
}
void RemoteQueryExecutor::processMergeTreeInitialReadAnnounecement(InitialAllRangesAnnouncement announcement)
{
if (!parallel_reading_coordinator)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Coordinator for parallel reading from replicas is not initialized");
parallel_reading_coordinator->handleInitialAllRangesAnnouncement(announcement);
}
void RemoteQueryExecutor::finish(std::unique_ptr<ReadContext> * read_context)
{
/** If one of:

View File

@ -10,6 +10,7 @@
#include <Interpreters/StorageID.h>
#include <Common/TimerDescriptor.h>
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
#include <sys/types.h>
namespace DB
@ -94,12 +95,60 @@ public:
/// Query is resent to a replica, the query itself can be modified.
std::atomic<bool> resent_query { false };
struct ReadResult
{
enum class Type : uint8_t
{
Data,
ParallelReplicasToken,
FileDescriptor,
Finished,
Nothing
};
explicit ReadResult(Block block_)
: type(Type::Data)
, block(std::move(block_))
{}
explicit ReadResult(int fd_)
: type(Type::FileDescriptor)
, fd(fd_)
{}
explicit ReadResult(Type type_)
: type(type_)
{
assert(type != Type::Data && type != Type::FileDescriptor);
}
Type getType() const { return type; }
Block getBlock()
{
chassert(type == Type::Data);
return std::move(block);
}
int getFileDescriptor() const
{
chassert(type == Type::FileDescriptor);
return fd;
}
Type type;
Block block;
int fd{-1};
};
/// Read next block of data. Returns empty block if query is finished.
Block read();
Block readBlock();
ReadResult read();
/// Async variant of read. Returns ready block or file descriptor which may be used for polling.
/// ReadContext is an internal read state. Pass empty ptr first time, reuse created one for every call.
std::variant<Block, int> read(std::unique_ptr<ReadContext> & read_context);
ReadResult read(std::unique_ptr<ReadContext> & read_context);
/// Receive all remain packets and finish query.
/// It should be cancelled after read returned empty block.
@ -231,11 +280,12 @@ private:
void processReadTaskRequest();
void processMergeTreeReadTaskRequest(PartitionReadRequest request);
void processMergeTreeReadTaskRequest(ParallelReadRequest request);
void processMergeTreeInitialReadAnnounecement(InitialAllRangesAnnouncement announcement);
/// Cancel query and restart it with info about duplicate UUIDs
/// only for `allow_experimental_query_deduplication`.
std::variant<Block, int> restartQueryWithoutDuplicatedUUIDs(std::unique_ptr<ReadContext> * read_context = nullptr);
ReadResult restartQueryWithoutDuplicatedUUIDs(std::unique_ptr<ReadContext> * read_context = nullptr);
/// If wasn't sent yet, send request to cancel all connections to replicas
void tryCancel(const char * reason, std::unique_ptr<ReadContext> * read_context);
@ -247,11 +297,10 @@ private:
bool hasThrownException() const;
/// Process packet for read and return data block if possible.
std::optional<Block> processPacket(Packet packet);
ReadResult processPacket(Packet packet);
/// Reads packet by packet
Block readPackets();
};
}

View File

@ -54,6 +54,7 @@
#include <Processors/Sinks/SinkToStorage.h>
#include "Core/Protocol.h"
#include "Storages/MergeTree/RequestResponse.h"
#include "TCPHandler.h"
#include "config_version.h"
@ -363,7 +364,17 @@ void TCPHandler::runImpl()
return receiveReadTaskResponseAssumeLocked();
});
query_context->setMergeTreeReadTaskCallback([this](PartitionReadRequest request) -> std::optional<PartitionReadResponse>
query_context->setMergeTreeAllRangesCallback([this](InitialAllRangesAnnouncement announcement)
{
std::lock_guard lock(task_callback_mutex);
if (state.is_cancelled)
return;
sendMergeTreeAllRangesAnnounecementAssumeLocked(announcement);
});
query_context->setMergeTreeReadTaskCallback([this](ParallelReadRequest request) -> std::optional<ParallelReadResponse>
{
std::lock_guard lock(task_callback_mutex);
@ -920,7 +931,15 @@ void TCPHandler::sendReadTaskRequestAssumeLocked()
}
void TCPHandler::sendMergeTreeReadTaskRequestAssumeLocked(PartitionReadRequest request)
void TCPHandler::sendMergeTreeAllRangesAnnounecementAssumeLocked(InitialAllRangesAnnouncement announcement)
{
writeVarUInt(Protocol::Server::MergeTreeAllRangesAnnounecement, *out);
announcement.serialize(*out);
out->next();
}
void TCPHandler::sendMergeTreeReadTaskRequestAssumeLocked(ParallelReadRequest request)
{
writeVarUInt(Protocol::Server::MergeTreeReadTaskRequest, *out);
request.serialize(*out);
@ -1348,7 +1367,7 @@ String TCPHandler::receiveReadTaskResponseAssumeLocked()
}
std::optional<PartitionReadResponse> TCPHandler::receivePartitionMergeTreeReadTaskResponseAssumeLocked()
std::optional<ParallelReadResponse> TCPHandler::receivePartitionMergeTreeReadTaskResponseAssumeLocked()
{
UInt64 packet_type = 0;
readVarUInt(packet_type, *in);
@ -1371,7 +1390,7 @@ std::optional<PartitionReadResponse> TCPHandler::receivePartitionMergeTreeReadTa
Protocol::Client::toString(packet_type));
}
}
PartitionReadResponse response;
ParallelReadResponse response;
response.deserialize(*in);
return response;
}

View File

@ -21,6 +21,7 @@
#include "IServer.h"
#include "Server/TCPProtocolStackData.h"
#include "Storages/MergeTree/RequestResponse.h"
#include "base/types.h"
@ -220,7 +221,7 @@ private:
void receiveQuery();
void receiveIgnoredPartUUIDs();
String receiveReadTaskResponseAssumeLocked();
std::optional<PartitionReadResponse> receivePartitionMergeTreeReadTaskResponseAssumeLocked();
std::optional<ParallelReadResponse> receivePartitionMergeTreeReadTaskResponseAssumeLocked();
bool receiveData(bool scalar);
bool readDataNext();
void readData();
@ -253,7 +254,8 @@ private:
void sendEndOfStream();
void sendPartUUIDs();
void sendReadTaskRequestAssumeLocked();
void sendMergeTreeReadTaskRequestAssumeLocked(PartitionReadRequest request);
void sendMergeTreeAllRangesAnnounecementAssumeLocked(InitialAllRangesAnnouncement announcement);
void sendMergeTreeReadTaskRequestAssumeLocked(ParallelReadRequest request);
void sendProfileInfo(const ProfileInfo & info);
void sendTotals(const Block & totals);
void sendExtremes(const Block & extremes);

View File

@ -136,7 +136,7 @@ struct HalfIntervals
MarkRanges convertToMarkRangesFinal()
{
MarkRanges result;
std::move(intervals.begin(), intervals.end(), std::back_inserter(result));
std::copy(intervals.begin(), intervals.end(), std::back_inserter(result));
return result;
}

View File

@ -1,5 +1,8 @@
#include "MarkRange.h"
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
namespace DB
{
@ -8,6 +11,11 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
size_t MarkRange::getNumberOfMarks() const
{
return end - begin;
}
bool MarkRange::operator==(const MarkRange & rhs) const
{
return begin == rhs.begin && end == rhs.end;
@ -65,4 +73,41 @@ void assertSortedAndNonIntersecting(const MarkRanges & ranges)
toString(ranges));
}
size_t MarkRanges::getNumberOfMarks() const
{
size_t result = 0;
for (const auto & mark : *this)
result += mark.getNumberOfMarks();
return result;
}
void MarkRanges::serialize(WriteBuffer & out) const
{
writeIntBinary(this->size(), out);
for (const auto & [begin, end] : *this)
{
writeIntBinary(begin, out);
writeIntBinary(end, out);
}
}
String MarkRanges::describe() const
{
return fmt::format("Size: {}, Data: {}", this->size(), fmt::join(*this, ","));
}
void MarkRanges::deserialize(ReadBuffer & in)
{
size_t size = 0;
readIntBinary(size, in);
this->resize(size);
for (size_t i = 0; i < size; ++i)
{
readIntBinary((*this)[i].begin, in);
readIntBinary((*this)[i].end, in);
}
}
}

View File

@ -4,7 +4,11 @@
#include <deque>
#include <set>
#include <fmt/core.h>
#include <fmt/format.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadBuffer.h>
namespace DB
{
@ -21,12 +25,22 @@ struct MarkRange
MarkRange() = default;
MarkRange(const size_t begin_, const size_t end_) : begin{begin_}, end{end_} {}
bool operator==(const MarkRange & rhs) const;
size_t getNumberOfMarks() const;
bool operator==(const MarkRange & rhs) const;
bool operator<(const MarkRange & rhs) const;
};
using MarkRanges = std::deque<MarkRange>;
struct MarkRanges : public std::deque<MarkRange>
{
using std::deque<MarkRange>::deque;
size_t getNumberOfMarks() const;
void serialize(WriteBuffer & out) const;
String describe() const;
void deserialize(ReadBuffer & in);
};
/** Get max range.end from ranges.
*/
@ -37,3 +51,26 @@ std::string toString(const MarkRanges & ranges);
void assertSortedAndNonIntersecting(const MarkRanges & ranges);
}
template <>
struct fmt::formatter<DB::MarkRange>
{
constexpr static auto parse(format_parse_context & ctx)
{
const auto * it = ctx.begin();
const auto * end = ctx.end();
/// Only support {}.
if (it != end && *it != '}')
throw format_error("invalid format");
return it;
}
template <typename FormatContext>
auto format(const DB::MarkRange & range, FormatContext & ctx)
{
return format_to(ctx.out(), "{}", fmt::format("({}, {})", range.begin, range.end));
}
};

View File

@ -49,8 +49,7 @@ IMergeTreeSelectAlgorithm::IMergeTreeSelectAlgorithm(
UInt64 preferred_max_column_in_block_size_bytes_,
const MergeTreeReaderSettings & reader_settings_,
bool use_uncompressed_cache_,
const Names & virt_column_names_,
std::optional<ParallelReadingExtension> extension_)
const Names & virt_column_names_)
: storage(storage_)
, storage_snapshot(storage_snapshot_)
, prewhere_info(prewhere_info_)
@ -62,7 +61,6 @@ IMergeTreeSelectAlgorithm::IMergeTreeSelectAlgorithm(
, use_uncompressed_cache(use_uncompressed_cache_)
, virt_column_names(virt_column_names_)
, partition_value_type(storage.getPartitionValueType())
, extension(extension_)
{
header_without_const_virtual_columns = applyPrewhereActions(std::move(header), prewhere_info);
size_t non_const_columns_offset = header_without_const_virtual_columns.columns();
@ -115,86 +113,11 @@ std::unique_ptr<PrewhereExprInfo> IMergeTreeSelectAlgorithm::getPrewhereActions(
bool IMergeTreeSelectAlgorithm::getNewTask()
{
/// No parallel reading feature
if (!extension.has_value())
if (getNewTaskImpl())
{
if (getNewTaskImpl())
{
finalizeNewTask();
return true;
}
return false;
}
return getNewTaskParallelReading();
}
bool IMergeTreeSelectAlgorithm::getNewTaskParallelReading()
{
if (getTaskFromBuffer())
finalizeNewTask();
return true;
if (no_more_tasks)
return getDelayedTasks();
while (true)
{
/// The end of execution. No task.
if (!getNewTaskImpl())
{
no_more_tasks = true;
return getDelayedTasks();
}
splitCurrentTaskRangesAndFillBuffer();
if (getTaskFromBuffer())
return true;
}
}
bool IMergeTreeSelectAlgorithm::getTaskFromBuffer()
{
while (!buffered_ranges.empty())
{
auto ranges = std::move(buffered_ranges.front());
buffered_ranges.pop_front();
assert(!ranges.empty());
auto res = performRequestToCoordinator(ranges, /*delayed=*/false);
if (Status::Accepted == res)
return true;
/// To avoid any possibility of ignoring cancellation, exception will be thrown.
if (Status::Cancelled == res)
throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query had been cancelled");
}
return false;
}
bool IMergeTreeSelectAlgorithm::getDelayedTasks()
{
while (!delayed_tasks.empty())
{
task = std::move(delayed_tasks.front());
delayed_tasks.pop_front();
assert(!task->mark_ranges.empty());
auto res = performRequestToCoordinator(task->mark_ranges, /*delayed=*/true);
if (Status::Accepted == res)
return true;
if (Status::Cancelled == res)
break;
}
finish();
return false;
}
@ -688,170 +611,6 @@ std::unique_ptr<MergeTreeBlockSizePredictor> IMergeTreeSelectAlgorithm::getSizeP
}
IMergeTreeSelectAlgorithm::Status IMergeTreeSelectAlgorithm::performRequestToCoordinator(MarkRanges requested_ranges, bool delayed)
{
String partition_id = task->data_part->info.partition_id;
String part_name;
String projection_name;
if (task->data_part->isProjectionPart())
{
part_name = task->data_part->getParentPart()->name;
projection_name = task->data_part->name;
}
else
{
part_name = task->data_part->name;
}
PartBlockRange block_range
{
.begin = task->data_part->info.min_block,
.end = task->data_part->info.max_block
};
PartitionReadRequest request
{
.partition_id = std::move(partition_id),
.part_name = std::move(part_name),
.projection_name = std::move(projection_name),
.block_range = std::move(block_range),
.mark_ranges = std::move(requested_ranges)
};
String request_description = request.toString();
/// Consistent hashing won't work with reading in order, because at the end of the execution
/// we could possibly seek back
if (!delayed && canUseConsistentHashingForParallelReading())
{
const auto hash = request.getConsistentHash(extension->count_participating_replicas);
if (hash != extension->number_of_current_replica)
{
auto delayed_task = std::make_unique<MergeTreeReadTask>(*task); // Create a copy
delayed_task->mark_ranges = std::move(request.mark_ranges);
delayed_tasks.emplace_back(std::move(delayed_task));
LOG_TRACE(log, "Request delayed by hash: {}", request_description);
return Status::Denied;
}
}
auto optional_response = extension.value().callback(std::move(request));
if (!optional_response.has_value())
{
LOG_TRACE(log, "Request cancelled: {}", request_description);
return Status::Cancelled;
}
auto response = optional_response.value();
task->mark_ranges = std::move(response.mark_ranges);
if (response.denied || task->mark_ranges.empty())
{
LOG_TRACE(log, "Request rejected: {}", request_description);
return Status::Denied;
}
finalizeNewTask();
LOG_TRACE(log, "Request accepted: {}", request_description);
return Status::Accepted;
}
size_t IMergeTreeSelectAlgorithm::estimateMaxBatchSizeForHugeRanges()
{
/// This is an empirical number and it is so,
/// because we have an adaptive granularity by default.
const size_t average_granule_size_bytes = 1024 * 1024 * 10; // 10 MiB
/// We want to have one RTT per one gigabyte of data read from disk
/// this could be configurable.
const size_t max_size_for_one_request = 1024 * 1024 * 1024; // 1 GiB
size_t sum_average_marks_size = 0;
/// getColumnSize is not fully implemented for compact parts
if (task->data_part->getType() == IMergeTreeDataPart::Type::Compact)
{
sum_average_marks_size = average_granule_size_bytes;
}
else
{
for (const auto & name : extension->colums_to_read)
{
auto size = task->data_part->getColumnSize(name);
assert(size.marks != 0);
sum_average_marks_size += size.data_uncompressed / size.marks;
}
}
if (sum_average_marks_size == 0)
sum_average_marks_size = average_granule_size_bytes; // 10 MiB
LOG_TEST(log, "Reading from {} part, average mark size is {}",
task->data_part->getTypeName(), sum_average_marks_size);
return max_size_for_one_request / sum_average_marks_size;
}
void IMergeTreeSelectAlgorithm::splitCurrentTaskRangesAndFillBuffer()
{
const size_t max_batch_size = estimateMaxBatchSizeForHugeRanges();
size_t current_batch_size = 0;
buffered_ranges.emplace_back();
for (const auto & range : task->mark_ranges)
{
auto expand_if_needed = [&]
{
if (current_batch_size > max_batch_size)
{
buffered_ranges.emplace_back();
current_batch_size = 0;
}
};
expand_if_needed();
if (range.end - range.begin < max_batch_size)
{
buffered_ranges.back().push_back(range);
current_batch_size += range.end - range.begin;
continue;
}
auto current_begin = range.begin;
auto current_end = range.begin + max_batch_size;
while (current_end < range.end)
{
auto current_range = MarkRange{current_begin, current_end};
buffered_ranges.back().push_back(current_range);
current_batch_size += current_end - current_begin;
current_begin = current_end;
current_end = current_end + max_batch_size;
expand_if_needed();
}
if (range.end - current_begin > 0)
{
auto current_range = MarkRange{current_begin, range.end};
buffered_ranges.back().push_back(current_range);
current_batch_size += range.end - current_begin;
expand_if_needed();
}
}
if (buffered_ranges.back().empty())
buffered_ranges.pop_back();
}
IMergeTreeSelectAlgorithm::~IMergeTreeSelectAlgorithm() = default;
}

View File

@ -24,6 +24,7 @@ struct ChunkAndProgress
struct ParallelReadingExtension
{
MergeTreeAllRangesCallback all_callback;
MergeTreeReadTaskCallback callback;
size_t count_participating_replicas{0};
size_t number_of_current_replica{0};
@ -48,8 +49,7 @@ public:
UInt64 preferred_max_column_in_block_size_bytes_,
const MergeTreeReaderSettings & reader_settings_,
bool use_uncompressed_cache_,
const Names & virt_column_names_ = {},
std::optional<ParallelReadingExtension> extension_ = {});
const Names & virt_column_names_ = {});
virtual ~IMergeTreeSelectAlgorithm();
@ -90,8 +90,6 @@ protected:
size_t estimateMaxBatchSizeForHugeRanges();
virtual bool canUseConsistentHashingForParallelReading() { return false; }
/// Closes readers and unlock part locks
virtual void finish() = 0;
@ -164,11 +162,6 @@ protected:
MergeTreeReadTaskPtr task;
std::optional<ParallelReadingExtension> extension;
bool no_more_tasks{false};
std::deque<MergeTreeReadTaskPtr> delayed_tasks;
std::deque<MarkRanges> buffered_ranges;
/// This setting is used in base algorithm only to additionally limit the number of granules to read.
/// It is changed in ctor of MergeTreeThreadSelectAlgorithm.
///
@ -186,44 +179,8 @@ private:
std::atomic<bool> is_cancelled{false};
enum class Status
{
Accepted,
Cancelled,
Denied
};
/// Calls getNewTaskImpl() to get new task, then performs a request to a coordinator
/// The coordinator may modify the set of ranges to read from a part or could
/// deny the whole request. In the latter case it creates new task and retries.
/// Then it calls finalizeNewTask() to create readers for a task if it is needed.
bool getNewTask();
bool getNewTaskParallelReading();
/// After PK analysis the range of marks could be extremely big
/// We divide this range to a set smaller consecutive ranges
/// Then, depending on the type of reading (concurrent, in order or in reverse order)
/// we can calculate a consistent hash function with the number of buckets equal to
/// the number of replicas involved. And after that we can throw away some ranges with
/// hash not equals to the number of the current replica.
bool getTaskFromBuffer();
/// But we can't throw that ranges completely, because if we have different sets of parts
/// on replicas (have merged part on one, but not on another), then such a situation is possible
/// - Coordinator allows to read from a big merged part, but this part is present only on one replica.
/// And that replica calculates consistent hash and throws away some ranges
/// - Coordinator denies other replicas to read from another parts (source parts for that big one)
/// At the end, the result of the query is wrong, because we didn't read all the data.
/// So, we have to remember parts and mark ranges with hash different then current replica number.
/// An we have to ask the coordinator about its permission to read from that "delayed" parts.
/// It won't work with reading in order or reading in reverse order, because we can possibly seek back.
bool getDelayedTasks();
/// It will form a request to coordinator and
/// then reinitialize the mark ranges of this->task object
Status performRequestToCoordinator(MarkRanges requested_ranges, bool delayed);
void splitCurrentTaskRangesAndFillBuffer();
static Block applyPrewhereActions(Block block, const PrewhereInfoPtr & prewhere_info);
};

View File

@ -135,10 +135,9 @@ NameSet injectRequiredColumns(
MergeTreeReadTask::MergeTreeReadTask(
const MergeTreeData::DataPartPtr & data_part_,
const DataPartPtr & data_part_,
const MarkRanges & mark_ranges_,
size_t part_index_in_query_,
const Names & ordered_names_,
const NameSet & column_name_set_,
const MergeTreeReadTaskColumns & task_columns_,
bool remove_prewhere_column_,
@ -146,7 +145,6 @@ MergeTreeReadTask::MergeTreeReadTask(
: data_part{data_part_}
, mark_ranges{mark_ranges_}
, part_index_in_query{part_index_in_query_}
, ordered_names{ordered_names_}
, column_name_set{column_name_set_}
, task_columns{task_columns_}
, remove_prewhere_column{remove_prewhere_column_}
@ -156,7 +154,7 @@ MergeTreeReadTask::MergeTreeReadTask(
MergeTreeBlockSizePredictor::MergeTreeBlockSizePredictor(
const MergeTreeData::DataPartPtr & data_part_, const Names & columns, const Block & sample_block)
const DataPartPtr & data_part_, const Names & columns, const Block & sample_block)
: data_part(data_part_)
{
number_of_rows_in_part = data_part->rows_count;

View File

@ -2,6 +2,7 @@
#include <optional>
#include <Core/NamesAndTypes.h>
#include <Storages/StorageSnapshot.h>
#include <Storages/MergeTree/RangesInDataPart.h>
#include <Storages/MergeTree/MergeTreeRangeReader.h>
@ -17,6 +18,9 @@ class IMergeTreeDataPartInfoForReader;
using MergeTreeReadTaskPtr = std::unique_ptr<MergeTreeReadTask>;
using MergeTreeBlockSizePredictorPtr = std::shared_ptr<MergeTreeBlockSizePredictor>;
class IMergeTreeDataPart;
using DataPartPtr = std::shared_ptr<const IMergeTreeDataPart>;
/** If some of the requested columns are not in the part,
* then find out which columns may need to be read further,
@ -44,13 +48,11 @@ struct MergeTreeReadTaskColumns
struct MergeTreeReadTask
{
/// data part which should be read while performing this task
MergeTreeData::DataPartPtr data_part;
DataPartPtr data_part;
/// Ranges to read from `data_part`.
MarkRanges mark_ranges;
/// for virtual `part_index` virtual column
size_t part_index_in_query;
/// ordered list of column names used in this query, allows returning blocks with consistent ordering
const Names & ordered_names;
/// used to determine whether column should be filtered during PREWHERE or WHERE
const NameSet & column_name_set;
/// column names to read during PREWHERE and WHERE
@ -68,10 +70,9 @@ struct MergeTreeReadTask
bool isFinished() const { return mark_ranges.empty() && range_reader.isCurrentRangeFinished(); }
MergeTreeReadTask(
const MergeTreeData::DataPartPtr & data_part_,
const DataPartPtr & data_part_,
const MarkRanges & mark_ranges_,
size_t part_index_in_query_,
const Names & ordered_names_,
const NameSet & column_name_set_,
const MergeTreeReadTaskColumns & task_columns_,
bool remove_prewhere_column_,
@ -88,7 +89,7 @@ MergeTreeReadTaskColumns getReadTaskColumns(
struct MergeTreeBlockSizePredictor
{
MergeTreeBlockSizePredictor(const MergeTreeData::DataPartPtr & data_part_, const Names & columns, const Block & sample_block);
MergeTreeBlockSizePredictor(const DataPartPtr & data_part_, const Names & columns, const Block & sample_block);
/// Reset some values for correct statistics calculating
void startBlock();
@ -137,7 +138,7 @@ struct MergeTreeBlockSizePredictor
protected:
MergeTreeData::DataPartPtr data_part;
DataPartPtr data_part;
struct ColumnInfo
{

View File

@ -1,36 +1,48 @@
#include "Storages/MergeTree/MergeTreeDataPartBuilder.h"
#include <Storages/MergeTree/MergeTreeData.h>
#include <AggregateFunctions/AggregateFunctionCount.h>
#include <Backups/BackupEntriesCollector.h>
#include <Backups/BackupEntryFromSmallFile.h>
#include <Backups/BackupEntryWrappedWith.h>
#include <Backups/IBackup.h>
#include <Backups/RestorerFromBackup.h>
#include <Common/escapeForFileName.h>
#include <Common/Increment.h>
#include <Common/noexcept_scope.h>
#include <Common/quoteString.h>
#include <Common/scope_guard_safe.h>
#include <Common/SimpleIncrement.h>
#include <Common/Stopwatch.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/typeid_cast.h>
#include <Compression/CompressedReadBuffer.h>
#include <Core/QueryProcessingStage.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/hasNullable.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/ObjectUtils.h>
#include <DataTypes/hasNullable.h>
#include <Disks/createVolume.h>
#include <Disks/ObjectStorages/DiskObjectStorage.h>
#include <Disks/TemporaryFileOnDisk.h>
#include <Functions/IFunction.h>
#include <Interpreters/Aggregator.h>
#include <Interpreters/Context.h>
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/inplaceBlockConversions.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/MergeTreeTransaction.h>
#include <Interpreters/PartLog.h>
#include <Interpreters/TransactionLog.h>
#include <Interpreters/TreeRewriter.h>
#include <IO/S3Common.h>
#include <IO/Operators.h>
#include <IO/WriteBufferFromString.h>
#include <IO/S3Common.h>
#include <Interpreters/Aggregator.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/PartLog.h>
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/inplaceBlockConversions.h>
#include <Interpreters/MergeTreeTransaction.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/TransactionLog.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/convertFieldToType.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTNameTypePair.h>
@ -40,29 +52,19 @@
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <Processors/Formats/IInputFormat.h>
#include <Processors/QueryPlan/ReadFromMergeTree.h>
#include <Storages/AlterCommands.h>
#include <Storages/Freeze.h>
#include <Storages/MergeTree/checkDataPart.h>
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
#include <Storages/MergeTree/MergeTreeDataPartCompact.h>
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
#include <Storages/MergeTree/MergeTreeDataPartWide.h>
#include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
#include <Storages/MergeTree/checkDataPart.h>
#include <Storages/StorageMergeTree.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/VirtualColumnUtils.h>
#include <Storages/Freeze.h>
#include <Common/Increment.h>
#include <Common/SimpleIncrement.h>
#include <Common/Stopwatch.h>
#include <Common/StringUtils/StringUtils.h>
#include <Disks/TemporaryFileOnDisk.h>
#include <Common/escapeForFileName.h>
#include <Common/quoteString.h>
#include <Common/typeid_cast.h>
#include <Common/noexcept_scope.h>
#include <Processors/Formats/IInputFormat.h>
#include <AggregateFunctions/AggregateFunctionCount.h>
#include <Common/scope_guard_safe.h>
#include <boost/range/algorithm_ext/erase.hpp>
#include <boost/algorithm/string/join.hpp>
@ -82,6 +84,7 @@
#include <filesystem>
#include <fmt/format.h>
#include <Poco/Logger.h>
template <>
struct fmt::formatter<DB::DataPartPtr> : fmt::formatter<std::string>
@ -6807,6 +6810,14 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage(
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info) const
{
if (query_context->getClientInfo().collaborate_with_initiator)
return QueryProcessingStage::Enum::FetchColumns;
if (query_context->getSettingsRef().allow_experimental_parallel_reading_from_replicas
&& !query_context->getClientInfo().collaborate_with_initiator
&& to_stage >= QueryProcessingStage::WithMergeableState)
return QueryProcessingStage::Enum::WithMergeableState;
if (to_stage >= QueryProcessingStage::Enum::WithMergeableState)
{
if (auto projection = getQueryProcessingStageWithAggregateProjection(query_context, storage_snapshot, query_info))

View File

@ -876,7 +876,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
ReadFromMergeTree::IndexStats & index_stats,
bool use_skip_indexes)
{
RangesInDataParts parts_with_ranges(parts.size());
RangesInDataParts parts_with_ranges;
parts_with_ranges.resize(parts.size());
const Settings & settings = context->getSettingsRef();
/// Let's start analyzing all useful indices
@ -1010,7 +1011,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
if (metadata_snapshot->hasPrimaryKey())
ranges.ranges = markRangesFromPKRange(part, metadata_snapshot, key_condition, settings, log);
else if (total_marks_count)
ranges.ranges = MarkRanges{MarkRange{0, total_marks_count}};
ranges.ranges = MarkRanges{{MarkRange{0, total_marks_count}}};
sum_marks_pk.fetch_add(ranges.getMarksCount(), std::memory_order_relaxed);

View File

@ -1,4 +1,6 @@
#include <Storages/MergeTree/MergeTreeInOrderSelectProcessor.h>
#include "Storages/MergeTree/RangesInDataPart.h"
#include <Storages/MergeTree/IntersectionsIndexes.h>
namespace DB
{
@ -18,23 +20,42 @@ try
initializeReaders();
MarkRanges mark_ranges_for_task;
/// If we need to read few rows, set one range per task to reduce number of read data.
if (has_limit_below_one_block)
if (!pool)
{
mark_ranges_for_task = { std::move(all_mark_ranges.front()) };
all_mark_ranges.pop_front();
/// If we need to read few rows, set one range per task to reduce number of read data.
if (has_limit_below_one_block)
{
mark_ranges_for_task = MarkRanges{};
mark_ranges_for_task.emplace_front(std::move(all_mark_ranges.front()));
all_mark_ranges.pop_front();
}
else
{
mark_ranges_for_task = std::move(all_mark_ranges);
all_mark_ranges.clear();
}
}
else
{
mark_ranges_for_task = std::move(all_mark_ranges);
all_mark_ranges.clear();
auto description = RangesInDataPartDescription{
.info = data_part->info,
/// We just ignore all the distribution done before
/// Everything will be done on coordinator side
.ranges = {},
};
mark_ranges_for_task = pool->getNewTask(description);
if (mark_ranges_for_task.empty())
return false;
}
auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr
: getSizePredictor(data_part, task_columns, sample_block);
task = std::make_unique<MergeTreeReadTask>(
data_part, mark_ranges_for_task, part_index_in_query, ordered_names, column_name_set, task_columns,
data_part, mark_ranges_for_task, part_index_in_query, column_name_set, task_columns,
prewhere_info && prewhere_info->remove_prewhere_column,
std::move(size_predictor));

View File

@ -2,6 +2,7 @@
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include "Core/ProtocolDefines.h"
namespace DB
{
@ -10,6 +11,7 @@ namespace ErrorCodes
{
extern const int BAD_DATA_PART_NAME;
extern const int INVALID_PARTITION_VALUE;
extern const int UNKNOWN_FORMAT_VERSION;
}
@ -253,6 +255,43 @@ String MergeTreePartInfo::getPartNameV0(DayNum left_date, DayNum right_date) con
return wb.str();
}
void MergeTreePartInfo::serialize(WriteBuffer & out) const
{
UInt64 version = DBMS_MERGE_TREE_PART_INFO_VERSION;
/// Must be the first
writeIntBinary(version, out);
writeStringBinary(partition_id, out);
writeIntBinary(min_block, out);
writeIntBinary(max_block, out);
writeIntBinary(level, out);
writeIntBinary(mutation, out);
writeBoolText(use_leagcy_max_level, out);
}
String MergeTreePartInfo::describe() const
{
return getPartNameV1();
}
void MergeTreePartInfo::deserialize(ReadBuffer & in)
{
UInt64 version;
readIntBinary(version, in);
if (version != DBMS_MERGE_TREE_PART_INFO_VERSION)
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Version for MergeTreePart info mismatched. Got: {}, supported version: {}",
version, DBMS_MERGE_TREE_PART_INFO_VERSION);
readStringBinary(partition_id, in);
readIntBinary(min_block, in);
readIntBinary(max_block, in);
readIntBinary(level, in);
readIntBinary(mutation, in);
readBoolText(use_leagcy_max_level, in);
}
DetachedPartInfo DetachedPartInfo::parseDetachedPartName(
const DiskPtr & disk, std::string_view dir_name, MergeTreeDataFormatVersion format_version)
{

View File

@ -7,6 +7,8 @@
#include <array>
#include <base/types.h>
#include <base/DayNum.h>
#include <IO/ReadBuffer.h>
#include <IO/WriteBuffer.h>
#include <Storages/MergeTree/MergeTreeDataFormatVersion.h>
@ -112,6 +114,10 @@ struct MergeTreePartInfo
return static_cast<UInt64>(max_block - min_block + 1);
}
void serialize(WriteBuffer & out) const;
String describe() const;
void deserialize(ReadBuffer & in);
/// Simple sanity check for partition ID. Checking that it's not too long or too short, doesn't contain a lot of '_'.
static void validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version);

View File

@ -1,6 +1,7 @@
#include <Storages/MergeTree/MergeTreeReadPool.h>
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
#include "Common/Stopwatch.h"
#include <Common/formatReadable.h>
#include <base/range.h>
@ -18,6 +19,52 @@ namespace ErrorCodes
namespace DB
{
std::vector<size_t> IMergeTreeReadPool::fillPerPartInfo(const RangesInDataParts & parts)
{
std::vector<size_t> per_part_sum_marks;
Block sample_block = storage_snapshot->metadata->getSampleBlock();
is_part_on_remote_disk.resize(parts.size());
for (const auto i : collections::range(0, parts.size()))
{
const auto & part = parts[i];
#ifndef NDEBUG
assertSortedAndNonIntersecting(part.ranges);
#endif
bool part_on_remote_disk = part.data_part->isStoredOnRemoteDisk();
is_part_on_remote_disk[i] = part_on_remote_disk;
do_not_steal_tasks |= part_on_remote_disk;
/// Read marks for every data part.
size_t sum_marks = 0;
for (const auto & range : part.ranges)
sum_marks += range.end - range.begin;
per_part_sum_marks.push_back(sum_marks);
auto task_columns = getReadTaskColumns(
LoadedMergeTreeDataPartInfoForReader(part.data_part), storage_snapshot,
column_names, virtual_column_names, prewhere_info, /*with_subcolumns=*/ true);
auto size_predictor = !predict_block_size_bytes ? nullptr
: IMergeTreeSelectAlgorithm::getSizePredictor(part.data_part, task_columns, sample_block);
auto & per_part = per_part_params.emplace_back();
per_part.data_part = part;
per_part.size_predictor = std::move(size_predictor);
/// will be used to distinguish between PREWHERE and WHERE columns when applying filter
const auto & required_column_names = task_columns.columns.getNames();
per_part.column_name_set = {required_column_names.begin(), required_column_names.end()};
per_part.task_columns = std::move(task_columns);
}
return per_part_sum_marks;
}
MergeTreeReadPool::MergeTreeReadPool(
size_t threads_,
size_t sum_marks_,
@ -30,23 +77,25 @@ MergeTreeReadPool::MergeTreeReadPool(
const BackoffSettings & backoff_settings_,
size_t preferred_block_size_bytes_,
bool do_not_steal_tasks_)
: backoff_settings{backoff_settings_}
: IMergeTreeReadPool(
storage_snapshot_,
column_names_,
virtual_column_names_,
min_marks_for_concurrent_read_,
prewhere_info_,
std::move(parts_),
(preferred_block_size_bytes_ > 0),
do_not_steal_tasks_)
, backoff_settings{backoff_settings_}
, backoff_state{threads_}
, storage_snapshot{storage_snapshot_}
, column_names{column_names_}
, virtual_column_names{virtual_column_names_}
, do_not_steal_tasks{do_not_steal_tasks_}
, predict_block_size_bytes{preferred_block_size_bytes_ > 0}
, prewhere_info{prewhere_info_}
, parts_ranges{std::move(parts_)}
{
/// parts don't contain duplicate MergeTreeDataPart's.
const auto per_part_sum_marks = fillPerPartInfo(parts_ranges);
fillPerThreadInfo(threads_, sum_marks_, per_part_sum_marks, parts_ranges, min_marks_for_concurrent_read_);
fillPerThreadInfo(threads_, sum_marks_, per_part_sum_marks, parts_ranges);
}
MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t min_marks_to_read, size_t thread, const Names & ordered_names)
MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t thread)
{
const std::lock_guard lock{mutex};
@ -86,18 +135,18 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t min_marks_to_read, size_t
auto & thread_task = thread_tasks.parts_and_ranges.back();
const auto part_idx = thread_task.part_idx;
auto & part = parts_with_idx[part_idx];
auto & part = per_part_params[part_idx].data_part;
auto & marks_in_part = thread_tasks.sum_marks_in_parts.back();
size_t need_marks;
if (is_part_on_remote_disk[part_idx]) /// For better performance with remote disks
need_marks = marks_in_part;
else /// Get whole part to read if it is small enough.
need_marks = std::min(marks_in_part, min_marks_to_read);
need_marks = std::min(marks_in_part, min_marks_for_concurrent_read);
/// Do not leave too little rows in part for next time.
if (marks_in_part > need_marks &&
marks_in_part - need_marks < min_marks_to_read)
marks_in_part - need_marks < min_marks_for_concurrent_read)
need_marks = marks_in_part;
MarkRanges ranges_to_get_from_part;
@ -142,7 +191,7 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t min_marks_to_read, size_t
: std::make_unique<MergeTreeBlockSizePredictor>(*per_part.size_predictor); /// make a copy
return std::make_unique<MergeTreeReadTask>(
part.data_part, ranges_to_get_from_part, part.part_index_in_query, ordered_names,
part.data_part, ranges_to_get_from_part, part.part_index_in_query,
per_part.column_name_set, per_part.task_columns,
prewhere_info && prewhere_info->remove_prewhere_column, std::move(curr_task_size_predictor));
}
@ -192,56 +241,9 @@ void MergeTreeReadPool::profileFeedback(ReadBufferFromFileBase::ProfileInfo info
}
std::vector<size_t> MergeTreeReadPool::fillPerPartInfo(const RangesInDataParts & parts)
{
std::vector<size_t> per_part_sum_marks;
Block sample_block = storage_snapshot->metadata->getSampleBlock();
is_part_on_remote_disk.resize(parts.size());
for (const auto i : collections::range(0, parts.size()))
{
const auto & part = parts[i];
#ifndef NDEBUG
assertSortedAndNonIntersecting(part.ranges);
#endif
bool part_on_remote_disk = part.data_part->isStoredOnRemoteDisk();
is_part_on_remote_disk[i] = part_on_remote_disk;
do_not_steal_tasks |= part_on_remote_disk;
/// Read marks for every data part.
size_t sum_marks = 0;
for (const auto & range : part.ranges)
sum_marks += range.end - range.begin;
per_part_sum_marks.push_back(sum_marks);
auto task_columns = getReadTaskColumns(
LoadedMergeTreeDataPartInfoForReader(part.data_part), storage_snapshot,
column_names, virtual_column_names, prewhere_info, /*with_subcolumns=*/ true);
auto size_predictor = !predict_block_size_bytes ? nullptr
: IMergeTreeSelectAlgorithm::getSizePredictor(part.data_part, task_columns, sample_block);
auto & per_part = per_part_params.emplace_back();
per_part.size_predictor = std::move(size_predictor);
/// will be used to distinguish between PREWHERE and WHERE columns when applying filter
const auto & required_column_names = task_columns.columns.getNames();
per_part.column_name_set = {required_column_names.begin(), required_column_names.end()};
per_part.task_columns = std::move(task_columns);
parts_with_idx.push_back({ part.data_part, part.part_index_in_query });
}
return per_part_sum_marks;
}
void MergeTreeReadPool::fillPerThreadInfo(
size_t threads, size_t sum_marks, std::vector<size_t> per_part_sum_marks,
const RangesInDataParts & parts, size_t min_marks_for_concurrent_read)
const RangesInDataParts & parts)
{
threads_tasks.resize(threads);
if (parts.empty())
@ -355,4 +357,148 @@ void MergeTreeReadPool::fillPerThreadInfo(
}
MergeTreeReadPoolParallelReplicas::~MergeTreeReadPoolParallelReplicas() = default;
Block MergeTreeReadPoolParallelReplicas::getHeader() const
{
return storage_snapshot->getSampleBlockForColumns(extension.colums_to_read);
}
MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t thread)
{
/// This parameter is needed only to satisfy the interface
UNUSED(thread);
std::lock_guard lock(mutex);
if (no_more_tasks_available)
return nullptr;
if (buffered_ranges.empty())
{
auto result = extension.callback(ParallelReadRequest{
.replica_num = extension.number_of_current_replica, .min_number_of_marks = min_marks_for_concurrent_read * threads});
if (!result || result->finish)
{
no_more_tasks_available = true;
return nullptr;
}
buffered_ranges = std::move(result->description);
}
if (buffered_ranges.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "No tasks to read. This is a bug");
auto & current_task = buffered_ranges.front();
RangesInDataPart part;
size_t part_idx = 0;
for (size_t index = 0; index < per_part_params.size(); ++index)
{
auto & other_part = per_part_params[index];
if (other_part.data_part.data_part->info == current_task.info)
{
part = other_part.data_part;
part_idx = index;
break;
}
}
MarkRanges ranges_to_read;
size_t current_sum_marks = 0;
while (current_sum_marks < min_marks_for_concurrent_read && !current_task.ranges.empty())
{
auto diff = min_marks_for_concurrent_read - current_sum_marks;
auto range = current_task.ranges.front();
if (range.getNumberOfMarks() > diff)
{
auto new_range = range;
new_range.end = range.begin + diff;
range.begin += diff;
current_task.ranges.front() = range;
ranges_to_read.push_back(new_range);
current_sum_marks += new_range.getNumberOfMarks();
continue;
}
ranges_to_read.push_back(range);
current_sum_marks += range.getNumberOfMarks();
current_task.ranges.pop_front();
}
if (current_task.ranges.empty())
buffered_ranges.pop_front();
const auto & per_part = per_part_params[part_idx];
auto curr_task_size_predictor
= !per_part.size_predictor ? nullptr : std::make_unique<MergeTreeBlockSizePredictor>(*per_part.size_predictor); /// make a copy
return std::make_unique<MergeTreeReadTask>(
part.data_part,
ranges_to_read,
part.part_index_in_query,
per_part.column_name_set,
per_part.task_columns,
prewhere_info && prewhere_info->remove_prewhere_column,
std::move(curr_task_size_predictor));
}
MarkRanges MergeTreeInOrderReadPoolParallelReplicas::getNewTask(RangesInDataPartDescription description)
{
std::lock_guard lock(mutex);
auto get_from_buffer = [&]() -> std::optional<MarkRanges>
{
for (auto & desc : buffered_tasks)
{
if (desc.info == description.info && !desc.ranges.empty())
{
auto result = std::move(desc.ranges);
desc.ranges = MarkRanges{};
return result;
}
}
return std::nullopt;
};
if (auto result = get_from_buffer(); result)
return result.value();
if (no_more_tasks)
return {};
auto response = extension.callback(ParallelReadRequest{
.mode = mode,
.replica_num = extension.number_of_current_replica,
.min_number_of_marks = min_marks_for_concurrent_read * request.size(),
.description = request,
});
if (!response || response->description.empty() || response->finish)
{
no_more_tasks = true;
return {};
}
/// Fill the buffer
for (size_t i = 0; i < request.size(); ++i)
{
auto & new_ranges = response->description[i].ranges;
auto & old_ranges = buffered_tasks[i].ranges;
std::move(new_ranges.begin(), new_ranges.end(), std::back_inserter(old_ranges));
}
if (auto result = get_from_buffer(); result)
return result.value();
return {};
}
}

View File

@ -1,10 +1,13 @@
#pragma once
#include <Core/NamesAndTypes.h>
#include <Storages/MergeTree/RangesInDataPart.h>
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/RangesInDataPart.h>
#include <Storages/MergeTree/RequestResponse.h>
#include <Storages/SelectQueryInfo.h>
#include <mutex>
@ -13,6 +16,64 @@ namespace DB
using MergeTreeReadTaskPtr = std::unique_ptr<MergeTreeReadTask>;
class IMergeTreeReadPool
{
public:
IMergeTreeReadPool(
StorageSnapshotPtr storage_snapshot_,
Names column_names_,
Names virtual_column_names_,
size_t min_marks_for_concurrent_read_,
PrewhereInfoPtr prewhere_info_,
RangesInDataParts parts_ranges_,
bool predict_block_size_bytes_,
bool do_not_steal_tasks_)
: storage_snapshot(storage_snapshot_)
, column_names(column_names_)
, virtual_column_names(virtual_column_names_)
, min_marks_for_concurrent_read(min_marks_for_concurrent_read_)
, prewhere_info(prewhere_info_)
, parts_ranges(parts_ranges_)
, predict_block_size_bytes(predict_block_size_bytes_)
, do_not_steal_tasks(do_not_steal_tasks_)
{}
virtual MergeTreeReadTaskPtr getTask(size_t thread) = 0;
virtual Block getHeader() const = 0;
virtual void profileFeedback(ReadBufferFromFileBase::ProfileInfo info) = 0;
virtual ~IMergeTreeReadPool() = default;
protected:
std::vector<size_t> fillPerPartInfo(const RangesInDataParts & parts);
/// Initialized in constructor
StorageSnapshotPtr storage_snapshot;
const Names column_names;
const Names virtual_column_names;
size_t min_marks_for_concurrent_read{0};
PrewhereInfoPtr prewhere_info;
RangesInDataParts parts_ranges;
bool predict_block_size_bytes;
bool do_not_steal_tasks;
struct PerPartParams
{
MergeTreeReadTaskColumns task_columns;
NameSet column_name_set;
MergeTreeBlockSizePredictorPtr size_predictor;
RangesInDataPart data_part;
};
std::vector<PerPartParams> per_part_params;
std::vector<bool> is_part_on_remote_disk;
mutable std::mutex mutex;
};
using IMergeTreeReadPoolPtr = std::shared_ptr<IMergeTreeReadPool>;
/** Provides read tasks for MergeTreeThreadSelectProcessor`s in fine-grained batches, allowing for more
* uniform distribution of work amongst multiple threads. All parts and their ranges are divided into `threads`
* workloads with at most `sum_marks / threads` marks. Then, threads are performing reads from these workloads
@ -20,7 +81,7 @@ using MergeTreeReadTaskPtr = std::unique_ptr<MergeTreeReadTask>;
* it's workload, it either is signaled that no more work is available (`do_not_steal_tasks == false`) or
* continues taking small batches from other threads' workloads (`do_not_steal_tasks == true`).
*/
class MergeTreeReadPool : private boost::noncopyable
class MergeTreeReadPool final: public IMergeTreeReadPool, private boost::noncopyable
{
public:
/** Pull could dynamically lower (backoff) number of threads, if read operation are too slow.
@ -82,47 +143,22 @@ public:
size_t preferred_block_size_bytes_,
bool do_not_steal_tasks_ = false);
MergeTreeReadTaskPtr getTask(size_t min_marks_to_read, size_t thread, const Names & ordered_names);
~MergeTreeReadPool() override = default;
MergeTreeReadTaskPtr getTask(size_t thread) override;
/** Each worker could call this method and pass information about read performance.
* If read performance is too low, pool could decide to lower number of threads: do not assign more tasks to several threads.
* This allows to overcome excessive load to disk subsystem, when reads are not from page cache.
*/
void profileFeedback(ReadBufferFromFileBase::ProfileInfo info);
void profileFeedback(ReadBufferFromFileBase::ProfileInfo info) override;
Block getHeader() const;
Block getHeader() const override;
private:
std::vector<size_t> fillPerPartInfo(const RangesInDataParts & parts);
void fillPerThreadInfo(
size_t threads, size_t sum_marks, std::vector<size_t> per_part_sum_marks,
const RangesInDataParts & parts, size_t min_marks_for_concurrent_read);
StorageSnapshotPtr storage_snapshot;
const Names column_names;
const Names virtual_column_names;
bool do_not_steal_tasks;
bool predict_block_size_bytes;
struct PerPartParams
{
MergeTreeReadTaskColumns task_columns;
NameSet column_name_set;
MergeTreeBlockSizePredictorPtr size_predictor;
};
std::vector<PerPartParams> per_part_params;
PrewhereInfoPtr prewhere_info;
struct Part
{
MergeTreeData::DataPartPtr data_part;
size_t part_index_in_query;
};
std::vector<Part> parts_with_idx;
const RangesInDataParts & parts);
struct ThreadTask
{
@ -137,18 +173,104 @@ private:
};
std::vector<ThreadTask> threads_tasks;
std::set<size_t> remaining_thread_tasks;
RangesInDataParts parts_ranges;
mutable std::mutex mutex;
Poco::Logger * log = &Poco::Logger::get("MergeTreeReadPool");
std::vector<bool> is_part_on_remote_disk;
};
using MergeTreeReadPoolPtr = std::shared_ptr<MergeTreeReadPool>;
class MergeTreeReadPoolParallelReplicas : public IMergeTreeReadPool, private boost::noncopyable
{
public:
MergeTreeReadPoolParallelReplicas(
StorageSnapshotPtr storage_snapshot_,
size_t threads_,
ParallelReadingExtension extension_,
const RangesInDataParts & parts_,
const PrewhereInfoPtr & prewhere_info_,
const Names & column_names_,
const Names & virtual_column_names_,
size_t min_marks_for_concurrent_read_
)
: IMergeTreeReadPool(
storage_snapshot_,
column_names_,
virtual_column_names_,
min_marks_for_concurrent_read_,
prewhere_info_,
parts_,
/*predict_block_size*/false,
/*do_not_steal_tasks*/false)
, extension(extension_)
, threads(threads_)
{
fillPerPartInfo(parts_ranges);
extension.all_callback({
.description = parts_ranges.getDescriptions(),
.replica_num = extension.number_of_current_replica
});
}
~MergeTreeReadPoolParallelReplicas() override;
MergeTreeReadTaskPtr getTask(size_t thread) override;
Block getHeader() const override;
void profileFeedback(ReadBufferFromFileBase::ProfileInfo) override {}
private:
ParallelReadingExtension extension;
RangesInDataPartsDescription buffered_ranges;
size_t threads;
bool no_more_tasks_available{false};
Poco::Logger * log = &Poco::Logger::get("MergeTreeReadPoolParallelReplicas");
};
using MergeTreeReadPoolParallelReplicasPtr = std::shared_ptr<MergeTreeReadPoolParallelReplicas>;
class MergeTreeInOrderReadPoolParallelReplicas : private boost::noncopyable
{
public:
MergeTreeInOrderReadPoolParallelReplicas(
RangesInDataParts parts_,
ParallelReadingExtension extension_,
CoordinationMode mode_,
size_t min_marks_for_concurrent_read_)
: parts_ranges(parts_)
, extension(extension_)
, mode(mode_)
, min_marks_for_concurrent_read(min_marks_for_concurrent_read_)
{
for (const auto & part : parts_ranges)
request.push_back({part.data_part->info, MarkRanges{}});
for (const auto & part : parts_ranges)
buffered_tasks.push_back({part.data_part->info, MarkRanges{}});
extension.all_callback({
.description = parts_ranges.getDescriptions(),
.replica_num = extension.number_of_current_replica
});
}
MarkRanges getNewTask(RangesInDataPartDescription description);
RangesInDataParts parts_ranges;
ParallelReadingExtension extension;
CoordinationMode mode;
size_t min_marks_for_concurrent_read{0};
bool no_more_tasks{false};
RangesInDataPartsDescription request;
RangesInDataPartsDescription buffered_tasks;
std::mutex mutex;
};
using MergeTreeInOrderReadPoolParallelReplicasPtr = std::shared_ptr<MergeTreeInOrderReadPoolParallelReplicas>;
}

View File

@ -1,4 +1,6 @@
#include <Storages/MergeTree/MergeTreeReverseSelectProcessor.h>
#include <Storages/MergeTree/IntersectionsIndexes.h>
#include "Storages/MergeTree/MergeTreeBaseSelectProcessor.h"
namespace DB
{
@ -10,6 +12,22 @@ namespace ErrorCodes
bool MergeTreeReverseSelectAlgorithm::getNewTaskImpl()
try
{
if (pool)
return getNewTaskParallelReplicas();
else
return getNewTaskOrdinaryReading();
}
catch (...)
{
/// Suspicion of the broken part. A part is added to the queue for verification.
if (getCurrentExceptionCode() != ErrorCodes::MEMORY_LIMIT_EXCEEDED)
storage.reportBrokenPart(data_part);
throw;
}
bool MergeTreeReverseSelectAlgorithm::getNewTaskOrdinaryReading()
{
if (chunks.empty() && all_mark_ranges.empty())
return false;
@ -23,25 +41,57 @@ try
initializeReaders();
/// Read ranges from right to left.
MarkRanges mark_ranges_for_task = { all_mark_ranges.back() };
MarkRanges mark_ranges_for_task{std::move(all_mark_ranges.back())};
all_mark_ranges.pop_back();
auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr
: getSizePredictor(data_part, task_columns, sample_block);
task = std::make_unique<MergeTreeReadTask>(
data_part, mark_ranges_for_task, part_index_in_query, ordered_names, column_name_set,
data_part, mark_ranges_for_task, part_index_in_query, column_name_set,
task_columns, prewhere_info && prewhere_info->remove_prewhere_column,
std::move(size_predictor));
return true;
}
catch (...)
bool MergeTreeReverseSelectAlgorithm::getNewTaskParallelReplicas()
{
/// Suspicion of the broken part. A part is added to the queue for verification.
if (getCurrentExceptionCode() != ErrorCodes::MEMORY_LIMIT_EXCEEDED)
storage.reportBrokenPart(data_part);
throw;
if (chunks.empty() && no_more_tasks)
return false;
/// We have some blocks to return in buffer.
/// Return true to continue reading, but actually don't create a task.
if (no_more_tasks)
return true;
if (!reader)
initializeReaders();
auto description = RangesInDataPartDescription{
.info = data_part->info,
/// We just ignore all the distribution done before
/// Everything will be done on coordinator side
.ranges = {},
};
auto mark_ranges_for_task = pool->getNewTask(description);
if (mark_ranges_for_task.empty())
{
/// If we have chunks in buffer - return true to continue reading from them
return !chunks.empty();
}
auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr
: getSizePredictor(data_part, task_columns, sample_block);
task = std::make_unique<MergeTreeReadTask>(
data_part, mark_ranges_for_task, part_index_in_query, column_name_set,
task_columns, prewhere_info && prewhere_info->remove_prewhere_column,
std::move(size_predictor));
return true;
}
MergeTreeReverseSelectAlgorithm::BlockAndProgress MergeTreeReverseSelectAlgorithm::readFromPart()

View File

@ -27,9 +27,16 @@ private:
bool getNewTaskImpl() override;
void finalizeNewTask() override {}
bool getNewTaskParallelReplicas();
bool getNewTaskOrdinaryReading();
BlockAndProgress readFromPart() override;
std::vector<BlockAndProgress> chunks;
/// Used for parallel replicas
bool no_more_tasks{false};
Poco::Logger * log = &Poco::Logger::get("MergeTreeReverseSelectProcessor");
};

View File

@ -21,21 +21,22 @@ MergeTreeSelectAlgorithm::MergeTreeSelectAlgorithm(
const PrewhereInfoPtr & prewhere_info_,
ExpressionActionsSettings actions_settings,
const MergeTreeReaderSettings & reader_settings_,
MergeTreeInOrderReadPoolParallelReplicasPtr pool_,
const Names & virt_column_names_,
size_t part_index_in_query_,
bool has_limit_below_one_block_,
std::optional<ParallelReadingExtension> extension_)
bool has_limit_below_one_block_)
: IMergeTreeSelectAlgorithm{
storage_snapshot_->getSampleBlockForColumns(required_columns_),
storage_, storage_snapshot_, prewhere_info_, std::move(actions_settings), max_block_size_rows_,
preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_,
reader_settings_, use_uncompressed_cache_, virt_column_names_, extension_},
reader_settings_, use_uncompressed_cache_, virt_column_names_},
required_columns{std::move(required_columns_)},
data_part{owned_data_part_},
sample_block(storage_snapshot_->metadata->getSampleBlock()),
all_mark_ranges(std::move(mark_ranges_)),
part_index_in_query(part_index_in_query_),
has_limit_below_one_block(has_limit_below_one_block_),
pool(pool_),
total_rows(data_part->index_granularity.getRowsCountInRanges(all_mark_ranges))
{
ordered_names = header_without_const_virtual_columns.getNames();

View File

@ -3,6 +3,7 @@
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MarkRange.h>
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
#include <Storages/MergeTree/MergeTreeReadPool.h>
#include <Storages/SelectQueryInfo.h>
@ -29,10 +30,10 @@ public:
const PrewhereInfoPtr & prewhere_info,
ExpressionActionsSettings actions_settings,
const MergeTreeReaderSettings & reader_settings,
MergeTreeInOrderReadPoolParallelReplicasPtr pool_,
const Names & virt_column_names = {},
size_t part_index_in_query_ = 0,
bool has_limit_below_one_block_ = false,
std::optional<ParallelReadingExtension> extension_ = {});
bool has_limit_below_one_block_ = false);
~MergeTreeSelectAlgorithm() override;
@ -64,6 +65,9 @@ protected:
/// It reduces amount of read data for queries with small LIMIT.
bool has_limit_below_one_block = false;
/// Pool for reading in order
MergeTreeInOrderReadPoolParallelReplicasPtr pool;
size_t total_rows = 0;
};

View File

@ -7,16 +7,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
MergeTreeThreadSelectAlgorithm::MergeTreeThreadSelectAlgorithm(
size_t thread_,
const MergeTreeReadPoolPtr & pool_,
size_t min_marks_to_read_,
UInt64 max_block_size_rows_,
IMergeTreeReadPoolPtr pool_,
size_t min_marks_for_concurrent_read_,
size_t max_block_size_rows_,
size_t preferred_block_size_bytes_,
size_t preferred_max_column_in_block_size_bytes_,
const MergeTreeData & storage_,
@ -25,74 +20,22 @@ MergeTreeThreadSelectAlgorithm::MergeTreeThreadSelectAlgorithm(
const PrewhereInfoPtr & prewhere_info_,
ExpressionActionsSettings actions_settings,
const MergeTreeReaderSettings & reader_settings_,
const Names & virt_column_names_,
std::optional<ParallelReadingExtension> extension_)
const Names & virt_column_names_)
:
IMergeTreeSelectAlgorithm{
pool_->getHeader(), storage_, storage_snapshot_, prewhere_info_, std::move(actions_settings), max_block_size_rows_,
preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_,
reader_settings_, use_uncompressed_cache_, virt_column_names_, extension_},
reader_settings_, use_uncompressed_cache_, virt_column_names_},
thread{thread_},
pool{pool_}
pool{std::move(pool_)}
{
/// round min_marks_to_read up to nearest multiple of block_size expressed in marks
/// If granularity is adaptive it doesn't make sense
/// Maybe it will make sense to add settings `max_block_size_bytes`
if (max_block_size_rows && !storage.canUseAdaptiveGranularity())
{
size_t fixed_index_granularity = storage.getSettings()->index_granularity;
min_marks_to_read = (min_marks_to_read_ * fixed_index_granularity + max_block_size_rows - 1)
/ max_block_size_rows * max_block_size_rows / fixed_index_granularity;
}
else if (extension.has_value())
{
/// Parallel reading from replicas is enabled.
/// We try to estimate the average number of bytes in a granule
/// to make one request over the network per one gigabyte of data
/// Actually we will ask MergeTreeReadPool to provide us heavier tasks to read
/// because the most part of each task will be postponed
/// (due to using consistent hash for better cache affinity)
const size_t amount_of_read_bytes_per_one_request = 1024 * 1024 * 1024; // 1GiB
/// In case of reading from compact parts (for which we can't estimate the average size of marks)
/// we will use this value
const size_t empirical_size_of_mark = 1024 * 1024 * 10; // 10 MiB
if (extension->colums_to_read.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "A set of column to read is empty. It is a bug");
size_t sum_average_marks_size = 0;
auto column_sizes = storage.getColumnSizes();
for (const auto & name : extension->colums_to_read)
{
auto it = column_sizes.find(name);
if (it == column_sizes.end())
continue;
auto size = it->second;
if (size.data_compressed == 0 || size.data_uncompressed == 0 || size.marks == 0)
continue;
sum_average_marks_size += size.data_uncompressed / size.marks;
}
if (sum_average_marks_size == 0)
sum_average_marks_size = empirical_size_of_mark * extension->colums_to_read.size();
min_marks_to_read = extension->count_participating_replicas * amount_of_read_bytes_per_one_request / sum_average_marks_size;
}
else
{
min_marks_to_read = min_marks_to_read_;
}
ordered_names = getHeader().getNames();
min_marks_to_read = min_marks_for_concurrent_read_;
}
/// Requests read task from MergeTreeReadPool and signals whether it got one
bool MergeTreeThreadSelectAlgorithm::getNewTaskImpl()
{
task = pool->getTask(min_marks_to_read, thread, ordered_names);
task = pool->getTask(thread);
return static_cast<bool>(task);
}
@ -113,19 +56,19 @@ void MergeTreeThreadSelectAlgorithm::finalizeNewTask()
owned_uncompressed_cache = storage.getContext()->getUncompressedCache();
owned_mark_cache = storage.getContext()->getMarkCache();
}
else if (part_name != last_readed_part_name)
else if (part_name != last_read_part_name)
{
value_size_map = reader->getAvgValueSizeHints();
}
const bool init_new_readers = !reader || part_name != last_readed_part_name;
const bool init_new_readers = !reader || part_name != last_read_part_name;
if (init_new_readers)
{
initializeMergeTreeReadersForPart(task->data_part, task->task_columns, metadata_snapshot,
task->mark_ranges, value_size_map, profile_callback);
}
last_readed_part_name = part_name;
last_read_part_name = part_name;
}

View File

@ -5,8 +5,8 @@
namespace DB
{
class MergeTreeReadPool;
class IMergeTreeReadPool;
using IMergeTreeReadPoolPtr = std::shared_ptr<IMergeTreeReadPool>;
/** Used in conjunction with MergeTreeReadPool, asking it for more work to do and performing whatever reads it is asked
* to perform.
@ -16,9 +16,9 @@ class MergeTreeThreadSelectAlgorithm final : public IMergeTreeSelectAlgorithm
public:
MergeTreeThreadSelectAlgorithm(
size_t thread_,
const std::shared_ptr<MergeTreeReadPool> & pool_,
size_t min_marks_to_read_,
UInt64 max_block_size_,
IMergeTreeReadPoolPtr pool_,
size_t min_marks_for_concurrent_read,
size_t max_block_size_,
size_t preferred_block_size_bytes_,
size_t preferred_max_column_in_block_size_bytes_,
const MergeTreeData & storage_,
@ -27,8 +27,7 @@ public:
const PrewhereInfoPtr & prewhere_info_,
ExpressionActionsSettings actions_settings,
const MergeTreeReaderSettings & reader_settings_,
const Names & virt_column_names_,
std::optional<ParallelReadingExtension> extension_);
const Names & virt_column_names_);
String getName() const override { return "MergeTreeThread"; }
@ -42,18 +41,14 @@ protected:
void finish() override;
bool canUseConsistentHashingForParallelReading() override { return true; }
private:
/// "thread" index (there are N threads and each thread is assigned index in interval [0..N-1])
size_t thread;
std::shared_ptr<MergeTreeReadPool> pool;
IMergeTreeReadPoolPtr pool;
/// Last part read in this thread
std::string last_readed_part_name;
/// Names from header. Used in order to order columns in read blocks.
Names ordered_names;
std::string last_read_part_name;
};
}

View File

@ -1,23 +1,95 @@
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
#include <algorithm>
#include <mutex>
#include <numeric>
#include <vector>
#include <map>
#include <set>
#include <consistent_hashing.h>
#include <Common/logger_useful.h>
#include <base/scope_guard.h>
#include <Common/Stopwatch.h>
#include <IO/WriteBufferFromString.h>
#include <Common/SipHash.h>
#include <Common/thread_local_rng.h>
#include <base/types.h>
#include "IO/WriteBufferFromString.h"
#include "Storages/MergeTree/RangesInDataPart.h"
#include "Storages/MergeTree/RequestResponse.h"
#include <Storages/MergeTree/MarkRange.h>
#include <Storages/MergeTree/IntersectionsIndexes.h>
#include <fmt/format.h>
namespace DB
{
class ParallelReplicasReadingCoordinator::Impl
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
class ParallelReplicasReadingCoordinator::ImplInterface
{
public:
using PartitionReadRequestPtr = std::unique_ptr<PartitionReadRequest>;
struct Stat
{
size_t number_of_requests{0};
size_t sum_marks{0};
};
using Stats = std::vector<Stat>;
static String toString(Stats stats)
{
String result = "Statistics: ";
for (size_t i = 0; i < stats.size(); ++i)
result += fmt::format("-- replica {}, requests: {} marks: {} ", i, stats[i].number_of_requests, stats[i].sum_marks);
return result;
}
Stats stats;
std::mutex mutex;
size_t replicas_count;
explicit ImplInterface(size_t replicas_count_)
: stats{replicas_count_}
, replicas_count(replicas_count_)
{}
virtual ~ImplInterface() = default;
virtual ParallelReadResponse handleRequest(ParallelReadRequest request) = 0;
virtual void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) = 0;
};
struct Part
{
mutable RangesInDataPartDescription description;
// FIXME: This is needed to put this struct in set
// and modify through iterator
mutable std::set<size_t> replicas;
bool operator<(const Part & rhs) const { return description.info < rhs.description.info; }
};
using Parts = std::set<Part>;
using PartRefs = std::deque<Parts::iterator>;
class DefaultCoordinator : public ParallelReplicasReadingCoordinator::ImplInterface
{
public:
using ParallelReadRequestPtr = std::unique_ptr<ParallelReadRequest>;
using PartToMarkRanges = std::map<PartToRead::PartAndProjectionNames, HalfIntervals>;
explicit DefaultCoordinator(size_t replicas_count_)
: ParallelReplicasReadingCoordinator::ImplInterface(replicas_count_)
, announcements(replicas_count_)
, reading_state(replicas_count_)
{
}
~DefaultCoordinator() override;
struct PartitionReading
{
PartSegments part_ranges;
@ -27,115 +99,423 @@ public:
using PartitionToBlockRanges = std::map<String, PartitionReading>;
PartitionToBlockRanges partitions;
std::mutex mutex;
size_t sent_initial_requests{0};
std::vector<InitialAllRangesAnnouncement> announcements;
PartitionReadResponse handleRequest(PartitionReadRequest request);
Parts all_parts_to_read;
/// Contains only parts which we haven't started to read from
PartRefs delayed_parts;
/// Per-replica preferred parts split by consistent hash
/// Once all task will be done by some replica, it can steal tasks
std::vector<PartRefs> reading_state;
Poco::Logger * log = &Poco::Logger::get("DefaultCoordinator");
std::atomic<bool> state_initialized{false};
ParallelReadResponse handleRequest(ParallelReadRequest request) override;
void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) override;
void updateReadingState(const InitialAllRangesAnnouncement & announcement);
void finalizeReadingState();
size_t computeConsistentHash(const MergeTreePartInfo & info) const
{
auto hash = SipHash();
hash.update(info.getPartNameV1());
return ConsistentHashing(hash.get64(), replicas_count);
}
void selectPartsAndRanges(const PartRefs & container, size_t replica_num, size_t min_number_of_marks, size_t & current_mark_size, ParallelReadResponse & response) const;
};
DefaultCoordinator::~DefaultCoordinator()
{
LOG_INFO(log, "Coordination done: {}", toString(stats));
}
void DefaultCoordinator::updateReadingState(const InitialAllRangesAnnouncement & announcement)
{
PartRefs parts_diff;
/// To get rid of duplicates
for (const auto & part: announcement.description)
{
auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(),
[&part] (const Part & other) { return other.description.info.getPartNameV1() == part.info.getPartNameV1(); });
/// We have the same part - add the info about presence on current replica to it
if (the_same_it != all_parts_to_read.end())
{
the_same_it->replicas.insert(announcement.replica_num);
continue;
}
auto covering_or_the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(),
[&part] (const Part & other) { return !other.description.info.isDisjoint(part.info); });
/// It is covering part or we have covering - skip it
if (covering_or_the_same_it != all_parts_to_read.end())
continue;
auto new_part = Part{
.description = part,
.replicas = {announcement.replica_num}
};
auto [insert_it, _] = all_parts_to_read.insert(new_part);
parts_diff.push_back(insert_it);
}
/// Split all parts by consistent hash
while (!parts_diff.empty())
{
auto current_part_it = parts_diff.front();
parts_diff.pop_front();
auto consistent_hash = computeConsistentHash(current_part_it->description.info);
/// Check whether the new part can easy go to replica queue
if (current_part_it->replicas.contains(consistent_hash))
{
reading_state[consistent_hash].emplace_back(current_part_it);
continue;
}
/// Add to delayed parts
delayed_parts.emplace_back(current_part_it);
}
}
void DefaultCoordinator::finalizeReadingState()
{
/// Clear all the delayed queue
while (!delayed_parts.empty())
{
auto current_part_it = delayed_parts.front();
auto consistent_hash = computeConsistentHash(current_part_it->description.info);
if (current_part_it->replicas.contains(consistent_hash))
{
reading_state[consistent_hash].emplace_back(current_part_it);
delayed_parts.pop_front();
continue;
}
/// In this situation just assign to a random replica which has this part
auto replica = *(std::next(current_part_it->replicas.begin(), thread_local_rng() % current_part_it->replicas.size()));
reading_state[replica].emplace_back(current_part_it);
delayed_parts.pop_front();
}
String description;
for (const auto & part : all_parts_to_read)
{
description += part.description.describe();
description += fmt::format("Replicas: ({}) --- ", fmt::join(part.replicas, ","));
}
LOG_INFO(log, "Reading state is fully initialized: {}", description);
}
void DefaultCoordinator::handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement)
{
std::lock_guard lock(mutex);
updateReadingState(announcement);
stats[announcement.replica_num].number_of_requests +=1;
++sent_initial_requests;
LOG_INFO(log, "{} {}", sent_initial_requests, replicas_count);
if (sent_initial_requests == replicas_count)
finalizeReadingState();
}
void DefaultCoordinator::selectPartsAndRanges(const PartRefs & container, size_t replica_num, size_t min_number_of_marks, size_t & current_mark_size, ParallelReadResponse & response) const
{
for (const auto & part : container)
{
if (current_mark_size >= min_number_of_marks)
{
LOG_TEST(log, "Current mark size {} is bigger than min_number_marks {}", current_mark_size, min_number_of_marks);
break;
}
if (part->description.ranges.empty())
{
LOG_TEST(log, "Part {} is already empty in reading state", part->description.info.getPartNameV1());
continue;
}
if (std::find(part->replicas.begin(), part->replicas.end(), replica_num) == part->replicas.end())
{
LOG_TEST(log, "Not found part {} on replica {}", part->description.info.getPartNameV1(), replica_num);
continue;
}
response.description.push_back({
.info = part->description.info,
.ranges = {},
});
while (!part->description.ranges.empty() && current_mark_size < min_number_of_marks)
{
auto & range = part->description.ranges.front();
if (range.getNumberOfMarks() > min_number_of_marks)
{
auto new_range = range;
range.begin += min_number_of_marks;
new_range.end = new_range.begin + min_number_of_marks;
response.description.back().ranges.emplace_back(new_range);
current_mark_size += new_range.getNumberOfMarks();
continue;
}
current_mark_size += part->description.ranges.front().getNumberOfMarks();
response.description.back().ranges.emplace_back(part->description.ranges.front());
part->description.ranges.pop_front();
}
}
}
ParallelReadResponse DefaultCoordinator::handleRequest(ParallelReadRequest request)
{
std::lock_guard lock(mutex);
LOG_TRACE(log, "Handling request from replica {}, minimal marks size is {}", request.replica_num, request.min_number_of_marks);
size_t current_mark_size = 0;
ParallelReadResponse response;
/// 1. Try to select from preferred set of parts for current replica
selectPartsAndRanges(reading_state[request.replica_num], request.replica_num, request.min_number_of_marks, current_mark_size, response);
/// 2. Try to use parts from delayed queue
while (!delayed_parts.empty() && current_mark_size < request.min_number_of_marks)
{
auto part = delayed_parts.front();
delayed_parts.pop_front();
reading_state[request.replica_num].emplace_back(part);
selectPartsAndRanges(reading_state[request.replica_num], request.replica_num, request.min_number_of_marks, current_mark_size, response);
}
/// 3. Try to steal tasks;
if (current_mark_size < request.min_number_of_marks)
{
for (size_t i = 0; i < replicas_count; ++i)
{
if (i != request.replica_num)
selectPartsAndRanges(reading_state[i], request.replica_num, request.min_number_of_marks, current_mark_size, response);
if (current_mark_size >= request.min_number_of_marks)
break;
}
}
stats[request.replica_num].number_of_requests += 1;
stats[request.replica_num].sum_marks += current_mark_size;
if (response.description.empty())
response.finish = true;
LOG_TRACE(log, "Going to respond to replica {} with {}", request.replica_num, response.describe());
return response;
}
template <CoordinationMode mode>
class InOrderCoordinator : public ParallelReplicasReadingCoordinator::ImplInterface
{
public:
explicit InOrderCoordinator([[ maybe_unused ]] size_t replicas_count_)
: ParallelReplicasReadingCoordinator::ImplInterface(replicas_count_)
{}
~InOrderCoordinator() override
{
LOG_INFO(log, "Coordination done: {}", toString(stats));
}
ParallelReadResponse handleRequest([[ maybe_unused ]] ParallelReadRequest request) override;
void handleInitialAllRangesAnnouncement([[ maybe_unused ]] InitialAllRangesAnnouncement announcement) override;
Parts all_parts_to_read;
Poco::Logger * log = &Poco::Logger::get(fmt::format("{}{}", magic_enum::enum_name(mode), "Coordinator"));
};
PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(PartitionReadRequest request)
template <CoordinationMode mode>
void InOrderCoordinator<mode>::handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement)
{
auto * log = &Poco::Logger::get("ParallelReplicasReadingCoordinator");
Stopwatch watch;
String request_description = request.toString();
std::lock_guard lock(mutex);
LOG_TRACE(log, "Received an announecement {}", announcement.describe());
auto partition_it = partitions.find(request.partition_id);
PartToRead::PartAndProjectionNames part_and_projection
/// To get rid of duplicates
for (const auto & part: announcement.description)
{
.part = request.part_name,
.projection = request.projection_name
};
auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(),
[&part] (const Part & other) { return other.description.info == part.info; });
/// We are the first who wants to process parts in partition
if (partition_it == partitions.end())
{
PartitionReading partition_reading;
/// We have the same part - add the info about presence on current replica to it
if (the_same_it != all_parts_to_read.end())
{
the_same_it->replicas.insert(announcement.replica_num);
continue;
}
PartToRead part_to_read;
part_to_read.range = request.block_range;
part_to_read.name = part_and_projection;
auto covering_or_the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(),
[&part] (const Part & other) { return other.description.info.contains(part.info) || part.info.contains(other.description.info); });
partition_reading.part_ranges.addPart(std::move(part_to_read));
/// It is covering part or we have covering - skip it
if (covering_or_the_same_it != all_parts_to_read.end())
continue;
/// As this query is first in partition, we will accept all ranges from it.
/// We need just to update our state.
auto request_ranges = HalfIntervals::initializeFromMarkRanges(request.mark_ranges);
auto mark_ranges_index = HalfIntervals::initializeWithEntireSpace();
mark_ranges_index.intersect(request_ranges.negate());
auto new_part = Part{
.description = part,
.replicas = {announcement.replica_num}
};
partition_reading.mark_ranges_in_part.insert({part_and_projection, std::move(mark_ranges_index)});
partitions.insert({request.partition_id, std::move(partition_reading)});
LOG_TRACE(log, "Request is first in partition, accepted in {} ns: {}", watch.elapsed(), request_description);
return {.denied = false, .mark_ranges = std::move(request.mark_ranges)};
auto insert_it = all_parts_to_read.insert(new_part);
auto & ranges = insert_it.first->description.ranges;
std::sort(ranges.begin(), ranges.end());
}
auto & partition_reading = partition_it->second;
PartToRead part_to_read;
part_to_read.range = request.block_range;
part_to_read.name = part_and_projection;
auto part_intersection_res = partition_reading.part_ranges.getIntersectionResult(part_to_read);
switch (part_intersection_res)
{
case PartSegments::IntersectionResult::REJECT:
{
LOG_TRACE(log, "Request rejected in {} ns: {}", watch.elapsed(), request_description);
return {.denied = true, .mark_ranges = {}};
}
case PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION:
{
auto marks_it = partition_reading.mark_ranges_in_part.find(part_and_projection);
auto & intervals_to_do = marks_it->second;
auto result = HalfIntervals::initializeFromMarkRanges(request.mark_ranges);
result.intersect(intervals_to_do);
/// Update intervals_to_do
intervals_to_do.intersect(HalfIntervals::initializeFromMarkRanges(std::move(request.mark_ranges)).negate());
auto result_ranges = result.convertToMarkRangesFinal();
const bool denied = result_ranges.empty();
if (denied)
LOG_TRACE(log, "Request rejected due to intersection in {} ns: {}", watch.elapsed(), request_description);
else
LOG_TRACE(log, "Request accepted partially in {} ns: {}", watch.elapsed(), request_description);
return {.denied = denied, .mark_ranges = std::move(result_ranges)};
}
case PartSegments::IntersectionResult::NO_INTERSECTION:
{
partition_reading.part_ranges.addPart(std::move(part_to_read));
auto mark_ranges_index = HalfIntervals::initializeWithEntireSpace().intersect(
HalfIntervals::initializeFromMarkRanges(request.mark_ranges).negate()
);
partition_reading.mark_ranges_in_part.insert({part_and_projection, std::move(mark_ranges_index)});
LOG_TRACE(log, "Request accepted in {} ns: {}", watch.elapsed(), request_description);
return {.denied = false, .mark_ranges = std::move(request.mark_ranges)};
}
}
UNREACHABLE();
}
PartitionReadResponse ParallelReplicasReadingCoordinator::handleRequest(PartitionReadRequest request)
template <CoordinationMode mode>
ParallelReadResponse InOrderCoordinator<mode>::handleRequest(ParallelReadRequest request)
{
std::lock_guard lock(mutex);
if (request.mode != mode)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Replica {} decided to read in {} mode, not in {}. This is a bug",
request.replica_num, magic_enum::enum_name(request.mode), magic_enum::enum_name(mode));
LOG_TRACE(log, "Got request from replica {}, data {}", request.replica_num, request.describe());
ParallelReadResponse response;
response.description = request.description;
size_t overall_number_of_marks = 0;
for (auto & part : response.description)
{
auto global_part_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(),
[&part] (const Part & other) { return other.description.info == part.info; });
if (global_part_it == all_parts_to_read.end())
continue;
if (!global_part_it->replicas.contains(request.replica_num))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} doesn't exist on replica {} according to the global state", part.info.getPartNameV1(), request.replica_num);
size_t current_mark_size = 0;
/// Now we can recommend to read more intervals
if constexpr (mode == CoordinationMode::ReverseOrder)
{
while (!global_part_it->description.ranges.empty() && current_mark_size < request.min_number_of_marks)
{
auto range = global_part_it->description.ranges.back();
if (range.getNumberOfMarks() > request.min_number_of_marks)
{
auto new_range = range;
range.end -= request.min_number_of_marks;
new_range.begin = new_range.end - request.min_number_of_marks;
global_part_it->description.ranges.back() = range;
part.ranges.emplace_front(new_range);
current_mark_size += new_range.getNumberOfMarks();
continue;
}
current_mark_size += global_part_it->description.ranges.back().getNumberOfMarks();
part.ranges.emplace_front(global_part_it->description.ranges.back());
global_part_it->description.ranges.pop_back();
}
}
else if constexpr (mode == CoordinationMode::WithOrder)
{
while (!global_part_it->description.ranges.empty() && current_mark_size < request.min_number_of_marks)
{
auto range = global_part_it->description.ranges.front();
if (range.getNumberOfMarks() > request.min_number_of_marks)
{
auto new_range = range;
range.begin += request.min_number_of_marks;
new_range.end = new_range.begin + request.min_number_of_marks;
global_part_it->description.ranges.front() = range;
part.ranges.emplace_back(new_range);
current_mark_size += new_range.getNumberOfMarks();
continue;
}
current_mark_size += global_part_it->description.ranges.front().getNumberOfMarks();
part.ranges.emplace_back(global_part_it->description.ranges.front());
global_part_it->description.ranges.pop_front();
}
}
overall_number_of_marks += current_mark_size;
}
if (!overall_number_of_marks)
response.finish = true;
stats[request.replica_num].number_of_requests += 1;
stats[request.replica_num].sum_marks += overall_number_of_marks;
LOG_TRACE(log, "Going to respond to replica {} with {}", request.replica_num, response.describe());
return response;
}
void ParallelReplicasReadingCoordinator::handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement)
{
if (!pimpl)
initialize();
return pimpl->handleInitialAllRangesAnnouncement(announcement);
}
ParallelReadResponse ParallelReplicasReadingCoordinator::handleRequest(ParallelReadRequest request)
{
if (!pimpl)
initialize();
return pimpl->handleRequest(std::move(request));
}
ParallelReplicasReadingCoordinator::ParallelReplicasReadingCoordinator()
void ParallelReplicasReadingCoordinator::setMode(CoordinationMode mode_)
{
pimpl = std::make_unique<ParallelReplicasReadingCoordinator::Impl>();
mode = mode_;
}
void ParallelReplicasReadingCoordinator::initialize()
{
switch (mode)
{
case CoordinationMode::Default:
pimpl = std::make_unique<DefaultCoordinator>(replicas_count);
return;
case CoordinationMode::WithOrder:
pimpl = std::make_unique<InOrderCoordinator<CoordinationMode::WithOrder>>(replicas_count);
return;
case CoordinationMode::ReverseOrder:
pimpl = std::make_unique<InOrderCoordinator<CoordinationMode::ReverseOrder>>(replicas_count);
return;
}
}
ParallelReplicasReadingCoordinator::ParallelReplicasReadingCoordinator(size_t replicas_count_) : replicas_count(replicas_count_) {}
ParallelReplicasReadingCoordinator::~ParallelReplicasReadingCoordinator() = default;
}

View File

@ -7,15 +7,28 @@
namespace DB
{
/// The main class to spread mark ranges across replicas dynamically
/// The reason why it uses pimpl - this header file is included in
/// multiple other files like Context or RemoteQueryExecutor
class ParallelReplicasReadingCoordinator
{
public:
ParallelReplicasReadingCoordinator();
class ImplInterface;
explicit ParallelReplicasReadingCoordinator(size_t replicas_count_);
~ParallelReplicasReadingCoordinator();
PartitionReadResponse handleRequest(PartitionReadRequest request);
void setMode(CoordinationMode mode);
void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement);
ParallelReadResponse handleRequest(ParallelReadRequest request);
private:
class Impl;
std::unique_ptr<Impl> pimpl;
void initialize();
CoordinationMode mode{CoordinationMode::Default};
size_t replicas_count{0};
std::atomic<bool> initialized{false};
std::unique_ptr<ImplInterface> pimpl;
};
using ParallelReplicasReadingCoordinatorPtr = std::shared_ptr<ParallelReplicasReadingCoordinator>;

View File

@ -0,0 +1,113 @@
#include <Storages/MergeTree/RangesInDataPart.h>
#include <Storages/MergeTree/IMergeTreeDataPart.h>
#include "IO/VarInt.h"
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
namespace DB
{
void RangesInDataPartDescription::serialize(WriteBuffer & out) const
{
info.serialize(out);
ranges.serialize(out);
}
String RangesInDataPartDescription::describe() const
{
String result;
result += fmt::format("Part: {}, ", info.getPartNameV1());
result += fmt::format("Ranges: [{}], ", fmt::join(ranges, ","));
return result;
}
void RangesInDataPartDescription::deserialize(ReadBuffer & in)
{
info.deserialize(in);
ranges.deserialize(in);
}
void RangesInDataPartsDescription::serialize(WriteBuffer & out) const
{
writeVarUInt(this->size(), out);
for (const auto & desc : *this)
desc.serialize(out);
}
String RangesInDataPartsDescription::describe() const
{
String result;
for (const auto & desc : *this)
result += desc.describe() + ",";
return result;
}
void RangesInDataPartsDescription::deserialize(ReadBuffer & in)
{
size_t new_size = 0;
readVarUInt(new_size, in);
this->resize(new_size);
for (auto & desc : *this)
desc.deserialize(in);
}
void RangesInDataPartsDescription::merge(RangesInDataPartsDescription & other)
{
for (const auto & desc : other)
this->emplace_back(desc);
}
RangesInDataPartDescription RangesInDataPart::getDescription() const
{
return RangesInDataPartDescription{
.info = data_part->info,
.ranges = ranges,
};
}
size_t RangesInDataPart::getMarksCount() const
{
size_t total = 0;
for (const auto & range : ranges)
total += range.end - range.begin;
return total;
}
size_t RangesInDataPart::getRowsCount() const
{
return data_part->index_granularity.getRowsCountInRanges(ranges);
}
RangesInDataPartsDescription RangesInDataParts::getDescriptions() const
{
RangesInDataPartsDescription result;
for (const auto & part : *this)
result.emplace_back(part.getDescription());
return result;
}
size_t RangesInDataParts::getMarksCountAllParts() const
{
size_t result = 0;
for (const auto & part : *this)
result += part.getMarksCount();
return result;
}
size_t RangesInDataParts::getRowsCountAllParts() const
{
size_t result = 0;
for (const auto & part: *this)
result += part.getRowsCount();
return result;
}
}

View File

@ -1,42 +1,73 @@
#pragma once
#include <Storages/MergeTree/MergeTreeData.h>
#include <vector>
#include <IO/WriteBuffer.h>
#include <IO/ReadBuffer.h>
#include <Storages/MergeTree/MarkRange.h>
#include "Storages/MergeTree/MergeTreePartInfo.h"
namespace DB
{
class IMergeTreeDataPart;
using DataPartPtr = std::shared_ptr<const IMergeTreeDataPart>;
/// The only purpose of this struct is that serialize and deserialize methods
/// they look natural here because we can fully serialize and then deserialize original DataPart class.
struct RangesInDataPartDescription
{
MergeTreePartInfo info;
MarkRanges ranges;
void serialize(WriteBuffer & out) const;
String describe() const;
void deserialize(ReadBuffer & in);
};
struct RangesInDataPartsDescription: public std::deque<RangesInDataPartDescription>
{
using std::deque<RangesInDataPartDescription>::deque;
void serialize(WriteBuffer & out) const;
String describe() const;
void deserialize(ReadBuffer & in);
void merge(RangesInDataPartsDescription & other);
};
struct RangesInDataPart
{
MergeTreeData::DataPartPtr data_part;
DataPartPtr data_part;
size_t part_index_in_query;
MarkRanges ranges;
RangesInDataPart() = default;
RangesInDataPart(const MergeTreeData::DataPartPtr & data_part_, const size_t part_index_in_query_,
const MarkRanges & ranges_ = MarkRanges{})
: data_part{data_part_}, part_index_in_query{part_index_in_query_}, ranges{ranges_}
{
}
RangesInDataPart(
const DataPartPtr & data_part_,
const size_t part_index_in_query_,
const MarkRanges & ranges_ = MarkRanges{})
: data_part{data_part_}
, part_index_in_query{part_index_in_query_}
, ranges{ranges_}
{}
size_t getMarksCount() const
{
size_t total = 0;
for (const auto & range : ranges)
total += range.end - range.begin;
RangesInDataPartDescription getDescription() const;
return total;
}
size_t getRowsCount() const
{
return data_part->index_granularity.getRowsCountInRanges(ranges);
}
size_t getMarksCount() const;
size_t getRowsCount() const;
};
using RangesInDataParts = std::vector<RangesInDataPart>;
struct RangesInDataParts: public std::vector<RangesInDataPart>
{
using std::vector<RangesInDataPart>::vector;
RangesInDataPartsDescription getDescriptions() const;
size_t getMarksCountAllParts() const;
size_t getRowsCountAllParts() const;
};
}

View File

@ -1,159 +1,129 @@
#include <chrono>
#include <Storages/MergeTree/RequestResponse.h>
#include <Core/ProtocolDefines.h>
#include <Common/SipHash.h>
#include "IO/VarInt.h"
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/Operators.h>
#include <consistent_hashing.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_PROTOCOL;
extern const int BAD_ARGUMENTS;
}
static void readMarkRangesBinary(MarkRanges & ranges, ReadBuffer & buf)
{
size_t size = 0;
readVarUInt(size, buf);
if (size > DEFAULT_MAX_STRING_SIZE)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Too large ranges size: {}.", size);
ranges.resize(size);
for (size_t i = 0; i < size; ++i)
{
readBinary(ranges[i].begin, buf);
readBinary(ranges[i].end, buf);
}
}
static void writeMarkRangesBinary(const MarkRanges & ranges, WriteBuffer & buf)
{
writeVarUInt(ranges.size(), buf);
for (const auto & [begin, end] : ranges)
{
writeBinary(begin, buf);
writeBinary(end, buf);
}
}
void PartitionReadRequest::serialize(WriteBuffer & out) const
void ParallelReadRequest::serialize(WriteBuffer & out) const
{
UInt64 version = DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION;
/// Must be the first
writeVarUInt(DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION, out);
writeIntBinary(version, out);
writeStringBinary(partition_id, out);
writeStringBinary(part_name, out);
writeStringBinary(projection_name, out);
writeVarInt(block_range.begin, out);
writeVarInt(block_range.end, out);
writeMarkRangesBinary(mark_ranges, out);
writeIntBinary(mode, out);
writeIntBinary(replica_num, out);
writeIntBinary(min_number_of_marks, out);
description.serialize(out);
}
String PartitionReadRequest::toString() const
String ParallelReadRequest::describe() const
{
WriteBufferFromOwnString out;
out << "partition: " << partition_id << ", part: " << part_name;
if (!projection_name.empty())
out << ", projection: " << projection_name;
out << ", block range: [" << block_range.begin << ", " << block_range.end << "]";
out << ", mark ranges: ";
bool is_first = true;
for (const auto & [begin, end] : mark_ranges)
{
if (!is_first)
out << ", ";
out << "[" << begin << ", " << end << ")";
is_first = false;
}
return out.str();
String result;
result += fmt::format("replica_num: {} \n", replica_num);
result += fmt::format("min_num_of_marks: {} \n", min_number_of_marks);
result += description.describe();
return result;
}
void PartitionReadRequest::deserialize(ReadBuffer & in)
void ParallelReadRequest::deserialize(ReadBuffer & in)
{
UInt64 version;
readVarUInt(version, in);
readIntBinary(version, in);
if (version != DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION)
throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading \
from replicas differ. Got: {}, supported version: {}",
throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading "\
"from replicas differ. Got: {}, supported version: {}",
version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION);
readStringBinary(partition_id, in);
readStringBinary(part_name, in);
readStringBinary(projection_name, in);
readVarInt(block_range.begin, in);
readVarInt(block_range.end, in);
readMarkRangesBinary(mark_ranges, in);
readIntBinary(mode, in);
readIntBinary(replica_num, in);
readIntBinary(min_number_of_marks, in);
description.deserialize(in);
}
UInt64 PartitionReadRequest::getConsistentHash(size_t buckets) const
void ParallelReadRequest::merge(ParallelReadRequest & other)
{
SipHash hash;
hash.update(partition_id.size());
hash.update(partition_id);
hash.update(part_name.size());
hash.update(part_name);
hash.update(projection_name.size());
hash.update(projection_name);
hash.update(block_range.begin);
hash.update(block_range.end);
hash.update(mark_ranges.size());
for (const auto & range : mark_ranges)
{
hash.update(range.begin);
hash.update(range.end);
}
return ConsistentHashing(hash.get64(), buckets);
assert(mode == other.mode);
assert(replica_num == other.replica_num);
assert(min_number_of_marks == other.min_number_of_marks);
description.merge(other.description);
}
void PartitionReadResponse::serialize(WriteBuffer & out) const
void ParallelReadResponse::serialize(WriteBuffer & out) const
{
UInt64 version = DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION;
/// Must be the first
writeVarUInt(DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION, out);
writeIntBinary(version, out);
writeBinary(denied, out);
writeMarkRangesBinary(mark_ranges, out);
writeBoolText(finish, out);
description.serialize(out);
}
String ParallelReadResponse::describe() const
{
String result;
result += fmt::format("finish: {} \n", finish);
result += description.describe();
return result;
}
void PartitionReadResponse::deserialize(ReadBuffer & in)
void ParallelReadResponse::deserialize(ReadBuffer & in)
{
UInt64 version;
readVarUInt(version, in);
readIntBinary(version, in);
if (version != DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION)
throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading \
from replicas differ. Got: {}, supported version: {}",
throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading " \
"from replicas differ. Got: {}, supported version: {}",
version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION);
UInt64 value;
readVarUInt(value, in);
denied = static_cast<bool>(value);
readMarkRangesBinary(mark_ranges, in);
readBoolText(finish, in);
description.deserialize(in);
}
void InitialAllRangesAnnouncement::serialize(WriteBuffer & out) const
{
UInt64 version = DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION;
/// Must be the first
writeIntBinary(version, out);
description.serialize(out);
writeIntBinary(replica_num, out);
}
String InitialAllRangesAnnouncement::describe()
{
String result;
result += description.describe();
result += fmt::format("----------\nReceived from {} replica\n", replica_num);
return result;
}
void InitialAllRangesAnnouncement::deserialize(ReadBuffer & in)
{
UInt64 version;
readIntBinary(version, in);
if (version != DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION)
throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading " \
"from replicas differ. Got: {}, supported version: {}",
version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION);
description.deserialize(in);
readIntBinary(replica_num, in);
}
}

View File

@ -1,5 +1,6 @@
#pragma once
#include <condition_variable>
#include <functional>
#include <optional>
@ -9,12 +10,21 @@
#include <IO/ReadBuffer.h>
#include <Storages/MergeTree/MarkRange.h>
#include <Storages/MergeTree/RangesInDataPart.h>
namespace DB
{
/// Represents a segment [left; right] of part's block numbers.
enum class CoordinationMode
{
Default,
/// For reading in order
WithOrder,
ReverseOrder
};
/// Represents a segment [left; right]
struct PartBlockRange
{
Int64 begin;
@ -26,34 +36,44 @@ struct PartBlockRange
}
};
struct PartitionReadRequest
struct ParallelReadRequest
{
String partition_id;
String part_name;
String projection_name;
PartBlockRange block_range;
MarkRanges mark_ranges;
CoordinationMode mode;
size_t replica_num;
size_t min_number_of_marks;
/// Extension for ordered mode
RangesInDataPartsDescription description;
void serialize(WriteBuffer & out) const;
String describe() const;
void deserialize(ReadBuffer & in);
UInt64 getConsistentHash(size_t buckets) const;
/// Describe it for debugging purposes.
String toString() const;
void merge(ParallelReadRequest & other);
};
struct PartitionReadResponse
struct ParallelReadResponse
{
bool denied{false};
MarkRanges mark_ranges{};
bool finish{false};
RangesInDataPartsDescription description;
void serialize(WriteBuffer & out) const;
String describe() const;
void deserialize(ReadBuffer & in);
};
using MergeTreeReadTaskCallback = std::function<std::optional<PartitionReadResponse>(PartitionReadRequest)>;
struct InitialAllRangesAnnouncement
{
RangesInDataPartsDescription description;
size_t replica_num;
void serialize(WriteBuffer & out) const;
String describe();
void deserialize(ReadBuffer & in);
};
using MergeTreeAllRangesCallback = std::function<void(InitialAllRangesAnnouncement)>;
using MergeTreeReadTaskCallback = std::function<std::optional<ParallelReadResponse>(ParallelReadRequest)>;
}

View File

@ -1,240 +0,0 @@
#include <gtest/gtest.h>
#include <utility>
#include <limits>
#include <set>
#include <Storages/MergeTree/IntersectionsIndexes.h>
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
using namespace DB;
TEST(HalfIntervals, Simple)
{
ASSERT_TRUE((
HalfIntervals{{{1, 2}, {3, 4}}}.negate() ==
HalfIntervals{{{0, 1}, {2, 3}, {4, 18446744073709551615UL}}}
));
{
auto left = HalfIntervals{{{0, 2}, {4, 6}}}.negate();
ASSERT_TRUE((
left ==
HalfIntervals{{{2, 4}, {6, 18446744073709551615UL}}}
));
}
{
auto left = HalfIntervals{{{0, 2}, {4, 6}}};
auto right = HalfIntervals{{{1, 5}}}.negate();
auto intersection = left.intersect(right);
ASSERT_TRUE((
intersection ==
HalfIntervals{{{0, 1}, {5, 6}}}
));
}
{
auto left = HalfIntervals{{{1, 2}, {2, 3}}};
auto right = HalfIntervals::initializeWithEntireSpace();
auto intersection = right.intersect(left.negate());
ASSERT_TRUE((
intersection ==
HalfIntervals{{{0, 1}, {3, 18446744073709551615UL}}}
));
}
{
auto left = HalfIntervals{{{1, 2}, {2, 3}, {3, 4}, {4, 5}}};
ASSERT_EQ(getIntersection(left, HalfIntervals{{{1, 4}}}).convertToMarkRangesFinal().size(), 3);
ASSERT_EQ(getIntersection(left, HalfIntervals{{{1, 5}}}).convertToMarkRangesFinal().size(), 4);
}
{
auto left = HalfIntervals{{{1, 3}, {3, 5}, {5, 7}}};
ASSERT_EQ(getIntersection(left, HalfIntervals{{{3, 5}}}).convertToMarkRangesFinal().size(), 1);
ASSERT_EQ(getIntersection(left, HalfIntervals{{{3, 7}}}).convertToMarkRangesFinal().size(), 2);
ASSERT_EQ(getIntersection(left, HalfIntervals{{{4, 6}}}).convertToMarkRangesFinal().size(), 2);
ASSERT_EQ(getIntersection(left, HalfIntervals{{{1, 7}}}).convertToMarkRangesFinal().size(), 3);
}
{
auto left = HalfIntervals{{{1, 3}}};
ASSERT_EQ(getIntersection(left, HalfIntervals{{{3, 4}}}).convertToMarkRangesFinal().size(), 0);
}
{
auto left = HalfIntervals{{{1, 2}, {3, 4}, {5, 6}}};
ASSERT_EQ(getIntersection(left, HalfIntervals{{{2, 3}}}).convertToMarkRangesFinal().size(), 0);
ASSERT_EQ(getIntersection(left, HalfIntervals{{{4, 5}}}).convertToMarkRangesFinal().size(), 0);
ASSERT_EQ(getIntersection(left, HalfIntervals{{{1, 6}}}).convertToMarkRangesFinal().size(), 3);
}
}
TEST(HalfIntervals, TwoRequests)
{
auto left = HalfIntervals{{{1, 2}, {2, 3}}};
auto right = HalfIntervals{{{2, 3}, {3, 4}}};
auto intersection = left.intersect(right);
ASSERT_TRUE((
intersection ==
HalfIntervals{{{2, 3}}}
));
/// With negation
left = HalfIntervals{{{1, 2}, {2, 3}}}.negate();
right = HalfIntervals{{{2, 3}, {3, 4}}};
intersection = left.intersect(right);
ASSERT_TRUE((
intersection ==
HalfIntervals{{{3, 4}}}
));
}
TEST(HalfIntervals, SelfIntersection)
{
auto left = HalfIntervals{{{1, 2}, {2, 3}, {4, 5}}};
auto right = left;
auto intersection = left.intersect(right);
ASSERT_TRUE((
intersection == right
));
left = HalfIntervals{{{1, 2}, {2, 3}, {4, 5}}};
right = left;
right.negate();
intersection = left.intersect(right);
ASSERT_TRUE((
intersection == HalfIntervals{}
));
}
TEST(Coordinator, Simple)
{
PartitionReadRequest request;
request.partition_id = "a";
request.part_name = "b";
request.projection_name = "c";
request.block_range = PartBlockRange{1, 2};
request.mark_ranges = MarkRanges{{1, 2}, {3, 4}};
ParallelReplicasReadingCoordinator coordinator;
auto response = coordinator.handleRequest(request);
ASSERT_FALSE(response.denied) << "Process request at first has to be accepted";
ASSERT_EQ(response.mark_ranges.size(), request.mark_ranges.size());
for (int i = 0; i < response.mark_ranges.size(); ++i)
EXPECT_EQ(response.mark_ranges[i], request.mark_ranges[i]);
response = coordinator.handleRequest(request);
ASSERT_TRUE(response.denied) << "Process the same request second time";
}
TEST(Coordinator, TwoRequests)
{
PartitionReadRequest first;
first.partition_id = "a";
first.part_name = "b";
first.projection_name = "c";
first.block_range = PartBlockRange{0, 0};
first.mark_ranges = MarkRanges{{1, 2}, {2, 3}};
auto second = first;
second.mark_ranges = MarkRanges{{2, 3}, {3, 4}};
ParallelReplicasReadingCoordinator coordinator;
auto response = coordinator.handleRequest(first);
ASSERT_FALSE(response.denied) << "First request must me accepted";
ASSERT_EQ(response.mark_ranges.size(), first.mark_ranges.size());
for (int i = 0; i < response.mark_ranges.size(); ++i)
EXPECT_EQ(response.mark_ranges[i], first.mark_ranges[i]);
response = coordinator.handleRequest(second);
ASSERT_FALSE(response.denied);
ASSERT_EQ(response.mark_ranges.size(), 1);
ASSERT_EQ(response.mark_ranges.front(), (MarkRange{3, 4}));
}
TEST(Coordinator, PartIntersections)
{
{
PartSegments boundaries;
boundaries.addPart(PartToRead{{1, 1}, {"TestPart", "TestProjection"}});
boundaries.addPart(PartToRead{{2, 2}, {"TestPart", "TestProjection"}});
boundaries.addPart(PartToRead{{3, 3}, {"TestPart", "TestProjection"}});
boundaries.addPart(PartToRead{{4, 4}, {"TestPart", "TestProjection"}});
ASSERT_EQ(boundaries.getIntersectionResult({{1, 4}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT);
ASSERT_EQ(boundaries.getIntersectionResult({{0, 5}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT);
ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION);
ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"ClickHouse", "AnotherProjection"}}), PartSegments::IntersectionResult::REJECT);
ASSERT_EQ(boundaries.getIntersectionResult({{1, 2}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT);
boundaries.addPart(PartToRead{{5, 5}, {"TestPart", "TestProjection"}});
boundaries.addPart(PartToRead{{0, 0}, {"TestPart", "TestProjection"}});
ASSERT_EQ(boundaries.getIntersectionResult({{0, 5}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT);
ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION);
ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"ClickHouse", "AnotherProjection"}}), PartSegments::IntersectionResult::REJECT);
ASSERT_EQ(boundaries.getIntersectionResult({{1, 2}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT);
ASSERT_EQ(boundaries.getIntersectionResult({{0, 3}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT);
}
{
PartSegments boundaries;
boundaries.addPart(PartToRead{{1, 3}, {"TestPart", "TestProjection"}});
boundaries.addPart(PartToRead{{4, 5}, {"TestPart", "TestProjection"}});
ASSERT_EQ(boundaries.getIntersectionResult({{2, 4}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT);
ASSERT_EQ(boundaries.getIntersectionResult({{0, 6}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT);
}
{
PartSegments boundaries;
boundaries.addPart(PartToRead{{1, 3}, {"TestPart", "TestProjection"}});
boundaries.addPart(PartToRead{{4, 6}, {"TestPart", "TestProjection"}});
boundaries.addPart(PartToRead{{7, 9}, {"TestPart", "TestProjection"}});
ASSERT_EQ(boundaries.getIntersectionResult({{2, 8}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT);
ASSERT_EQ(boundaries.getIntersectionResult({{4, 6}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION);
ASSERT_EQ(boundaries.getIntersectionResult({{3, 7}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT);
ASSERT_EQ(boundaries.getIntersectionResult({{5, 7}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT);
}
{
PartSegments boundaries;
ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::NO_INTERSECTION);
ASSERT_EQ(boundaries.getIntersectionResult({{1, 3}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::NO_INTERSECTION);
ASSERT_EQ(boundaries.getIntersectionResult({{0, 100500}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::NO_INTERSECTION);
boundaries.addPart(PartToRead{{1, 1}, {"TestPart", "TestProjection"}});
boundaries.addPart(PartToRead{{2, 2}, {"TestPart", "TestProjection"}});
boundaries.addPart(PartToRead{{3, 3}, {"TestPart", "TestProjection"}});
ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION);
ASSERT_EQ(boundaries.getIntersectionResult({{1, 3}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT);
ASSERT_EQ(boundaries.getIntersectionResult({{100, 100500}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::NO_INTERSECTION);
}
}

View File

@ -1,15 +1,16 @@
#pragma once
#include <Interpreters/PreparedSets.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Core/SortDescription.h>
#include <Core/Names.h>
#include <Storages/ProjectionsDescription.h>
#include <Interpreters/AggregateDescription.h>
#include <QueryPipeline/StreamLocalLimits.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/TableExpressionModifiers.h>
#include <Core/Names.h>
#include <Core/SortDescription.h>
#include <Interpreters/AggregateDescription.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/PreparedSets.h>
#include <Planner/PlannerContext.h>
#include <QueryPipeline/StreamLocalLimits.h>
#include <Storages/ProjectionsDescription.h>
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
#include <memory>
@ -207,6 +208,8 @@ struct SelectQueryInfo
/// Configured in StorageDistributed::getQueryProcessingStage()
ClusterPtr optimized_cluster;
mutable ParallelReplicasReadingCoordinatorPtr coordinator;
TreeRewriterResultPtr syntax_analyzer_result;
/// This is an additional filer applied to current table.

View File

@ -140,52 +140,6 @@ namespace ActionLocks
namespace
{
/// select query has database, table and table function names as AST pointers
/// Creates a copy of query, changes database, table and table function names.
ASTPtr rewriteSelectQuery(
ContextPtr context,
const ASTPtr & query,
const std::string & remote_database,
const std::string & remote_table,
ASTPtr table_function_ptr = nullptr)
{
auto modified_query_ast = query->clone();
ASTSelectQuery & select_query = modified_query_ast->as<ASTSelectQuery &>();
// Get rid of the settings clause so we don't send them to remote. Thus newly non-important
// settings won't break any remote parser. It's also more reasonable since the query settings
// are written into the query context and will be sent by the query pipeline.
select_query.setExpression(ASTSelectQuery::Expression::SETTINGS, {});
if (table_function_ptr)
select_query.addTableFunction(table_function_ptr);
else
select_query.replaceDatabaseAndTable(remote_database, remote_table);
/// Restore long column names (cause our short names are ambiguous).
/// TODO: aliased table functions & CREATE TABLE AS table function cases
if (!table_function_ptr)
{
RestoreQualifiedNamesVisitor::Data data;
data.distributed_table = DatabaseAndTableWithAlias(*getTableExpression(query->as<ASTSelectQuery &>(), 0));
data.remote_table.database = remote_database;
data.remote_table.table = remote_table;
RestoreQualifiedNamesVisitor(data).visit(modified_query_ast);
}
/// To make local JOIN works, default database should be added to table names.
/// But only for JOIN section, since the following should work using default_database:
/// - SELECT * FROM d WHERE value IN (SELECT l.value FROM l) ORDER BY value
/// (see 01487_distributed_in_not_default_db)
AddDefaultDatabaseVisitor visitor(context, context->getCurrentDatabase(),
/* only_replace_current_database_function_= */false,
/* only_replace_in_join_= */true);
visitor.visit(modified_query_ast);
return modified_query_ast;
}
/// Calculate maximum number in file names in directory and all subdirectories.
/// To ensure global order of data blocks yet to be sent across server restarts.
UInt64 getMaximumFileNumber(const std::string & dir_path)
@ -696,6 +650,7 @@ void StorageDistributed::read(
const size_t /*max_block_size*/,
const size_t /*num_streams*/)
{
const auto * select_query = query_info.query->as<ASTSelectQuery>();
if (select_query->final() && local_context->getSettingsRef().allow_experimental_parallel_reading_from_replicas)
throw Exception(ErrorCodes::ILLEGAL_FINAL, "Final modifier is not allowed together with parallel reading from replicas feature");
@ -719,10 +674,11 @@ void StorageDistributed::read(
query_ast = query_info.query;
}
auto modified_query_ast = rewriteSelectQuery(
local_context, query_ast,
const auto & modified_query_ast = ClusterProxy::rewriteSelectQuery(
local_context, query_info.query,
remote_database, remote_table, remote_table_function_ptr);
/// Return directly (with correct header) if no shard to query.
if (query_info.getCluster()->getShardsInfo().empty())
{
@ -746,25 +702,13 @@ void StorageDistributed::read(
storage_snapshot,
processed_stage);
auto settings = local_context->getSettingsRef();
bool parallel_replicas = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas && !settings.use_hedged_requests;
if (parallel_replicas)
ClusterProxy::executeQueryWithParallelReplicas(
query_plan, main_table, remote_table_function_ptr,
select_stream_factory, modified_query_ast,
local_context, query_info,
sharding_key_expr, sharding_key_column_name,
query_info.cluster, processed_stage);
else
ClusterProxy::executeQuery(
query_plan, header, processed_stage,
main_table, remote_table_function_ptr,
select_stream_factory, log, modified_query_ast,
local_context, query_info,
sharding_key_expr, sharding_key_column_name,
query_info.cluster);
ClusterProxy::executeQuery(
query_plan, header, processed_stage,
main_table, remote_table_function_ptr,
select_stream_factory, log, modified_query_ast,
local_context, query_info,
sharding_key_expr, sharding_key_column_name,
query_info.cluster);
/// This is a bug, it is possible only when there is no shards to query, and this is handled earlier.
if (!query_plan.isInitialized())

View File

@ -1,4 +1,5 @@
#include "StorageMergeTree.h"
#include "Core/QueryProcessingStage.h"
#include "Storages/MergeTree/IMergeTreeDataPart.h"
#include <optional>
@ -14,6 +15,8 @@
#include <Interpreters/MutationsInterpreter.h>
#include <Interpreters/Context.h>
#include <Interpreters/TransactionLog.h>
#include <Interpreters/ClusterProxy/executeQuery.h>
#include <Interpreters/ClusterProxy/SelectStreamFactory.h>
#include <IO/copyData.h>
#include <Parsers/ASTCheckQuery.h>
#include <Parsers/ASTFunction.h>
@ -209,15 +212,39 @@ void StorageMergeTree::read(
size_t max_block_size,
size_t num_streams)
{
/// If true, then we will ask initiator if we can read chosen ranges
bool enable_parallel_reading = local_context->getClientInfo().collaborate_with_initiator;
if (local_context->canUseParallelReplicasOnInitiator())
{
auto table_id = getStorageID();
if (enable_parallel_reading)
LOG_TRACE(log, "Parallel reading from replicas enabled: {}", enable_parallel_reading);
const auto & modified_query_ast = ClusterProxy::rewriteSelectQuery(
local_context, query_info.query,
table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr);
if (auto plan = reader.read(
column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams, processed_stage, nullptr, enable_parallel_reading))
query_plan = std::move(*plan);
auto cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas);
Block header =
InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
ClusterProxy::SelectStreamFactory select_stream_factory =
ClusterProxy::SelectStreamFactory(
header,
{},
storage_snapshot,
processed_stage);
ClusterProxy::executeQueryWithParallelReplicas(
query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr,
select_stream_factory, modified_query_ast,
local_context, query_info, cluster);
}
else
{
if (auto plan = reader.read(
column_names, storage_snapshot, query_info,
local_context, max_block_size, num_streams,
processed_stage, nullptr, /*enable_parallel_reading*/local_context->canUseParallelReplicasOnFollower()))
query_plan = std::move(*plan);
}
/// Now, copy of parts that is required for the query, stored in the processors,
/// while snapshot_data.parts includes all parts, even one that had been filtered out with partition pruning,

View File

@ -162,4 +162,3 @@ public:
}

View File

@ -71,15 +71,18 @@
#include <IO/ConnectionTimeouts.h>
#include <IO/ConnectionTimeoutsContext.h>
#include <Interpreters/InterpreterAlterQuery.h>
#include <Interpreters/PartLog.h>
#include <Interpreters/ClusterProxy/executeQuery.h>
#include <Interpreters/ClusterProxy/SelectStreamFactory.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLTask.h>
#include <Interpreters/InterserverCredentials.h>
#include <Interpreters/SelectQueryOptions.h>
#include <Interpreters/InterpreterAlterQuery.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterserverCredentials.h>
#include <Interpreters/PartLog.h>
#include <Interpreters/SelectQueryOptions.h>
#include <Interpreters/JoinedTables.h>
#include <Backups/BackupEntriesCollector.h>
#include <Backups/IBackup.h>
#include <Backups/IBackupCoordination.h>
@ -4543,9 +4546,6 @@ void StorageReplicatedMergeTree::read(
const size_t max_block_size,
const size_t num_streams)
{
/// If true, then we will ask initiator if we can read chosen ranges
const bool enable_parallel_reading = local_context->getClientInfo().collaborate_with_initiator;
SCOPE_EXIT({
/// Now, copy of parts that is required for the query, stored in the processors,
/// while snapshot_data.parts includes all parts, even one that had been filtered out with partition pruning,
@ -4564,16 +4564,43 @@ void StorageReplicatedMergeTree::read(
auto max_added_blocks = std::make_shared<ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock>(getMaxAddedBlocks());
if (auto plan = reader.read(
column_names, storage_snapshot, query_info, local_context,
max_block_size, num_streams, processed_stage, std::move(max_added_blocks), enable_parallel_reading))
max_block_size, num_streams, processed_stage, std::move(max_added_blocks), /*enable_parallel_reading*/false))
query_plan = std::move(*plan);
return;
}
if (auto plan = reader.read(
column_names, storage_snapshot, query_info, local_context,
max_block_size, num_streams, processed_stage, nullptr, enable_parallel_reading))
if (local_context->canUseParallelReplicasOnInitiator())
{
query_plan = std::move(*plan);
auto table_id = getStorageID();
const auto & modified_query_ast = ClusterProxy::rewriteSelectQuery(
local_context, query_info.query,
table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr);
auto cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas);
Block header =
InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
ClusterProxy::SelectStreamFactory select_stream_factory =
ClusterProxy::SelectStreamFactory(
header,
{},
storage_snapshot,
processed_stage);
ClusterProxy::executeQueryWithParallelReplicas(
query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr,
select_stream_factory, modified_query_ast,
local_context, query_info, cluster);
}
else
{
if (auto plan = reader.read(
column_names, storage_snapshot, query_info,
local_context, max_block_size, num_streams,
processed_stage, nullptr, /*enable_parallel_reading*/local_context->canUseParallelReplicasOnFollower()))
query_plan = std::move(*plan);
}
}

View File

@ -79,7 +79,7 @@ ColumnsDescription getStructureOfRemoteTableInShard(
ParserExpression expr_parser;
while (Block current = executor.read())
while (Block current = executor.readBlock())
{
ColumnPtr name = current.getByName("name").column;
ColumnPtr type = current.getByName("type").column;
@ -187,7 +187,7 @@ ColumnsDescriptionByShardNum getExtendedObjectsOfRemoteTables(
executor.setMainTable(remote_table_id);
ColumnsDescription res;
while (auto block = executor.read())
while (auto block = executor.readBlock())
{
const auto & name_col = *block.getByName("name").column;
const auto & type_col = *block.getByName("type").column;

View File

@ -209,6 +209,26 @@ CI_CONFIG = {
"Stateful tests (release, DatabaseReplicated)": {
"required_build": "package_release",
},
# Stateful tests for parallel replicas
"Stateful tests (release, ParallelReplicas)": {
"required_build": "package_release",
},
"Stateful tests (debug, ParallelReplicas)": {
"required_build": "package_debug",
},
"Stateful tests (asan, ParallelReplicas)": {
"required_build": "package_asan",
},
"Stateful tests (msan, ParallelReplicas)": {
"required_build": "package_msan",
},
"Stateful tests (ubsan, ParallelReplicas)": {
"required_build": "package_ubsan",
},
"Stateful tests (tsan, ParallelReplicas)": {
"required_build": "package_tsan",
},
# End stateful tests for parallel replicas
"Stateless tests (asan)": {
"required_build": "package_asan",
},

View File

@ -48,7 +48,8 @@ def get_additional_envs(check_name, run_by_hash_num, run_by_hash_total):
result.append("USE_DATABASE_ORDINARY=1")
if "wide parts enabled" in check_name:
result.append("USE_POLYMORPHIC_PARTS=1")
if "ParallelReplicas" in check_name:
result.append("USE_PARALLEL_REPLICAS=1")
if "s3 storage" in check_name:
result.append("USE_S3_STORAGE_FOR_MERGE_TREE=1")
@ -355,16 +356,34 @@ def main():
print(f"::notice:: {check_name} Report url: {report_url}")
if args.post_commit_status == "commit_status":
post_commit_status(
gh, pr_info.sha, check_name_with_group, description, state, report_url
)
if "parallelreplicas" in check_name.lower():
post_commit_status(
gh,
pr_info.sha,
check_name_with_group,
description,
"success",
report_url,
)
else:
post_commit_status(
gh, pr_info.sha, check_name_with_group, description, state, report_url
)
elif args.post_commit_status == "file":
post_commit_status_to_file(
post_commit_path,
description,
state,
report_url,
)
if "parallelreplicas" in check_name.lower():
post_commit_status_to_file(
post_commit_path,
description,
"success",
report_url,
)
else:
post_commit_status_to_file(
post_commit_path,
description,
state,
report_url,
)
else:
raise Exception(
f'Unknown post_commit_status option "{args.post_commit_status}"'
@ -382,7 +401,11 @@ def main():
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
if state != "success":
if FORCE_TESTS_LABEL in pr_info.labels:
# Parallel replicas are always green for now
if (
FORCE_TESTS_LABEL in pr_info.labels
or "parallelreplicas" in check_name.lower()
):
print(f"'{FORCE_TESTS_LABEL}' enabled, will report success")
else:
sys.exit(1)

View File

@ -442,6 +442,7 @@ class FailureReason(enum.Enum):
STRESS = "stress"
BUILD = "not running for current build"
BACKWARD_INCOMPATIBLE = "test is backward incompatible"
NO_PARALLEL_REPLICAS = "smth in not supported with parallel replicas"
# UNKNOWN reasons
NO_REFERENCE = "no reference file"
@ -729,6 +730,9 @@ class TestCase:
):
return FailureReason.DISABLED
elif "no-parallel-replicas" in tags and args.no_parallel_replicas:
return FailureReason.NO_PARALLEL_REPLICAS
elif args.skip and any(s in self.name for s in args.skip):
return FailureReason.SKIP
@ -2399,6 +2403,13 @@ if __name__ == "__main__":
default=False,
help="Report statistics about log messages",
)
parser.add_argument(
"--no-parallel-replicas",
action="store_true",
default=False,
help="Do not include tests that are not supported with parallel replicas feature",
)
args = parser.parse_args()
if args.queries and not os.path.isdir(args.queries):

View File

@ -11,7 +11,5 @@
<query>select avg(a) from remote('127.0.0.{{1,2}}', default, t_mbm) group by a format Null</query>
<query>select * from remote('127.0.0.{{1,2}}', default, t_mbm) group by a format Null settings allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 2, use_hedged_requests = 0</query>
<drop_query>drop table t_mbm</drop_query>
</test>

View File

@ -98,8 +98,9 @@ select a, count() from dist_t_different_dbs group by a, b order by a limit 5 off
502 2000
503 2000
504 2000
1000000
-- { echoOn } --
explain pipeline select a from dist_pr_t group by a order by a limit 5 offset 500;
explain pipeline select a from pr_t group by a order by a limit 5 offset 500;
(Expression)
ExpressionTransform
(Limit)
@ -112,28 +113,29 @@ ExpressionTransform
(Expression)
ExpressionTransform × 4
(MergingAggregated)
MergingAggregatedBucketTransform × 4
Resize 1 → 4
FinishAggregatingInOrderTransform 3 → 1
(Union)
(Aggregating)
SortingAggregatedForMemoryBoundMergingTransform 4 → 1
MergingAggregatedBucketTransform × 4
Resize 1 → 4
FinishAggregatingInOrderTransform 4 → 1
AggregatingInOrderTransform × 4
(Expression)
ExpressionTransform × 4
(ReadFromMergeTree)
MergeTreeInOrder × 4 0 → 1
(ReadFromRemoteParallelReplicas)
select a, count() from dist_pr_t group by a order by a limit 5 offset 500;
Resize 1 → 4
SortingAggregatedTransform 4 → 1
MergingAggregatedBucketTransform × 4
Resize 1 → 4
GroupingAggregatedTransform 6 → 1
(Union)
(Aggregating)
MergingAggregatedBucketTransform × 4
Resize 1 → 4
FinishAggregatingInOrderTransform 4 → 1
AggregatingInOrderTransform × 4
(Expression)
ExpressionTransform × 4
(ReadFromMergeTree)
MergeTreeInOrder × 4 0 → 1
(ReadFromRemoteParallelReplicas)
select a, count() from pr_t group by a order by a limit 5 offset 500;
500 1000
501 1000
502 1000
503 1000
504 1000
select a, count() from dist_pr_t group by a, b order by a limit 5 offset 500;
select a, count() from pr_t group by a, b order by a limit 5 offset 500;
500 1000
501 1000
502 1000

View File

@ -1,13 +1,13 @@
-- Tags: no-parallel
drop table if exists pr_t;
drop table if exists dist_pr_t;
drop table if exists dist_t_different_dbs;
drop table if exists shard_1.t_different_dbs;
drop table if exists t_different_dbs;
drop table if exists dist_t;
drop table if exists t;
create table t(a UInt64, b UInt64) engine=MergeTree order by a;
system stop merges t;
insert into t select number, number from numbers_mt(1e6);
@ -15,6 +15,7 @@ insert into t select number, number from numbers_mt(1e6);
set enable_memory_bound_merging_of_aggregation_results = 1;
set max_threads = 4;
set optimize_aggregation_in_order = 1;
set optimize_read_in_order = 1;
set prefer_localhost_replica = 1;
-- slightly different transforms will be generated by reading steps if we let settings randomisation to change this setting value --
@ -56,26 +57,28 @@ select a, count() from dist_t_different_dbs group by a, b order by a limit 5 off
-- { echoOff } --
create table pr_t(a UInt64, b UInt64) engine=MergeTree order by a;
insert into pr_t select number % 1000, number % 1000 from numbers_mt(1e6);
set allow_experimental_parallel_reading_from_replicas = 1;
set max_parallel_replicas = 3;
set use_hedged_requests = 0;
set cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost';
set distributed_aggregation_memory_efficient=1;
create table pr_t(a UInt64, b UInt64) engine=MergeTree order by a;
insert into pr_t select number % 1000, number % 1000 from numbers_mt(1e6);
create table dist_pr_t as pr_t engine = Distributed(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), pr_t);
select count() from pr_t;
-- { echoOn } --
explain pipeline select a from dist_pr_t group by a order by a limit 5 offset 500;
explain pipeline select a from pr_t group by a order by a limit 5 offset 500;
select a, count() from dist_pr_t group by a order by a limit 5 offset 500;
select a, count() from dist_pr_t group by a, b order by a limit 5 offset 500;
select a, count() from pr_t group by a order by a limit 5 offset 500;
select a, count() from pr_t group by a, b order by a limit 5 offset 500;
-- { echoOff } --
drop table pr_t;
drop table dist_pr_t;
drop table dist_t_different_dbs;
drop table shard_1.t_different_dbs;
drop table t_different_dbs;
drop table dist_t;
drop table t;
drop table if exists pr_t;
drop table if exists dist_t_different_dbs;
drop table if exists shard_1.t_different_dbs;
drop table if exists t_different_dbs;
drop table if exists dist_t;
drop table if exists t;

View File

@ -1,3 +1,4 @@
-- Tags: distributed
SELECT uniq(UserID), uniqIf(UserID, CounterID = 800784), uniqIf(FUniqID, RegionID = 213) FROM remote('127.0.0.{1,2}', test, hits)

View File

@ -1,3 +1,4 @@
-- Tags: distributed
SELECT EventTime::DateTime('Asia/Dubai') FROM remote('127.0.0.{1,2}', test, hits) ORDER BY EventTime DESC LIMIT 10

View File

@ -1,2 +1,4 @@
-- Tags: no-parallel-replicas
SELECT ParsedParams.Key1 FROM test.visits FINAL WHERE VisitID != 0 AND notEmpty(ParsedParams.Key1) ORDER BY VisitID LIMIT 10

View File

@ -1,3 +1,4 @@
-- Tags: distributed
SELECT anyIf(SearchPhrase, CounterID = -1) FROM remote('127.0.0.{1,2}:9000', test, hits)

View File

@ -1,3 +1,5 @@
-- Tags: no-parallel-replicas
DROP TABLE IF EXISTS test.merge_hits;
CREATE TABLE IF NOT EXISTS test.merge_hits AS test.hits ENGINE = Merge(test, '^hits$');
SELECT count() FROM test.merge_hits WHERE AdvEngineID = 2;

View File

@ -1,3 +1,5 @@
-- Tags: no-parallel-replicas
SELECT
EventDate,
hits,

View File

@ -1,3 +1,5 @@
-- Tags: no-parallel-replicas
SELECT
EventDate,
count() AS hits,

View File

@ -1,3 +1,5 @@
-- Tags: no-parallel-replicas
SELECT
domain,
hits,

View File

@ -1,15 +1,17 @@
-- Tags: no-parallel-replicas
SET any_join_distinct_right_table_keys = 1;
SET joined_subquery_requires_alias = 0;
SELECT
loyalty,
loyalty,
count()
FROM test.hits ANY LEFT JOIN
FROM test.hits ANY LEFT JOIN
(
SELECT
UserID,
sum(SearchEngineID = 2) AS yandex,
sum(SearchEngineID = 3) AS google,
UserID,
sum(SearchEngineID = 2) AS yandex,
sum(SearchEngineID = 3) AS google,
toInt8(if(yandex > google, yandex / (yandex + google), -google / (yandex + google)) * 10) AS loyalty
FROM test.hits
WHERE (SearchEngineID = 2) OR (SearchEngineID = 3)
@ -21,18 +23,18 @@ ORDER BY loyalty ASC;
SELECT
loyalty,
loyalty,
count()
FROM
(
SELECT UserID
FROM test.hits
) ANY LEFT JOIN
) ANY LEFT JOIN
(
SELECT
UserID,
sum(SearchEngineID = 2) AS yandex,
sum(SearchEngineID = 3) AS google,
UserID,
sum(SearchEngineID = 2) AS yandex,
sum(SearchEngineID = 3) AS google,
toInt8(if(yandex > google, yandex / (yandex + google), -google / (yandex + google)) * 10) AS loyalty
FROM test.hits
WHERE (SearchEngineID = 2) OR (SearchEngineID = 3)
@ -44,23 +46,23 @@ ORDER BY loyalty ASC;
SELECT
loyalty,
loyalty,
count()
FROM
(
SELECT
loyalty,
loyalty,
UserID
FROM
(
SELECT UserID
FROM test.hits
) ANY LEFT JOIN
) ANY LEFT JOIN
(
SELECT
UserID,
sum(SearchEngineID = 2) AS yandex,
sum(SearchEngineID = 3) AS google,
UserID,
sum(SearchEngineID = 2) AS yandex,
sum(SearchEngineID = 3) AS google,
toInt8(if(yandex > google, yandex / (yandex + google), -google / (yandex + google)) * 10) AS loyalty
FROM test.hits
WHERE (SearchEngineID = 2) OR (SearchEngineID = 3)
@ -73,19 +75,19 @@ ORDER BY loyalty ASC;
SELECT
loyalty,
count() AS c,
loyalty,
count() AS c,
bar(log(c + 1) * 1000, 0, log(3000000) * 1000, 80)
FROM test.hits ANY INNER JOIN
FROM test.hits ANY INNER JOIN
(
SELECT
UserID,
UserID,
toInt8(if(yandex > google, yandex / (yandex + google), -google / (yandex + google)) * 10) AS loyalty
FROM
(
SELECT
UserID,
sum(SearchEngineID = 2) AS yandex,
UserID,
sum(SearchEngineID = 2) AS yandex,
sum(SearchEngineID = 3) AS google
FROM test.hits
WHERE (SearchEngineID = 2) OR (SearchEngineID = 3)

View File

@ -1,3 +1,5 @@
-- Tags: no-parallel-replicas
USE test;
DROP TABLE IF EXISTS join;
@ -7,7 +9,7 @@ INSERT INTO join
SELECT
UserID,
toInt8(if((sum(SearchEngineID = 2) AS yandex) > (sum(SearchEngineID = 3) AS google),
yandex / (yandex + google),
yandex / (yandex + google),
-google / (yandex + google)) * 10) AS loyalty
FROM hits
WHERE (SearchEngineID = 2) OR (SearchEngineID = 3)

Some files were not shown because too many files have changed in this diff Show More