aboutsummaryrefslogtreecommitdiffstats
path: root/contrib/clickhouse/src/Processors/Sources/DelayedSource.cpp
blob: f7928f8901548cea53afc2858a640ba431694271 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
#include <Processors/Sources/DelayedSource.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/Sinks/NullSink.h>
#include <Processors/ResizeProcessor.h>
#include <QueryPipeline/QueryPipelineBuilder.h>

namespace DB
{

DelayedSource::DelayedSource(const Block & header, Creator processors_creator, bool add_totals_port, bool add_extremes_port)
    : IProcessor({}, OutputPorts(1 + (add_totals_port ? 1 : 0) + (add_extremes_port ? 1 : 0), header))
    , creator(std::move(processors_creator))
{
    auto output = outputs.begin();

    main = &*output;
    ++output;

    if (add_totals_port)
    {
        totals = &*output;
        ++output;
    }

    if (add_extremes_port)
    {
        extremes = &*output;
        ++output;
    }
}

IProcessor::Status DelayedSource::prepare()
{
    /// At first, wait for main output is needed and expand pipeline.
    if (inputs.empty())
    {
        auto & first_output = outputs.front();

        /// If main output port was finished before callback was called, stop execution.
        if (first_output.isFinished())
        {
            for (auto & output : outputs)
                output.finish();

            return Status::Finished;
        }

        if (!first_output.isNeeded())
            return Status::PortFull;

        /// Call creator callback to get processors.
        if (processors.empty())
            return Status::Ready;

        return Status::ExpandPipeline;
    }

    /// Process ports in order: main, totals, extremes
    auto output = outputs.begin();
    for (auto input = inputs.begin(); input != inputs.end(); ++input, ++output)
    {
        if (output->isFinished())
        {
            input->close();
            continue;
        }

        if (!output->canPush())
            return Status::PortFull;

        if (input->isFinished())
        {
            output->finish();
            continue;
        }

        input->setNeeded();
        if (!input->hasData())
            return Status::NeedData;

        output->pushData(input->pullData(true));
        return Status::PortFull;
    }

    return Status::Finished;
}

/// Fix port from returned pipe. Create source_port if created or drop if source_port is null.
void synchronizePorts(OutputPort *& pipe_port, OutputPort * source_port, const Block & header, Processors & processors)
{
    if (source_port)
    {
        /// Need port in DelayedSource. Create NullSource.
        if (!pipe_port)
        {
            processors.emplace_back(std::make_shared<NullSource>(header));
            pipe_port = &processors.back()->getOutputs().back();
        }
    }
    else
    {
        /// Has port in pipe, but don't need it. Create NullSink.
        if (pipe_port)
        {
            auto sink = std::make_shared<NullSink>(header);
            connect(*pipe_port, sink->getPort());
            processors.emplace_back(std::move(sink));
            pipe_port = nullptr;
        }
    }
}

void DelayedSource::work()
{
    auto builder = creator();
    auto pipe = QueryPipelineBuilder::getPipe(std::move(builder), resources);

    const auto & header = main->getHeader();

    if (pipe.empty())
    {
        auto source = std::make_shared<NullSource>(header);
        main_output = &source->getPort();
        processors.emplace_back(std::move(source));
        return;
    }

    pipe.resize(1);

    main_output = pipe.getOutputPort(0);
    totals_output = pipe.getTotalsPort();
    extremes_output = pipe.getExtremesPort();

    processors = Pipe::detachProcessors(std::move(pipe));

    if (rows_before_limit)
    {
        for (auto & processor : processors)
            processor->setRowsBeforeLimitCounter(rows_before_limit);
    }

    synchronizePorts(totals_output, totals, header, processors);
    synchronizePorts(extremes_output, extremes, header, processors);
}

Processors DelayedSource::expandPipeline()
{
    /// Add new inputs. They must have the same header as output.
    for (const auto & output : {main_output, totals_output, extremes_output})
    {
        if (!output)
            continue;

        inputs.emplace_back(outputs.front().getHeader(), this);
        /// Connect checks that header is same for ports.
        connect(*output, inputs.back());

        if (output == main_output)
            inputs.back().setNeeded();
    }

    /// Executor will check that all processors are connected.
    return std::move(processors);
}

Pipe createDelayedPipe(const Block & header, DelayedSource::Creator processors_creator, bool add_totals_port, bool add_extremes_port)
{
    auto source = std::make_shared<DelayedSource>(header, std::move(processors_creator), add_totals_port, add_extremes_port);

    auto * main = &source->getPort();
    auto * totals = source->getTotalsPort();
    auto * extremes = source->getExtremesPort();

    return Pipe(std::move(source), main, totals, extremes);
}

}