mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge branch 'master' into variance-nonnegative
This commit is contained in:
commit
7df4748753
@ -21,7 +21,7 @@ set(CMAKE_C_STANDARD_LIBRARIES ${DEFAULT_LIBS})
|
||||
# glibc-compatibility library relies to fixed version of libc headers
|
||||
# (because minor changes in function attributes between different glibc versions will introduce incompatibilities)
|
||||
# This is for x86_64. For other architectures we have separate toolchains.
|
||||
if (ARCH_AMD64)
|
||||
if (ARCH_AMD64 AND NOT_UNBUNDLED)
|
||||
set(CMAKE_C_STANDARD_INCLUDE_DIRECTORIES ${ClickHouse_SOURCE_DIR}/contrib/libc-headers/x86_64-linux-gnu ${ClickHouse_SOURCE_DIR}/contrib/libc-headers)
|
||||
set(CMAKE_CXX_STANDARD_INCLUDE_DIRECTORIES ${ClickHouse_SOURCE_DIR}/contrib/libc-headers/x86_64-linux-gnu ${ClickHouse_SOURCE_DIR}/contrib/libc-headers)
|
||||
endif ()
|
||||
|
@ -28,7 +28,7 @@
|
||||
#include <Compression/CompressionFactory.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
#include <Processors/Formats/LazyOutputFormat.h>
|
||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||
|
||||
#include "TCPHandler.h"
|
||||
|
||||
@ -553,68 +553,23 @@ void TCPHandler::processOrdinaryQueryWithProcessors()
|
||||
|
||||
/// Send header-block, to allow client to prepare output format for data to send.
|
||||
{
|
||||
auto & header = pipeline.getHeader();
|
||||
const auto & header = pipeline.getHeader();
|
||||
|
||||
if (header)
|
||||
sendData(header);
|
||||
}
|
||||
|
||||
auto lazy_format = std::make_shared<LazyOutputFormat>(pipeline.getHeader());
|
||||
pipeline.setOutput(lazy_format);
|
||||
|
||||
{
|
||||
auto thread_group = CurrentThread::getGroup();
|
||||
ThreadPool pool(1);
|
||||
auto executor = pipeline.execute();
|
||||
std::atomic_bool exception = false;
|
||||
PullingPipelineExecutor executor(pipeline);
|
||||
CurrentMetrics::Increment query_thread_metric_increment{CurrentMetrics::QueryThread};
|
||||
|
||||
pool.scheduleOrThrowOnError([&]()
|
||||
{
|
||||
/// ThreadStatus thread_status;
|
||||
|
||||
if (thread_group)
|
||||
CurrentThread::attachTo(thread_group);
|
||||
|
||||
SCOPE_EXIT(
|
||||
if (thread_group)
|
||||
CurrentThread::detachQueryIfNotDetached();
|
||||
);
|
||||
|
||||
CurrentMetrics::Increment query_thread_metric_increment{CurrentMetrics::QueryThread};
|
||||
setThreadName("QueryPipelineEx");
|
||||
|
||||
try
|
||||
{
|
||||
executor->execute(pipeline.getNumThreads());
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
exception = true;
|
||||
throw;
|
||||
}
|
||||
});
|
||||
|
||||
/// Wait in case of exception happened outside of pool.
|
||||
SCOPE_EXIT(
|
||||
lazy_format->finish();
|
||||
|
||||
try
|
||||
{
|
||||
pool.wait();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// If exception was thrown during pipeline execution, skip it while processing other exception.
|
||||
tryLogCurrentException(log);
|
||||
}
|
||||
);
|
||||
|
||||
while (!lazy_format->isFinished() && !exception)
|
||||
Block block;
|
||||
while (executor.pull(block, query_context->getSettingsRef().interactive_delay / 1000))
|
||||
{
|
||||
if (isQueryCancelled())
|
||||
{
|
||||
/// A packet was received requesting to stop execution of the request.
|
||||
executor->cancel();
|
||||
executor.cancel();
|
||||
break;
|
||||
}
|
||||
|
||||
@ -627,17 +582,13 @@ void TCPHandler::processOrdinaryQueryWithProcessors()
|
||||
|
||||
sendLogs();
|
||||
|
||||
if (auto block = lazy_format->getBlock(query_context->getSettingsRef().interactive_delay / 1000))
|
||||
if (block)
|
||||
{
|
||||
if (!state.io.null_format)
|
||||
sendData(block);
|
||||
}
|
||||
}
|
||||
|
||||
/// Finish lazy_format before waiting. Otherwise some thread may write into it, and waiting will lock.
|
||||
lazy_format->finish();
|
||||
pool.wait();
|
||||
|
||||
/** If data has run out, we will send the profiling data and total values to
|
||||
* the last zero block to be able to use
|
||||
* this information in the suffix output of stream.
|
||||
@ -647,9 +598,9 @@ void TCPHandler::processOrdinaryQueryWithProcessors()
|
||||
*/
|
||||
if (!isQueryCancelled())
|
||||
{
|
||||
sendTotals(lazy_format->getTotals());
|
||||
sendExtremes(lazy_format->getExtremes());
|
||||
sendProfileInfo(lazy_format->getProfileInfo());
|
||||
sendTotals(executor.getTotalsBlock());
|
||||
sendExtremes(executor.getExtremesBlock());
|
||||
sendProfileInfo(executor.getProfileInfo());
|
||||
sendProgress();
|
||||
sendLogs();
|
||||
}
|
||||
|
@ -370,7 +370,7 @@ if (ENABLE_TESTS AND USE_GTEST)
|
||||
endmacro()
|
||||
|
||||
# attach all dbms gtest sources
|
||||
grep_gtest_sources(${ClickHouse_SOURCE_DIR}/dbms dbms_gtest_sources)
|
||||
grep_gtest_sources(${ClickHouse_SOURCE_DIR}/src dbms_gtest_sources)
|
||||
add_executable(unit_tests_dbms ${dbms_gtest_sources})
|
||||
|
||||
# gtest framework has substandard code
|
||||
|
@ -113,7 +113,7 @@ public:
|
||||
|
||||
Field operator[](size_t n) const override { return DecimalField(data[n], scale); }
|
||||
|
||||
StringRef getRawData() const override { return StringRef(reinterpret_cast<const char*>(data.data()), data.size()); }
|
||||
StringRef getRawData() const override { return StringRef(reinterpret_cast<const char*>(data.data()), byteSize()); }
|
||||
StringRef getDataAt(size_t n) const override { return StringRef(reinterpret_cast<const char *>(&data[n]), sizeof(data[n])); }
|
||||
void get(size_t n, Field & res) const override { res = (*this)[n]; }
|
||||
bool getBool(size_t n) const override { return bool(data[n]); }
|
||||
|
@ -264,7 +264,7 @@ public:
|
||||
|
||||
bool isFixedAndContiguous() const override { return true; }
|
||||
size_t sizeOfValueIfFixed() const override { return sizeof(T); }
|
||||
StringRef getRawData() const override { return StringRef(reinterpret_cast<const char*>(data.data()), data.size()); }
|
||||
StringRef getRawData() const override { return StringRef(reinterpret_cast<const char*>(data.data()), byteSize()); }
|
||||
|
||||
|
||||
bool structureEquals(const IColumn & rhs) const override
|
||||
|
@ -104,7 +104,7 @@ namespace
|
||||
|
||||
// Read (bitwise) one object of type T
|
||||
template <typename T>
|
||||
std::enable_if_t<std::is_pod_v<T>, T> read(std::string_view & sp)
|
||||
std::enable_if_t<std::is_trivial_v<T> && std::is_standard_layout_v<T>, T> read(std::string_view & sp)
|
||||
{
|
||||
SAFE_CHECK(sp.size() >= sizeof(T), "underflow");
|
||||
T x;
|
||||
|
@ -62,10 +62,15 @@ bool BlockStreamProfileInfo::hasAppliedLimit() const
|
||||
|
||||
|
||||
void BlockStreamProfileInfo::update(Block & block)
|
||||
{
|
||||
update(block.rows(), block.bytes());
|
||||
}
|
||||
|
||||
void BlockStreamProfileInfo::update(size_t num_rows, size_t num_bytes)
|
||||
{
|
||||
++blocks;
|
||||
rows += block.rows();
|
||||
bytes += block.bytes();
|
||||
rows += num_rows;
|
||||
bytes += num_bytes;
|
||||
}
|
||||
|
||||
|
||||
|
@ -40,6 +40,7 @@ struct BlockStreamProfileInfo
|
||||
bool hasAppliedLimit() const;
|
||||
|
||||
void update(Block & block);
|
||||
void update(size_t num_rows, size_t num_bytes);
|
||||
|
||||
/// Binary serialization and deserialization of main fields.
|
||||
/// Writes only main fields i.e. fields that required by internal transmission protocol.
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <unordered_map>
|
||||
#include <string>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,105 +0,0 @@
|
||||
#include <Common/EventCounter.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Processors/Executors/ParallelPipelineExecutor.h>
|
||||
#include <Processors/Executors/traverse.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
//
|
||||
//ParallelPipelineExecutor::ParallelPipelineExecutor(const std::vector<ProcessorPtr> & processors, ThreadPool & pool)
|
||||
// : processors(processors), pool(pool)
|
||||
//{
|
||||
//}
|
||||
//
|
||||
//
|
||||
//ParallelPipelineExecutor::Status ParallelPipelineExecutor::prepare()
|
||||
//{
|
||||
// current_processor = nullptr;
|
||||
//
|
||||
// bool has_someone_to_wait = false;
|
||||
//
|
||||
// for (auto & element : processors)
|
||||
// {
|
||||
// traverse(*element,
|
||||
// [&] (IProcessor & processor)
|
||||
// {
|
||||
// {
|
||||
// std::lock_guard lock(mutex);
|
||||
// if (active_processors.count(&processor))
|
||||
// {
|
||||
// has_someone_to_wait = true;
|
||||
// return Status::Wait;
|
||||
// }
|
||||
// }
|
||||
//
|
||||
// Status status = processor.prepare();
|
||||
//
|
||||
// if (status == Status::Wait)
|
||||
// has_someone_to_wait = true;
|
||||
//
|
||||
// if (status == Status::Ready || status == Status::Async)
|
||||
// {
|
||||
// current_processor = &processor;
|
||||
// current_status = status;
|
||||
// }
|
||||
//
|
||||
// return status;
|
||||
// });
|
||||
//
|
||||
// if (current_processor)
|
||||
// break;
|
||||
// }
|
||||
//
|
||||
// if (current_processor)
|
||||
// return Status::Async;
|
||||
//
|
||||
// if (has_someone_to_wait)
|
||||
// return Status::Wait;
|
||||
//
|
||||
// for (auto & element : processors)
|
||||
// {
|
||||
// if (element->prepare() == Status::NeedData)
|
||||
// throw Exception("Pipeline stuck: " + element->getName() + " processor needs input data but no one is going to generate it", ErrorCodes::LOGICAL_ERROR);
|
||||
// if (element->prepare() == Status::PortFull)
|
||||
// throw Exception("Pipeline stuck: " + element->getName() + " processor has data in output port but no one is going to consume it", ErrorCodes::LOGICAL_ERROR);
|
||||
// }
|
||||
//
|
||||
// return Status::Finished;
|
||||
//}
|
||||
//
|
||||
//
|
||||
//void ParallelPipelineExecutor::schedule(EventCounter & watch)
|
||||
//{
|
||||
// if (!current_processor)
|
||||
// throw Exception("Bad pipeline", ErrorCodes::LOGICAL_ERROR);
|
||||
//
|
||||
// if (current_status == Status::Async)
|
||||
// {
|
||||
// current_processor->schedule(watch);
|
||||
// }
|
||||
// else
|
||||
// {
|
||||
// {
|
||||
// std::lock_guard lock(mutex);
|
||||
// active_processors.insert(current_processor);
|
||||
// }
|
||||
//
|
||||
// pool.scheduleOrThrowOnError([processor = current_processor, &watch, this]
|
||||
// {
|
||||
// processor->work();
|
||||
// {
|
||||
// std::lock_guard lock(mutex);
|
||||
// active_processors.erase(processor);
|
||||
// }
|
||||
// watch.notify();
|
||||
// });
|
||||
// }
|
||||
//}
|
||||
|
||||
}
|
||||
|
@ -1,41 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <vector>
|
||||
#include <set>
|
||||
#include <mutex>
|
||||
#include <Processors/IProcessor.h>
|
||||
|
||||
template <typename>
|
||||
class ThreadPoolImpl;
|
||||
class ThreadFromGlobalPool;
|
||||
using ThreadPool = ThreadPoolImpl<ThreadFromGlobalPool>;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Wraps pipeline in a single processor.
|
||||
* This processor has no inputs and outputs and just executes the pipeline,
|
||||
* performing all synchronous work within a threadpool.
|
||||
*/
|
||||
//class ParallelPipelineExecutor : public IProcessor
|
||||
//{
|
||||
//private:
|
||||
// Processors processors;
|
||||
// ThreadPool & pool;
|
||||
//
|
||||
// std::set<IProcessor *> active_processors;
|
||||
// std::mutex mutex;
|
||||
//
|
||||
// IProcessor * current_processor = nullptr;
|
||||
// Status current_status;
|
||||
//
|
||||
//public:
|
||||
// ParallelPipelineExecutor(const Processors & processors, ThreadPool & pool);
|
||||
//
|
||||
// String getName() const override { return "ParallelPipelineExecutor"; }
|
||||
//
|
||||
// Status prepare() override;
|
||||
// void schedule(EventCounter & watch) override;
|
||||
//};
|
||||
|
||||
}
|
198
src/Processors/Executors/PullingPipelineExecutor.cpp
Normal file
198
src/Processors/Executors/PullingPipelineExecutor.cpp
Normal file
@ -0,0 +1,198 @@
|
||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||
#include <Processors/Executors/PipelineExecutor.h>
|
||||
#include <Processors/Formats/LazyOutputFormat.h>
|
||||
#include <Processors/Transforms/AggregatingTransform.h>
|
||||
#include <Processors/QueryPipeline.h>
|
||||
|
||||
#include <Common/setThreadName.h>
|
||||
#include <ext/scope_guard.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct PullingPipelineExecutor::Data
|
||||
{
|
||||
PipelineExecutorPtr executor;
|
||||
std::exception_ptr exception;
|
||||
std::atomic_bool is_executed = false;
|
||||
std::atomic_bool has_exception = false;
|
||||
ThreadFromGlobalPool thread;
|
||||
|
||||
~Data()
|
||||
{
|
||||
if (thread.joinable())
|
||||
thread.join();
|
||||
}
|
||||
|
||||
void rethrowExceptionIfHas()
|
||||
{
|
||||
if (has_exception)
|
||||
{
|
||||
has_exception = false;
|
||||
std::rethrow_exception(std::move(exception));
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
PullingPipelineExecutor::PullingPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_)
|
||||
{
|
||||
lazy_format = std::make_shared<LazyOutputFormat>(pipeline.getHeader());
|
||||
pipeline.setOutput(lazy_format);
|
||||
}
|
||||
|
||||
PullingPipelineExecutor::~PullingPipelineExecutor()
|
||||
{
|
||||
try
|
||||
{
|
||||
cancel();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException("PullingPipelineExecutor");
|
||||
}
|
||||
}
|
||||
|
||||
static void threadFunction(PullingPipelineExecutor::Data & data, ThreadGroupStatusPtr thread_group, size_t num_threads)
|
||||
{
|
||||
if (thread_group)
|
||||
CurrentThread::attachTo(thread_group);
|
||||
|
||||
SCOPE_EXIT(
|
||||
if (thread_group)
|
||||
CurrentThread::detachQueryIfNotDetached();
|
||||
);
|
||||
|
||||
setThreadName("QueryPipelineEx");
|
||||
|
||||
try
|
||||
{
|
||||
data.executor->execute(num_threads);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
data.exception = std::current_exception();
|
||||
data.has_exception = true;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
bool PullingPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds)
|
||||
{
|
||||
if (!data)
|
||||
{
|
||||
data = std::make_unique<Data>();
|
||||
data->executor = pipeline.execute();
|
||||
|
||||
auto func = [&, thread_group = CurrentThread::getGroup()]()
|
||||
{
|
||||
threadFunction(*data, thread_group, pipeline.getNumThreads());
|
||||
};
|
||||
|
||||
data->thread = ThreadFromGlobalPool(std::move(func));
|
||||
}
|
||||
|
||||
if (data->has_exception)
|
||||
{
|
||||
/// Finish lazy format in case of exception. Otherwise thread.join() may hung.
|
||||
lazy_format->finish();
|
||||
data->has_exception = false;
|
||||
std::rethrow_exception(std::move(data->exception));
|
||||
}
|
||||
|
||||
if (lazy_format->isFinished())
|
||||
{
|
||||
data->is_executed = true;
|
||||
/// Wait thread ant rethrow exception if any.
|
||||
cancel();
|
||||
return false;
|
||||
}
|
||||
|
||||
chunk = lazy_format->getChunk(milliseconds);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool PullingPipelineExecutor::pull(Block & block, uint64_t milliseconds)
|
||||
{
|
||||
Chunk chunk;
|
||||
|
||||
if (!pull(chunk, milliseconds))
|
||||
return false;
|
||||
|
||||
if (!chunk)
|
||||
{
|
||||
/// In case if timeout exceeded.
|
||||
block.clear();
|
||||
return true;
|
||||
}
|
||||
|
||||
block = lazy_format->getPort(IOutputFormat::PortKind::Main).getHeader().cloneWithColumns(chunk.detachColumns());
|
||||
|
||||
if (auto chunk_info = chunk.getChunkInfo())
|
||||
{
|
||||
if (const auto * agg_info = typeid_cast<const AggregatedChunkInfo *>(chunk_info.get()))
|
||||
{
|
||||
block.info.bucket_num = agg_info->bucket_num;
|
||||
block.info.is_overflows = agg_info->is_overflows;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
void PullingPipelineExecutor::cancel()
|
||||
{
|
||||
/// Cancel execution if it wasn't finished.
|
||||
if (data && !data->is_executed && data->executor)
|
||||
data->executor->cancel();
|
||||
|
||||
/// Finish lazy format. Otherwise thread.join() may hung.
|
||||
if (!lazy_format->isFinished())
|
||||
lazy_format->finish();
|
||||
|
||||
/// Join thread here to wait for possible exception.
|
||||
if (data && data->thread.joinable())
|
||||
data->thread.join();
|
||||
|
||||
/// Rethrow exception to not swallow it in destructor.
|
||||
if (data)
|
||||
data->rethrowExceptionIfHas();
|
||||
}
|
||||
|
||||
Chunk PullingPipelineExecutor::getTotals()
|
||||
{
|
||||
return lazy_format->getTotals();
|
||||
}
|
||||
|
||||
Chunk PullingPipelineExecutor::getExtremes()
|
||||
{
|
||||
return lazy_format->getExtremes();
|
||||
}
|
||||
|
||||
Block PullingPipelineExecutor::getTotalsBlock()
|
||||
{
|
||||
auto totals = getTotals();
|
||||
|
||||
if (totals.empty())
|
||||
return {};
|
||||
|
||||
const auto & header = lazy_format->getPort(IOutputFormat::PortKind::Totals).getHeader();
|
||||
return header.cloneWithColumns(totals.detachColumns());
|
||||
}
|
||||
|
||||
Block PullingPipelineExecutor::getExtremesBlock()
|
||||
{
|
||||
auto extremes = getExtremes();
|
||||
|
||||
if (extremes.empty())
|
||||
return {};
|
||||
|
||||
const auto & header = lazy_format->getPort(IOutputFormat::PortKind::Extremes).getHeader();
|
||||
return header.cloneWithColumns(extremes.detachColumns());
|
||||
}
|
||||
|
||||
BlockStreamProfileInfo & PullingPipelineExecutor::getProfileInfo()
|
||||
{
|
||||
return lazy_format->getProfileInfo();
|
||||
}
|
||||
|
||||
}
|
54
src/Processors/Executors/PullingPipelineExecutor.h
Normal file
54
src/Processors/Executors/PullingPipelineExecutor.h
Normal file
@ -0,0 +1,54 @@
|
||||
#pragma once
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class QueryPipeline;
|
||||
class Block;
|
||||
class Chunk;
|
||||
class LazyOutputFormat;
|
||||
struct BlockStreamProfileInfo;
|
||||
|
||||
/// Pulling executor for QueryPipeline.
|
||||
/// Typical usage is:
|
||||
///
|
||||
/// PullingPipelineExecutor executor(query_pipeline);
|
||||
/// while (executor.pull(chunk))
|
||||
/// ... process chunk ...
|
||||
class PullingPipelineExecutor
|
||||
{
|
||||
public:
|
||||
explicit PullingPipelineExecutor(QueryPipeline & pipeline_);
|
||||
~PullingPipelineExecutor();
|
||||
|
||||
/// Methods return false if query is finished.
|
||||
/// If milliseconds > 0, returns empty object and `true` after timeout exceeded.
|
||||
/// You can use any pull method.
|
||||
bool pull(Chunk & chunk, uint64_t milliseconds = 0);
|
||||
bool pull(Block & block, uint64_t milliseconds = 0);
|
||||
|
||||
/// Stop execution. It is not necessary, but helps to stop execution before executor is destroyed.
|
||||
void cancel();
|
||||
|
||||
/// Get totals and extremes. Returns empty chunk if doesn't have any.
|
||||
Chunk getTotals();
|
||||
Chunk getExtremes();
|
||||
|
||||
/// Get totals and extremes. Returns empty chunk if doesn't have any.
|
||||
Block getTotalsBlock();
|
||||
Block getExtremesBlock();
|
||||
|
||||
/// Get query profile info.
|
||||
BlockStreamProfileInfo & getProfileInfo();
|
||||
|
||||
/// Internal executor data.
|
||||
struct Data;
|
||||
|
||||
private:
|
||||
QueryPipeline & pipeline;
|
||||
std::shared_ptr<LazyOutputFormat> lazy_format;
|
||||
std::unique_ptr<Data> data;
|
||||
};
|
||||
|
||||
}
|
@ -1,83 +0,0 @@
|
||||
#include <Processors/Executors/SequentialPipelineExecutor.h>
|
||||
#include <Processors/Executors/traverse.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
//SequentialPipelineExecutor::SequentialPipelineExecutor(const Processors & processors)
|
||||
// : processors(processors)
|
||||
//{
|
||||
//}
|
||||
//
|
||||
//
|
||||
//SequentialPipelineExecutor::Status SequentialPipelineExecutor::prepare()
|
||||
//{
|
||||
// current_processor = nullptr;
|
||||
//
|
||||
// bool has_someone_to_wait = false;
|
||||
// Status found_status = Status::Finished;
|
||||
//
|
||||
// for (auto & element : processors)
|
||||
// {
|
||||
// traverse(*element,
|
||||
// [&] (IProcessor & processor)
|
||||
// {
|
||||
// Status status = processor.prepare();
|
||||
//
|
||||
// if (status == Status::Wait)
|
||||
// has_someone_to_wait = true;
|
||||
//
|
||||
// if (status == Status::Ready || status == Status::Async)
|
||||
// {
|
||||
// current_processor = &processor;
|
||||
// found_status = status;
|
||||
// }
|
||||
//
|
||||
// return status;
|
||||
// });
|
||||
//
|
||||
// if (current_processor)
|
||||
// break;
|
||||
// }
|
||||
//
|
||||
// if (current_processor)
|
||||
// return found_status;
|
||||
// if (has_someone_to_wait)
|
||||
// return Status::Wait;
|
||||
//
|
||||
// for (auto & element : processors)
|
||||
// {
|
||||
// if (element->prepare() == Status::NeedData)
|
||||
// throw Exception("Pipeline stuck: " + element->getName() + " processor needs input data but no one is going to generate it", ErrorCodes::LOGICAL_ERROR);
|
||||
// if (element->prepare() == Status::PortFull)
|
||||
// throw Exception("Pipeline stuck: " + element->getName() + " processor has data in output port but no one is going to consume it", ErrorCodes::LOGICAL_ERROR);
|
||||
// }
|
||||
//
|
||||
// return Status::Finished;
|
||||
//}
|
||||
//
|
||||
//
|
||||
//void SequentialPipelineExecutor::work()
|
||||
//{
|
||||
// if (!current_processor)
|
||||
// throw Exception("Bad pipeline", ErrorCodes::LOGICAL_ERROR);
|
||||
//
|
||||
// current_processor->work();
|
||||
//}
|
||||
//
|
||||
//
|
||||
//void SequentialPipelineExecutor::schedule(EventCounter & watch)
|
||||
//{
|
||||
// if (!current_processor)
|
||||
// throw Exception("Bad pipeline", ErrorCodes::LOGICAL_ERROR);
|
||||
//
|
||||
// current_processor->schedule(watch);
|
||||
//}
|
||||
|
||||
}
|
||||
|
@ -1,31 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <vector>
|
||||
|
||||
#include <Processors/IProcessor.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Wraps pipeline in a single processor.
|
||||
* This processor has no inputs and outputs and just executes the pipeline,
|
||||
* performing all synchronous work from the current thread.
|
||||
*/
|
||||
//class SequentialPipelineExecutor : public IProcessor
|
||||
//{
|
||||
//private:
|
||||
// Processors processors;
|
||||
// IProcessor * current_processor = nullptr;
|
||||
//
|
||||
//public:
|
||||
// SequentialPipelineExecutor(const Processors & processors);
|
||||
//
|
||||
// String getName() const override { return "SequentialPipelineExecutor"; }
|
||||
//
|
||||
// Status prepare() override;
|
||||
// void work() override;
|
||||
// void schedule(EventCounter & watch) override;
|
||||
//};
|
||||
|
||||
}
|
@ -7,7 +7,7 @@ namespace DB
|
||||
|
||||
WriteBuffer LazyOutputFormat::out(nullptr, 0);
|
||||
|
||||
Block LazyOutputFormat::getBlock(UInt64 milliseconds)
|
||||
Chunk LazyOutputFormat::getChunk(UInt64 milliseconds)
|
||||
{
|
||||
if (finished_processing)
|
||||
{
|
||||
@ -19,38 +19,20 @@ Block LazyOutputFormat::getBlock(UInt64 milliseconds)
|
||||
if (!queue.tryPop(chunk, milliseconds))
|
||||
return {};
|
||||
|
||||
if (!chunk)
|
||||
return {};
|
||||
if (chunk)
|
||||
info.update(chunk.getNumRows(), chunk.allocatedBytes());
|
||||
|
||||
auto block = getPort(PortKind::Main).getHeader().cloneWithColumns(chunk.detachColumns());
|
||||
info.update(block);
|
||||
|
||||
if (auto chunk_info = chunk.getChunkInfo())
|
||||
{
|
||||
if (const auto * agg_info = typeid_cast<const AggregatedChunkInfo *>(chunk_info.get()))
|
||||
{
|
||||
block.info.bucket_num = agg_info->bucket_num;
|
||||
block.info.is_overflows = agg_info->is_overflows;
|
||||
}
|
||||
}
|
||||
|
||||
return block;
|
||||
return chunk;
|
||||
}
|
||||
|
||||
Block LazyOutputFormat::getTotals()
|
||||
Chunk LazyOutputFormat::getTotals()
|
||||
{
|
||||
if (!totals)
|
||||
return {};
|
||||
|
||||
return getPort(PortKind::Totals).getHeader().cloneWithColumns(totals.detachColumns());
|
||||
return std::move(totals);
|
||||
}
|
||||
|
||||
Block LazyOutputFormat::getExtremes()
|
||||
Chunk LazyOutputFormat::getExtremes()
|
||||
{
|
||||
if (!extremes)
|
||||
return {};
|
||||
|
||||
return getPort(PortKind::Extremes).getHeader().cloneWithColumns(extremes.detachColumns());
|
||||
return std::move(extremes);
|
||||
}
|
||||
|
||||
void LazyOutputFormat::setRowsBeforeLimit(size_t rows_before_limit)
|
||||
|
@ -8,8 +8,8 @@ namespace DB
|
||||
{
|
||||
|
||||
/// LazyOutputFormat is used to retrieve ready data from executing pipeline.
|
||||
/// You can periodically call `getBlock` from separate thread.
|
||||
/// Used in TCPHandler.
|
||||
/// You can periodically call `getChunk` from separate thread.
|
||||
/// Used in PullingPipelineExecutor.
|
||||
class LazyOutputFormat : public IOutputFormat
|
||||
{
|
||||
|
||||
@ -19,9 +19,9 @@ public:
|
||||
|
||||
String getName() const override { return "LazyOutputFormat"; }
|
||||
|
||||
Block getBlock(UInt64 milliseconds = 0);
|
||||
Block getTotals();
|
||||
Block getExtremes();
|
||||
Chunk getChunk(UInt64 milliseconds = 0);
|
||||
Chunk getTotals();
|
||||
Chunk getExtremes();
|
||||
|
||||
bool isFinished() { return finished_processing && queue.size() == 0; }
|
||||
|
||||
|
@ -9,8 +9,6 @@
|
||||
#include <Processors/ForkProcessor.h>
|
||||
#include <Processors/LimitTransform.h>
|
||||
#include <Processors/QueueBuffer.h>
|
||||
#include <Processors/Executors/SequentialPipelineExecutor.h>
|
||||
#include <Processors/Executors/ParallelPipelineExecutor.h>
|
||||
#include <Processors/printPipeline.h>
|
||||
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
@ -9,8 +9,6 @@
|
||||
#include <Processors/ForkProcessor.h>
|
||||
#include <Processors/LimitTransform.h>
|
||||
#include <Processors/QueueBuffer.h>
|
||||
#include <Processors/Executors/SequentialPipelineExecutor.h>
|
||||
#include <Processors/Executors/ParallelPipelineExecutor.h>
|
||||
#include <Processors/printPipeline.h>
|
||||
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
@ -10,9 +10,8 @@ SRCS(
|
||||
Chunk.cpp
|
||||
ConcatProcessor.cpp
|
||||
DelayedPortsProcessor.cpp
|
||||
Executors/ParallelPipelineExecutor.cpp
|
||||
Executors/PipelineExecutor.cpp
|
||||
Executors/SequentialPipelineExecutor.cpp
|
||||
Executors/PullingPipelineExecutor.cpp
|
||||
Executors/TreeExecutorBlockInputStream.cpp
|
||||
ForkProcessor.cpp
|
||||
Formats/IInputFormat.cpp
|
||||
|
@ -2,3 +2,5 @@ OK
|
||||
OK
|
||||
5 10
|
||||
5 10
|
||||
dropped
|
||||
end
|
||||
|
@ -21,6 +21,7 @@ $CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107"
|
||||
$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 -q "ATTACH DATABASE test_01107 ENGINE=Atomic"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(100)" &
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" && echo "end" &
|
||||
sleep 1
|
||||
$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01107"
|
||||
$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01107" && sleep 1 && echo "dropped"
|
||||
wait
|
||||
|
@ -12,5 +12,6 @@ mt 00001114-0000-4000-8000-000000000002 CREATE TABLE test_01114_2.mt UUID \'0000
|
||||
CREATE TABLE test_01114_1.mt UUID \'00001114-0000-4000-8000-000000000001\'\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192
|
||||
CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192
|
||||
5
|
||||
dropped
|
||||
20 190
|
||||
30 435
|
||||
|
@ -57,7 +57,7 @@ $CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_1.mt" # result: 5
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt" > /dev/null & # 15s
|
||||
sleep 1
|
||||
$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_1"
|
||||
$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_1" && echo "dropped"
|
||||
|
||||
wait # for INSERT
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user