Fix logical error in partial result with totals

This commit is contained in:
vdimir 2023-09-22 13:03:11 +00:00
parent 4e86eec7d1
commit 99945b7d27
No known key found for this signature in database
GPG Key ID: 6EE4CE2BEDC51862

View File

@ -14,6 +14,8 @@
#include <Columns/ColumnConst.h>
#include <Common/logger_useful.h>
#include <QueryPipeline/printPipeline.h>
namespace DB
{
@ -654,7 +656,15 @@ void Pipe::addPartialResultTransform(const ProcessorPtr & transform)
{
if (isPartialResultActive())
{
size_t new_outputs_size = transform->getOutputs().size();
size_t new_outputs_size = 0;
for (const auto & output : transform->getOutputs())
{
/// We do not use totals_port and extremes_port in partial result
if ((totals_port && totals_port == &output) || (extremes_port && extremes_port == &output))
continue;
++new_outputs_size;
}
auto partial_result_status = transform->getPartialResultProcessorSupportStatus();
if (partial_result_status == IProcessor::PartialResultStatus::SkipSupported && new_outputs_size != partial_result_ports.size())
@ -671,6 +681,7 @@ void Pipe::addPartialResultTransform(const ProcessorPtr & transform)
dropPort(partial_result_port, *processors, collected_processors);
partial_result_ports.assign(new_outputs_size, nullptr);
return;
}
if (partial_result_status != IProcessor::PartialResultStatus::FullSupported)
@ -680,12 +691,18 @@ void Pipe::addPartialResultTransform(const ProcessorPtr & transform)
auto & inputs = partial_result_transform->getInputs();
if (inputs.size() != partial_result_ports.size())
{
WriteBufferFromOwnString out;
if (processors && !processors->empty())
printPipeline(*processors, out);
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Cannot add partial result transform {} to Pipe because it has {} input ports, but {} expected",
"Cannot add partial result transform {} to Pipe because it has {} input ports, but {} expected\n{}",
partial_result_transform->getName(),
inputs.size(),
partial_result_ports.size());
partial_result_ports.size(), out.str());
}
size_t next_port = 0;
for (auto & input : inputs)