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>
|
2019-01-18 16:30:35 +00:00
|
|
|
#include <memory>
|
|
|
|
#include <unordered_map>
|
2022-07-19 10:28:21 +00:00
|
|
|
#include <vector>
|
2023-06-16 19:38:50 +00:00
|
|
|
#include <future>
|
2022-07-18 15:53:30 +00:00
|
|
|
#include <Storages/IStorage_fwd.h>
|
2023-06-16 19:38:50 +00:00
|
|
|
#include <Interpreters/Context_fwd.h>
|
2023-06-22 14:23:04 +00:00
|
|
|
#include <Interpreters/SetKeys.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>;
|
2023-06-22 14:23:04 +00:00
|
|
|
struct SetKeyColumns;
|
2022-07-18 15:53:30 +00:00
|
|
|
|
2023-06-09 17:51:59 +00:00
|
|
|
class IQueryTreeNode;
|
|
|
|
using QueryTreeNodePtr = std::shared_ptr<IQueryTreeNode>;
|
|
|
|
|
2023-06-16 19:38:50 +00:00
|
|
|
struct Settings;
|
|
|
|
|
2023-06-21 20:55:27 +00:00
|
|
|
/// This is a structure for prepared sets cache.
|
|
|
|
/// SetPtr can be taken from cache, so we should pass holder for it.
|
|
|
|
struct SetAndKey
|
|
|
|
{
|
|
|
|
String key;
|
|
|
|
SetPtr set;
|
|
|
|
};
|
|
|
|
|
|
|
|
using SetAndKeyPtr = std::shared_ptr<SetAndKey>;
|
|
|
|
|
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.
|
2023-06-16 19:38:50 +00:00
|
|
|
/// Set may be useful for indexes, in this case special ordered set with stored elements is build inplace.
|
2023-04-24 19:35:49 +00:00
|
|
|
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-06-22 14:23:04 +00:00
|
|
|
/// Returns set if set is ready (created and filled) or nullptr if not.
|
2023-04-24 19:35:49 +00:00
|
|
|
virtual SetPtr get() const = 0;
|
2023-06-22 14:23:04 +00:00
|
|
|
/// Returns set->getElementsTypes(), even if set is not created yet.
|
2023-09-12 18:26:53 +00:00
|
|
|
virtual DataTypes getTypes() const = 0;
|
2023-06-22 14:23:04 +00:00
|
|
|
/// If possible, return set with stored elements useful for PK analysis.
|
2023-06-21 20:55:27 +00:00
|
|
|
virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0;
|
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-06-22 14:23:04 +00:00
|
|
|
/// Future set from already filled set.
|
|
|
|
/// Usually it is from StorageSet.
|
|
|
|
class FutureSetFromStorage final : public FutureSet
|
2023-04-24 19:35:49 +00:00
|
|
|
{
|
|
|
|
public:
|
2023-12-22 17:26:31 +00:00
|
|
|
explicit FutureSetFromStorage(SetPtr set_);
|
2023-05-23 18:36:02 +00:00
|
|
|
|
2023-06-16 19:38:50 +00:00
|
|
|
SetPtr get() const override;
|
2023-09-12 18:26:53 +00:00
|
|
|
DataTypes getTypes() const override;
|
2023-06-21 20:55:27 +00:00
|
|
|
SetPtr buildOrderedSetInplace(const ContextPtr &) override;
|
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-04-04 10:01:01 +00:00
|
|
|
};
|
|
|
|
|
2024-01-05 10:53:07 +00:00
|
|
|
using FutureSetFromStoragePtr = std::shared_ptr<FutureSetFromStorage>;
|
|
|
|
|
2023-06-22 14:23:04 +00:00
|
|
|
/// Set from tuple is filled as well as set from storage.
|
|
|
|
/// Additionally, it can be converted to set useful for PK.
|
|
|
|
class FutureSetFromTuple final : public FutureSet
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
FutureSetFromTuple(Block block, const Settings & settings);
|
2023-06-16 19:38:50 +00:00
|
|
|
|
2023-06-22 14:23:04 +00:00
|
|
|
SetPtr get() const override { return set; }
|
|
|
|
SetPtr buildOrderedSetInplace(const ContextPtr & context) override;
|
2022-07-18 15:53:30 +00:00
|
|
|
|
2023-09-12 18:26:53 +00:00
|
|
|
DataTypes getTypes() const override;
|
2022-07-19 10:28:21 +00:00
|
|
|
|
2023-06-22 14:23:04 +00:00
|
|
|
private:
|
|
|
|
SetPtr set;
|
|
|
|
SetKeyColumns set_key_columns;
|
|
|
|
};
|
2022-07-18 15:53:30 +00:00
|
|
|
|
2024-01-05 10:53:07 +00:00
|
|
|
using FutureSetFromTuplePtr = std::shared_ptr<FutureSetFromTuple>;
|
|
|
|
|
2023-06-22 14:23:04 +00:00
|
|
|
/// Set from subquery can be built inplace for PK or in CreatingSet step.
|
|
|
|
/// If use_index_for_in_with_subqueries_max_values is reached, set for PK won't be created,
|
|
|
|
/// but ordinary set would be created instead.
|
|
|
|
class FutureSetFromSubquery final : public FutureSet
|
2023-04-24 19:35:49 +00:00
|
|
|
{
|
|
|
|
public:
|
2023-06-21 20:55:27 +00:00
|
|
|
FutureSetFromSubquery(
|
|
|
|
String key,
|
|
|
|
std::unique_ptr<QueryPlan> source_,
|
|
|
|
StoragePtr external_table_,
|
2024-01-05 10:53:07 +00:00
|
|
|
std::shared_ptr<FutureSetFromSubquery> external_table_set_,
|
2024-01-26 13:34:26 +00:00
|
|
|
const Settings & settings);
|
2023-06-21 20:55:27 +00:00
|
|
|
|
|
|
|
FutureSetFromSubquery(
|
|
|
|
String key,
|
|
|
|
QueryTreeNodePtr query_tree_,
|
|
|
|
const Settings & settings);
|
2023-04-24 19:35:49 +00:00
|
|
|
|
2023-06-19 12:56:24 +00:00
|
|
|
SetPtr get() const override;
|
2023-09-12 18:26:53 +00:00
|
|
|
DataTypes getTypes() const override;
|
2023-05-25 19:18:11 +00:00
|
|
|
SetPtr buildOrderedSetInplace(const ContextPtr & context) override;
|
2023-04-24 19:35:49 +00:00
|
|
|
|
2023-06-16 19:38:50 +00:00
|
|
|
std::unique_ptr<QueryPlan> build(const ContextPtr & context);
|
2024-01-05 10:53:07 +00:00
|
|
|
void buildSetInplace(const ContextPtr & context);
|
2023-04-24 19:35:49 +00:00
|
|
|
|
2023-06-21 20:55:27 +00:00
|
|
|
QueryTreeNodePtr detachQueryTree() { return std::move(query_tree); }
|
|
|
|
void setQueryPlan(std::unique_ptr<QueryPlan> source_);
|
2023-06-09 17:51:59 +00:00
|
|
|
|
2023-04-24 19:35:49 +00:00
|
|
|
private:
|
2023-06-21 20:55:27 +00:00
|
|
|
SetAndKeyPtr set_and_key;
|
|
|
|
StoragePtr external_table;
|
2024-01-05 10:53:07 +00:00
|
|
|
std::shared_ptr<FutureSetFromSubquery> external_table_set;
|
2023-04-24 19:35:49 +00:00
|
|
|
|
2023-06-21 20:55:27 +00:00
|
|
|
std::unique_ptr<QueryPlan> source;
|
|
|
|
QueryTreeNodePtr query_tree;
|
2023-04-24 19:35:49 +00:00
|
|
|
};
|
|
|
|
|
2024-01-05 10:53:07 +00:00
|
|
|
using FutureSetFromSubqueryPtr = std::shared_ptr<FutureSetFromSubquery>;
|
|
|
|
|
2023-06-22 14:23:04 +00:00
|
|
|
/// Container for all the sets used in query.
|
2022-07-18 15:53:30 +00:00
|
|
|
class PreparedSets
|
|
|
|
{
|
|
|
|
public:
|
2023-06-19 19:01:56 +00:00
|
|
|
|
2023-07-06 00:35:44 +00:00
|
|
|
using Hash = CityHash_v1_0_2::uint128;
|
2023-06-19 19:01:56 +00:00
|
|
|
struct Hashing
|
2023-05-25 19:18:11 +00:00
|
|
|
{
|
2023-07-06 00:35:44 +00:00
|
|
|
UInt64 operator()(const Hash & key) const { return key.low64 ^ key.high64; }
|
2023-05-25 19:18:11 +00:00
|
|
|
};
|
2023-04-25 18:14:08 +00:00
|
|
|
|
2024-01-05 10:53:07 +00:00
|
|
|
using SetsFromTuple = std::unordered_map<Hash, std::vector<FutureSetFromTuplePtr>, Hashing>;
|
|
|
|
using SetsFromStorage = std::unordered_map<Hash, FutureSetFromStoragePtr, Hashing>;
|
|
|
|
using SetsFromSubqueries = std::unordered_map<Hash, FutureSetFromSubqueryPtr, Hashing>;
|
2023-06-19 19:01:56 +00:00
|
|
|
|
2024-01-05 10:53:07 +00:00
|
|
|
FutureSetFromStoragePtr addFromStorage(const Hash & key, SetPtr set_);
|
|
|
|
FutureSetFromTuplePtr addFromTuple(const Hash & key, Block block, const Settings & settings);
|
2023-06-21 20:55:27 +00:00
|
|
|
|
2024-01-05 10:53:07 +00:00
|
|
|
FutureSetFromSubqueryPtr addFromSubquery(
|
2023-06-21 20:55:27 +00:00
|
|
|
const Hash & key,
|
|
|
|
std::unique_ptr<QueryPlan> source,
|
|
|
|
StoragePtr external_table,
|
2024-01-05 10:53:07 +00:00
|
|
|
FutureSetFromSubqueryPtr external_table_set,
|
2024-01-26 13:34:26 +00:00
|
|
|
const Settings & settings);
|
2023-06-21 20:55:27 +00:00
|
|
|
|
2024-01-05 10:53:07 +00:00
|
|
|
FutureSetFromSubqueryPtr addFromSubquery(
|
2023-06-21 20:55:27 +00:00
|
|
|
const Hash & key,
|
|
|
|
QueryTreeNodePtr query_tree,
|
|
|
|
const Settings & settings);
|
2022-07-18 15:53:30 +00:00
|
|
|
|
2024-01-05 10:53:07 +00:00
|
|
|
FutureSetFromTuplePtr findTuple(const Hash & key, const DataTypes & types) const;
|
|
|
|
FutureSetFromStoragePtr findStorage(const Hash & key) const;
|
|
|
|
FutureSetFromSubqueryPtr findSubquery(const Hash & key) const;
|
2023-06-19 19:01:56 +00:00
|
|
|
|
2024-01-05 10:53:07 +00:00
|
|
|
using Subqueries = std::vector<FutureSetFromSubqueryPtr>;
|
2023-12-21 16:21:02 +00:00
|
|
|
Subqueries getSubqueries() const;
|
|
|
|
bool hasSubqueries() const { return !sets_from_subqueries.empty(); }
|
2022-07-18 15:53:30 +00:00
|
|
|
|
2023-06-20 17:33:09 +00:00
|
|
|
const SetsFromTuple & getSetsFromTuple() const { return sets_from_tuple; }
|
2023-06-22 14:23:04 +00:00
|
|
|
// const SetsFromStorage & getSetsFromStorage() const { return sets_from_storage; }
|
|
|
|
// const SetsFromSubqueries & getSetsFromSubquery() const { return sets_from_subqueries; }
|
2022-07-18 15:53:30 +00:00
|
|
|
|
2023-06-19 19:01:56 +00:00
|
|
|
static String toString(const Hash & key, const DataTypes & types);
|
2022-07-18 15:53:30 +00:00
|
|
|
|
2023-06-19 19:01:56 +00:00
|
|
|
private:
|
2023-06-20 17:33:09 +00:00
|
|
|
SetsFromTuple sets_from_tuple;
|
|
|
|
SetsFromStorage sets_from_storage;
|
2023-06-19 19:01:56 +00:00
|
|
|
SetsFromSubqueries sets_from_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
|
|
|
}
|