ClickHouse/src/Interpreters/PreparedSets.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

197 lines
6.1 KiB
C++
Raw Normal View History

2019-01-18 16:30:35 +00:00
#pragma once
#include <Parsers/IAST.h>
#include <DataTypes/IDataType.h>
2019-01-18 16:30:35 +00:00
#include <memory>
#include <unordered_map>
#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>
#include <Core/Settings.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>;
class InterpreterSelectWithUnionQuery;
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;
/// 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-24 19:35:49 +00:00
virtual SetPtr get() const = 0;
virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0;
2023-04-25 18:14:08 +00:00
2023-06-01 21:15:15 +00:00
virtual DataTypes getTypes() const = 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-06-16 19:38:50 +00:00
class FutureSetFromStorage : public FutureSet
2023-04-24 19:35:49 +00:00
{
public:
2023-06-16 19:38:50 +00:00
FutureSetFromStorage(SetPtr set_);
2023-05-23 18:36:02 +00:00
2023-06-16 19:38:50 +00:00
SetPtr get() const override;
SetPtr buildOrderedSetInplace(const ContextPtr &) override;
DataTypes getTypes() const override;
2023-05-04 17:54:08 +00:00
private:
2023-04-24 19:35:49 +00:00
SetPtr set;
2023-04-04 10:01:01 +00:00
};
2023-06-16 19:38:50 +00:00
2022-07-18 15:53:30 +00:00
/// Information on how to build set for the [GLOBAL] IN section.
class SubqueryForSet
2022-07-18 15:53:30 +00:00
{
public:
2022-07-18 15:53:30 +00:00
void createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_ = nullptr);
bool hasSource() const;
/// 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;
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;
/// The source is obtained using the InterpreterSelectQuery subquery.
std::unique_ptr<QueryPlan> source;
2023-06-09 17:51:59 +00:00
QueryTreeNodePtr query_tree;
2022-07-18 15:53:30 +00:00
};
2023-06-01 21:15:15 +00:00
class FutureSetFromSubquery : public FutureSet, public std::enable_shared_from_this<FutureSetFromSubquery>
2023-04-24 19:35:49 +00:00
{
public:
2023-06-01 21:15:15 +00:00
FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_);
2023-04-24 19:35:49 +00:00
2023-06-19 12:56:24 +00:00
SetPtr get() const override;
2023-04-24 19:35:49 +00:00
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);
2023-04-24 19:35:49 +00:00
2023-06-01 21:15:15 +00:00
DataTypes getTypes() const override;
2023-06-09 17:51:59 +00:00
SubqueryForSet & getSubquery() { return subquery; }
2023-04-24 19:35:49 +00:00
private:
SubqueryForSet subquery;
2023-05-25 19:18:11 +00:00
FutureSetPtr external_table_set;
2023-06-01 21:15:15 +00:00
bool transform_null_in;
2023-04-24 19:35:49 +00:00
std::unique_ptr<QueryPlan> buildPlan(const ContextPtr & context, bool create_ordered_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;
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-05-25 19:18:11 +00:00
struct SetAndName
{
String name;
std::shared_ptr<FutureSetFromSubquery> set;
};
using SubqueriesForSets = std::vector<SetAndName>;
2023-04-25 18:14:08 +00:00
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-06-01 21:15:15 +00:00
FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set);
2022-07-18 15:53:30 +00:00
2023-04-25 18:14:08 +00:00
FutureSetPtr getFuture(const PreparedSetKey & key) const;
2022-07-18 15:53:30 +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-25 19:18:11 +00:00
SubqueriesForSets detachSubqueries();
2022-07-18 15:53:30 +00:00
2023-05-04 17:54:08 +00:00
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
/// 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>;
/// 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.
std::variant<std::promise<SetPtr>, SharedSet> findOrPromiseToBuild(const String & key);
private:
struct Entry
{
SharedSet future; /// Other tasks can wait for the set to be built.
};
std::mutex cache_mutex;
std::unordered_map<String, Entry> cache;
};
using PreparedSetsCachePtr = std::shared_ptr<PreparedSetsCache>;
2019-01-18 16:30:35 +00:00
}