2019-01-18 16:30:35 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <Parsers/IAST.h>
|
2019-01-22 12:33:56 +00:00
|
|
|
#include <DataTypes/IDataType.h>
|
2023-03-02 19:19:58 +00:00
|
|
|
#include <future>
|
2019-01-18 16:30:35 +00:00
|
|
|
#include <memory>
|
|
|
|
#include <unordered_map>
|
2022-07-19 10:28:21 +00:00
|
|
|
#include <vector>
|
2019-04-30 10:34:12 +00:00
|
|
|
#include <DataTypes/DataTypeLowCardinality.h>
|
2022-07-18 15:53:30 +00:00
|
|
|
#include <Storages/IStorage_fwd.h>
|
2022-07-27 11:22:16 +00:00
|
|
|
#include <QueryPipeline/SizeLimits.h>
|
|
|
|
#include <Processors/QueryPlan/QueryPlan.h>
|
2023-04-24 19:35:49 +00:00
|
|
|
#include "Core/Block.h"
|
|
|
|
#include "Interpreters/Context.h"
|
|
|
|
#include "Interpreters/Set.h"
|
|
|
|
#include "Processors/Executors/CompletedPipelineExecutor.h"
|
|
|
|
#include "Processors/QueryPlan/BuildQueryPipelineSettings.h"
|
|
|
|
#include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h"
|
2023-05-05 18:30:08 +00:00
|
|
|
#include "Processors/Sinks/EmptySink.h"
|
2023-04-24 19:35:49 +00:00
|
|
|
#include "Processors/Sinks/NullSink.h"
|
|
|
|
#include <QueryPipeline/QueryPipelineBuilder.h>
|
2020-10-26 19:12:40 +00:00
|
|
|
|
2019-01-18 16:30:35 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2022-07-18 15:53:30 +00:00
|
|
|
class QueryPlan;
|
|
|
|
|
|
|
|
class Set;
|
|
|
|
using SetPtr = std::shared_ptr<Set>;
|
2022-07-19 10:28:21 +00:00
|
|
|
class InterpreterSelectWithUnionQuery;
|
2022-07-18 15:53:30 +00:00
|
|
|
|
2023-04-05 18:24:38 +00:00
|
|
|
/// Represents a set in a query that might be referenced at analysis time and built later during execution.
|
|
|
|
/// Also it can represent a constant set that is ready to use.
|
|
|
|
/// At analysis stage the FutureSets are created but not necessarily filled. Then for non-constant sets there
|
|
|
|
/// must be an explicit step to build them before they can be used.
|
|
|
|
/// FutureSet objects can be stored in PreparedSets and are not intended to be used from multiple threads.
|
2023-04-24 19:35:49 +00:00
|
|
|
// class FutureSet final
|
|
|
|
// {
|
|
|
|
// public:
|
|
|
|
// FutureSet() = default;
|
|
|
|
|
|
|
|
// /// Create FutureSet from an object that will be created in the future.
|
|
|
|
// explicit FutureSet(const std::shared_future<SetPtr> & future_set_) : future_set(future_set_) {}
|
|
|
|
|
|
|
|
// /// Create FutureSet from a ready set.
|
|
|
|
// explicit FutureSet(SetPtr readySet);
|
|
|
|
|
|
|
|
// /// The set object will be ready in the future, as opposed to 'null' object when FutureSet is default constructed.
|
|
|
|
// bool isValid() const { return future_set.valid(); }
|
|
|
|
|
|
|
|
// /// The the value of SetPtr is ready, but the set object might not have been filled yet.
|
|
|
|
// bool isReady() const;
|
|
|
|
|
|
|
|
// /// The set object is ready and filled.
|
|
|
|
// bool isCreated() const;
|
|
|
|
|
|
|
|
// SetPtr get() const { chassert(isReady()); return future_set.get(); }
|
|
|
|
|
|
|
|
// private:
|
|
|
|
// std::shared_future<SetPtr> future_set;
|
|
|
|
// };
|
|
|
|
|
|
|
|
class FutureSet
|
2023-04-04 10:01:01 +00:00
|
|
|
{
|
2023-04-04 21:47:05 +00:00
|
|
|
public:
|
2023-04-24 19:35:49 +00:00
|
|
|
virtual ~FutureSet() = default;
|
2023-04-05 18:24:38 +00:00
|
|
|
|
2023-04-24 19:35:49 +00:00
|
|
|
virtual bool isReady() const = 0;
|
2023-05-04 17:54:08 +00:00
|
|
|
virtual bool isFilled() const = 0;
|
2023-04-24 19:35:49 +00:00
|
|
|
virtual SetPtr get() const = 0;
|
2023-04-05 18:24:38 +00:00
|
|
|
|
2023-04-24 19:35:49 +00:00
|
|
|
virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0;
|
|
|
|
virtual std::unique_ptr<QueryPlan> build(const ContextPtr & context) = 0;
|
2023-04-25 18:14:08 +00:00
|
|
|
|
|
|
|
static SizeLimits getSizeLimitsForSet(const Settings & settings, bool ordered_set);
|
2023-04-24 19:35:49 +00:00
|
|
|
};
|
2023-04-04 10:01:01 +00:00
|
|
|
|
2023-04-25 18:14:08 +00:00
|
|
|
using FutureSetPtr = std::shared_ptr<FutureSet>;
|
2023-04-05 18:24:38 +00:00
|
|
|
|
2023-04-24 19:35:49 +00:00
|
|
|
class FutureSetFromTuple final : public FutureSet
|
|
|
|
{
|
|
|
|
public:
|
2023-05-25 13:33:52 +00:00
|
|
|
FutureSetFromTuple(Block block, const Settings & settings);
|
2023-04-04 10:01:01 +00:00
|
|
|
|
2023-05-25 13:33:52 +00:00
|
|
|
bool isReady() const override { return true; }
|
2023-05-04 17:54:08 +00:00
|
|
|
bool isFilled() const override { return true; }
|
2023-04-24 19:35:49 +00:00
|
|
|
SetPtr get() const override { return set; }
|
2023-04-05 18:24:38 +00:00
|
|
|
|
2023-05-23 18:36:02 +00:00
|
|
|
SetPtr buildOrderedSetInplace(const ContextPtr & context) override;
|
|
|
|
|
2023-05-25 13:33:52 +00:00
|
|
|
std::unique_ptr<QueryPlan> build(const ContextPtr &) override;
|
2023-05-23 18:36:02 +00:00
|
|
|
|
2023-05-25 13:33:52 +00:00
|
|
|
/// void buildForTuple(SizeLimits size_limits, bool transform_null_in);
|
2023-05-04 17:54:08 +00:00
|
|
|
|
2023-04-05 18:24:38 +00:00
|
|
|
private:
|
2023-04-24 19:35:49 +00:00
|
|
|
SetPtr set;
|
2023-05-25 13:33:52 +00:00
|
|
|
Set::SetKeyColumns set_key_columns;
|
2023-04-24 19:35:49 +00:00
|
|
|
|
2023-05-25 13:33:52 +00:00
|
|
|
//void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set);
|
2023-04-04 10:01:01 +00:00
|
|
|
};
|
|
|
|
|
2022-07-18 15:53:30 +00:00
|
|
|
/// Information on how to build set for the [GLOBAL] IN section.
|
2022-07-19 10:28:21 +00:00
|
|
|
class SubqueryForSet
|
2022-07-18 15:53:30 +00:00
|
|
|
{
|
2022-07-19 10:28:21 +00:00
|
|
|
public:
|
2022-07-18 15:53:30 +00:00
|
|
|
|
2022-07-27 11:22:16 +00:00
|
|
|
void createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_ = nullptr);
|
2022-07-19 10:28:21 +00:00
|
|
|
|
|
|
|
bool hasSource() const;
|
|
|
|
|
2022-07-27 11:22:16 +00:00
|
|
|
/// Returns query plan for the set's source
|
|
|
|
/// and removes it from SubqueryForSet because we need to build it only once.
|
|
|
|
std::unique_ptr<QueryPlan> detachSource();
|
2022-07-18 15:53:30 +00:00
|
|
|
|
|
|
|
/// Build this set from the result of the subquery.
|
2023-04-04 21:47:05 +00:00
|
|
|
String key;
|
2023-04-24 19:35:49 +00:00
|
|
|
SetPtr set;
|
2023-04-13 16:24:57 +00:00
|
|
|
/// After set_in_progress is finished it will be put into promise_to_fill_set and thus all FutureSet's
|
|
|
|
/// that are referencing this set will be filled.
|
2023-04-24 19:35:49 +00:00
|
|
|
|
2023-04-04 10:01:01 +00:00
|
|
|
std::promise<SetPtr> promise_to_fill_set;
|
2023-04-24 19:35:49 +00:00
|
|
|
// FutureSet set = FutureSet{promise_to_fill_set.get_future()};
|
2022-07-18 15:53:30 +00:00
|
|
|
|
|
|
|
/// If set, put the result into the table.
|
|
|
|
/// This is a temporary table for transferring to remote servers for distributed query processing.
|
|
|
|
StoragePtr table;
|
2022-07-19 10:28:21 +00:00
|
|
|
|
|
|
|
/// The source is obtained using the InterpreterSelectQuery subquery.
|
|
|
|
std::unique_ptr<QueryPlan> source;
|
2022-07-18 15:53:30 +00:00
|
|
|
};
|
|
|
|
|
2023-04-24 19:35:49 +00:00
|
|
|
class FutureSetFromSubquery : public FutureSet
|
|
|
|
{
|
|
|
|
public:
|
2023-04-25 18:14:08 +00:00
|
|
|
FutureSetFromSubquery(SubqueryForSet subquery_);
|
2023-04-24 19:35:49 +00:00
|
|
|
|
|
|
|
bool isReady() const override { return set != nullptr; }
|
2023-05-04 17:54:08 +00:00
|
|
|
bool isFilled() const override { return isReady(); }
|
2023-04-24 19:35:49 +00:00
|
|
|
SetPtr get() const override { return set; }
|
|
|
|
|
|
|
|
SetPtr buildOrderedSetInplace(const ContextPtr & context) override
|
|
|
|
{
|
2023-04-25 18:14:08 +00:00
|
|
|
if (!context->getSettingsRef().use_index_for_in_with_subqueries)
|
|
|
|
return nullptr;
|
|
|
|
|
2023-04-24 19:35:49 +00:00
|
|
|
auto plan = buildPlan(context, true);
|
2023-05-25 13:33:52 +00:00
|
|
|
if (!plan)
|
|
|
|
return nullptr;
|
2023-04-24 19:35:49 +00:00
|
|
|
|
|
|
|
auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
|
|
|
|
auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
|
2023-05-05 18:30:08 +00:00
|
|
|
pipeline.complete(std::make_shared<EmptySink>(Block()));
|
2023-04-24 19:35:49 +00:00
|
|
|
|
|
|
|
CompletedPipelineExecutor executor(pipeline);
|
|
|
|
executor.execute();
|
|
|
|
|
|
|
|
return set;
|
|
|
|
}
|
|
|
|
|
|
|
|
std::unique_ptr<QueryPlan> build(const ContextPtr & context) override
|
|
|
|
{
|
|
|
|
return buildPlan(context, false);
|
|
|
|
}
|
|
|
|
|
2023-04-25 18:14:08 +00:00
|
|
|
void addStorage(StoragePtr storage) { subquery.table = std::move(storage); }
|
|
|
|
|
2023-04-24 19:35:49 +00:00
|
|
|
private:
|
|
|
|
SetPtr set;
|
|
|
|
SubqueryForSet subquery;
|
|
|
|
|
|
|
|
std::unique_ptr<QueryPlan> buildPlan(const ContextPtr & context, bool create_ordered_set);
|
|
|
|
};
|
|
|
|
|
2023-04-25 18:14:08 +00:00
|
|
|
class FutureSetFromStorage : public FutureSet
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
FutureSetFromStorage(SetPtr set_); // : set(std::move(set_) {}
|
|
|
|
|
|
|
|
bool isReady() const override { return set != nullptr; }
|
2023-05-04 17:54:08 +00:00
|
|
|
bool isFilled() const override { return isReady(); }
|
2023-04-25 18:14:08 +00:00
|
|
|
SetPtr get() const override { return set; }
|
|
|
|
|
|
|
|
SetPtr buildOrderedSetInplace(const ContextPtr &) override
|
|
|
|
{
|
|
|
|
return set->hasExplicitSetElements() ? set : nullptr;
|
|
|
|
}
|
|
|
|
|
|
|
|
std::unique_ptr<QueryPlan> build(const ContextPtr &) override { return nullptr; }
|
|
|
|
|
|
|
|
private:
|
|
|
|
SetPtr set;
|
|
|
|
};
|
|
|
|
|
2023-04-24 19:35:49 +00:00
|
|
|
// class FutureSetFromFuture : public FutureSet
|
|
|
|
// {
|
|
|
|
// public:
|
|
|
|
// FutureSetFromFuture(std::shared_future<SetPtr> future_set_);
|
|
|
|
|
|
|
|
// bool isReady() const override { return future_set.wait_for(std::chrono::seconds(0)) == std::future_status::ready; }
|
|
|
|
// SetPtr get() const override { return future_set.get(); }
|
|
|
|
|
|
|
|
// SetPtr buildOrderedSetInplace(const ContextPtr &) override
|
|
|
|
// {
|
|
|
|
// fill(true);
|
|
|
|
// return set;
|
|
|
|
// }
|
|
|
|
|
|
|
|
// std::unique_ptr<QueryPlan> build(const ContextPtr &) override
|
|
|
|
// {
|
|
|
|
// fill(false);
|
|
|
|
// return nullptr;
|
|
|
|
// }
|
|
|
|
|
|
|
|
// private:
|
|
|
|
// std::shared_future<SetPtr> future_set;
|
|
|
|
// }
|
|
|
|
|
2023-04-04 21:47:05 +00:00
|
|
|
struct PreparedSetKey
|
|
|
|
{
|
2023-05-04 17:54:08 +00:00
|
|
|
using Hash = std::pair<UInt64, UInt64>;
|
|
|
|
|
2023-04-04 21:47:05 +00:00
|
|
|
/// Prepared sets for tuple literals are indexed by the hash of the tree contents and by the desired
|
|
|
|
/// data types of set elements (two different Sets can be required for two tuples with the same contents
|
|
|
|
/// if left hand sides of the IN operators have different types).
|
2023-05-04 17:54:08 +00:00
|
|
|
static PreparedSetKey forLiteral(Hash hash, DataTypes types_);
|
2023-04-04 21:47:05 +00:00
|
|
|
|
|
|
|
/// Prepared sets for subqueries are indexed only by the AST contents because the type of the resulting
|
|
|
|
/// set is fully determined by the subquery.
|
2023-05-04 17:54:08 +00:00
|
|
|
static PreparedSetKey forSubquery(Hash hash);
|
2023-04-04 21:47:05 +00:00
|
|
|
|
2023-05-04 17:54:08 +00:00
|
|
|
Hash ast_hash;
|
2023-04-04 21:47:05 +00:00
|
|
|
DataTypes types; /// Empty for subqueries.
|
|
|
|
|
|
|
|
bool operator==(const PreparedSetKey & other) const;
|
|
|
|
|
2023-04-05 18:24:38 +00:00
|
|
|
String toString() const;
|
|
|
|
|
2023-05-04 17:54:08 +00:00
|
|
|
struct Hashing
|
2023-04-04 21:47:05 +00:00
|
|
|
{
|
|
|
|
UInt64 operator()(const PreparedSetKey & key) const { return key.ast_hash.first; }
|
|
|
|
};
|
|
|
|
};
|
|
|
|
|
2022-07-18 15:53:30 +00:00
|
|
|
class PreparedSets
|
|
|
|
{
|
|
|
|
public:
|
2023-04-25 18:14:08 +00:00
|
|
|
using SubqueriesForSets = std::unordered_map<String, std::shared_ptr<FutureSetFromSubquery>>;
|
|
|
|
|
|
|
|
// SubqueryForSet & createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key,
|
|
|
|
// SizeLimits set_size_limit, bool transform_null_in);
|
|
|
|
|
|
|
|
FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_);
|
2023-05-25 13:33:52 +00:00
|
|
|
FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings);
|
2023-04-25 18:14:08 +00:00
|
|
|
FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery);
|
2022-07-18 15:53:30 +00:00
|
|
|
|
2023-04-25 18:14:08 +00:00
|
|
|
void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage);
|
2022-07-18 15:53:30 +00:00
|
|
|
|
2023-04-25 18:14:08 +00:00
|
|
|
FutureSetPtr getFuture(const PreparedSetKey & key) const;
|
|
|
|
//SubqueryForSet & getSubquery(const String & subquery_id);
|
|
|
|
// SetPtr get(const PreparedSetKey & key) const;
|
2022-07-18 15:53:30 +00:00
|
|
|
|
2022-07-27 11:22:16 +00:00
|
|
|
/// Get subqueries and clear them.
|
|
|
|
/// We need to build a plan for subqueries just once. That's why we can clear them after accessing them.
|
|
|
|
/// SetPtr would still be available for consumers of PreparedSets.
|
2023-05-04 17:54:08 +00:00
|
|
|
SubqueriesForSets detachSubqueries(const ContextPtr &);
|
2022-07-18 15:53:30 +00:00
|
|
|
|
2022-07-19 10:28:21 +00:00
|
|
|
/// Returns all sets that match the given ast hash not checking types
|
2022-07-18 15:53:30 +00:00
|
|
|
/// Used in KeyCondition and MergeTreeIndexConditionBloomFilter to make non exact match for types in PreparedSetKey
|
2023-05-04 17:54:08 +00:00
|
|
|
//std::vector<FutureSetPtr> getByTreeHash(IAST::Hash ast_hash) const;
|
|
|
|
|
|
|
|
const std::unordered_map<PreparedSetKey, FutureSetPtr, PreparedSetKey::Hashing> & getSets() const { return sets; }
|
2022-07-18 15:53:30 +00:00
|
|
|
|
|
|
|
bool empty() const;
|
|
|
|
|
|
|
|
private:
|
2023-05-04 17:54:08 +00:00
|
|
|
std::unordered_map<PreparedSetKey, FutureSetPtr, PreparedSetKey::Hashing> sets;
|
2022-07-18 15:53:30 +00:00
|
|
|
|
|
|
|
/// This is the information required for building sets
|
2023-04-25 18:14:08 +00:00
|
|
|
SubqueriesForSets subqueries;
|
2022-07-18 15:53:30 +00:00
|
|
|
};
|
2019-01-18 16:30:35 +00:00
|
|
|
|
2022-07-18 15:53:30 +00:00
|
|
|
using PreparedSetsPtr = std::shared_ptr<PreparedSets>;
|
2019-01-18 16:30:35 +00:00
|
|
|
|
2023-04-05 18:24:38 +00:00
|
|
|
/// A reference to a set that is being built by another task.
|
|
|
|
/// The difference from FutureSet is that this object can be used to wait for the set to be built in another thread.
|
|
|
|
using SharedSet = std::shared_future<SetPtr>;
|
|
|
|
|
2023-03-02 19:19:58 +00:00
|
|
|
/// This set cache is used to avoid building the same set multiple times. It is different from PreparedSets in way that
|
|
|
|
/// it can be used across multiple queries. One use case is when we execute the same mutation on multiple parts. In this
|
|
|
|
/// case each part is processed by a separate mutation task but they can share the same set.
|
|
|
|
class PreparedSetsCache
|
|
|
|
{
|
|
|
|
public:
|
2023-04-11 21:19:44 +00:00
|
|
|
/// Lookup for set in the cache.
|
|
|
|
/// If it is found, get the future to be able to wait for the set to be built.
|
|
|
|
/// Otherwise create a promise, build the set and set the promise value.
|
2023-04-05 18:24:38 +00:00
|
|
|
std::variant<std::promise<SetPtr>, SharedSet> findOrPromiseToBuild(const String & key);
|
2023-03-02 19:19:58 +00:00
|
|
|
|
|
|
|
private:
|
|
|
|
struct Entry
|
|
|
|
{
|
2023-04-05 18:24:38 +00:00
|
|
|
SharedSet future; /// Other tasks can wait for the set to be built.
|
2023-03-02 19:19:58 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
std::mutex cache_mutex;
|
2023-04-05 18:24:38 +00:00
|
|
|
std::unordered_map<String, Entry> cache;
|
2023-03-02 19:19:58 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
using PreparedSetsCachePtr = std::shared_ptr<PreparedSetsCache>;
|
|
|
|
|
2019-01-18 16:30:35 +00:00
|
|
|
}
|