mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #3581 from yandex/alter-order-by
ALTER MODIFY ORDER BY
This commit is contained in:
commit
cec49357da
@ -125,12 +125,13 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
bool has_zookeeper = config().has("zookeeper");
|
||||
|
||||
zkutil::ZooKeeperNodeCache main_config_zk_node_cache([&] { return global_context->getZooKeeper(); });
|
||||
zkutil::EventPtr main_config_zk_changed_event = std::make_shared<Poco::Event>();
|
||||
if (loaded_config.has_zk_includes)
|
||||
{
|
||||
auto old_configuration = loaded_config.configuration;
|
||||
ConfigProcessor config_processor(config_path);
|
||||
loaded_config = config_processor.loadConfigWithZooKeeperIncludes(
|
||||
main_config_zk_node_cache, /* fallback_to_preprocessed = */ true);
|
||||
main_config_zk_node_cache, main_config_zk_changed_event, /* fallback_to_preprocessed = */ true);
|
||||
config_processor.savePreprocessedConfig(loaded_config, config().getString("path", DBMS_DEFAULT_PATH));
|
||||
config().removeConfiguration(old_configuration.get());
|
||||
config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false);
|
||||
@ -305,6 +306,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
include_from_path,
|
||||
config().getString("path", ""),
|
||||
std::move(main_config_zk_node_cache),
|
||||
main_config_zk_changed_event,
|
||||
[&](ConfigurationPtr config)
|
||||
{
|
||||
buildLoggers(*config);
|
||||
@ -327,6 +329,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
include_from_path,
|
||||
config().getString("path", ""),
|
||||
zkutil::ZooKeeperNodeCache([&] { return global_context->getZooKeeper(); }),
|
||||
std::make_shared<Poco::Event>(),
|
||||
[&](ConfigurationPtr config) { global_context->setUsersConfig(config); },
|
||||
/* already_loaded = */ false);
|
||||
|
||||
|
@ -231,6 +231,7 @@ void ConfigProcessor::doIncludesRecursive(
|
||||
XMLDocumentPtr include_from,
|
||||
Node * node,
|
||||
zkutil::ZooKeeperNodeCache * zk_node_cache,
|
||||
const zkutil::EventPtr & zk_changed_event,
|
||||
std::unordered_set<std::string> & contributing_zk_paths)
|
||||
{
|
||||
if (node->nodeType() == Node::TEXT_NODE)
|
||||
@ -349,12 +350,12 @@ void ConfigProcessor::doIncludesRecursive(
|
||||
XMLDocumentPtr zk_document;
|
||||
auto get_zk_node = [&](const std::string & name) -> const Node *
|
||||
{
|
||||
std::optional<std::string> contents = zk_node_cache->get(name);
|
||||
if (!contents)
|
||||
zkutil::ZooKeeperNodeCache::ZNode znode = zk_node_cache->get(name, zk_changed_event);
|
||||
if (!znode.exists)
|
||||
return nullptr;
|
||||
|
||||
/// Enclose contents into a fake <from_zk> tag to allow pure text substitutions.
|
||||
zk_document = dom_parser.parseString("<from_zk>" + *contents + "</from_zk>");
|
||||
zk_document = dom_parser.parseString("<from_zk>" + znode.contents + "</from_zk>");
|
||||
return getRootNode(zk_document.get());
|
||||
};
|
||||
|
||||
@ -380,13 +381,13 @@ void ConfigProcessor::doIncludesRecursive(
|
||||
}
|
||||
|
||||
if (included_something)
|
||||
doIncludesRecursive(config, include_from, node, zk_node_cache, contributing_zk_paths);
|
||||
doIncludesRecursive(config, include_from, node, zk_node_cache, zk_changed_event, contributing_zk_paths);
|
||||
else
|
||||
{
|
||||
NodeListPtr children = node->childNodes();
|
||||
Node * child = nullptr;
|
||||
for (size_t i = 0; (child = children->item(i)); ++i)
|
||||
doIncludesRecursive(config, include_from, child, zk_node_cache, contributing_zk_paths);
|
||||
doIncludesRecursive(config, include_from, child, zk_node_cache, zk_changed_event, contributing_zk_paths);
|
||||
}
|
||||
}
|
||||
|
||||
@ -430,7 +431,8 @@ ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string &
|
||||
|
||||
XMLDocumentPtr ConfigProcessor::processConfig(
|
||||
bool * has_zk_includes,
|
||||
zkutil::ZooKeeperNodeCache * zk_node_cache)
|
||||
zkutil::ZooKeeperNodeCache * zk_node_cache,
|
||||
const zkutil::EventPtr & zk_changed_event)
|
||||
{
|
||||
XMLDocumentPtr config = dom_parser.parse(path);
|
||||
|
||||
@ -460,7 +462,7 @@ XMLDocumentPtr ConfigProcessor::processConfig(
|
||||
if (node)
|
||||
{
|
||||
/// if we include_from env or zk.
|
||||
doIncludesRecursive(config, nullptr, node, zk_node_cache, contributing_zk_paths);
|
||||
doIncludesRecursive(config, nullptr, node, zk_node_cache, zk_changed_event, contributing_zk_paths);
|
||||
include_from_path = node->innerText();
|
||||
}
|
||||
else
|
||||
@ -475,7 +477,7 @@ XMLDocumentPtr ConfigProcessor::processConfig(
|
||||
include_from = dom_parser.parse(include_from_path);
|
||||
}
|
||||
|
||||
doIncludesRecursive(config, include_from, getRootNode(config.get()), zk_node_cache, contributing_zk_paths);
|
||||
doIncludesRecursive(config, include_from, getRootNode(config.get()), zk_node_cache, zk_changed_event, contributing_zk_paths);
|
||||
}
|
||||
catch (Poco::Exception & e)
|
||||
{
|
||||
@ -524,6 +526,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfig(bool allow_zk_includes
|
||||
|
||||
ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes(
|
||||
zkutil::ZooKeeperNodeCache & zk_node_cache,
|
||||
const zkutil::EventPtr & zk_changed_event,
|
||||
bool fallback_to_preprocessed)
|
||||
{
|
||||
XMLDocumentPtr config_xml;
|
||||
@ -531,7 +534,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes(
|
||||
bool processed_successfully = false;
|
||||
try
|
||||
{
|
||||
config_xml = processConfig(&has_zk_includes, &zk_node_cache);
|
||||
config_xml = processConfig(&has_zk_includes, &zk_node_cache, zk_changed_event);
|
||||
processed_successfully = true;
|
||||
}
|
||||
catch (const Poco::Exception & ex)
|
||||
|
@ -22,6 +22,7 @@
|
||||
namespace zkutil
|
||||
{
|
||||
class ZooKeeperNodeCache;
|
||||
using EventPtr = std::shared_ptr<Poco::Event>;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
@ -61,7 +62,8 @@ public:
|
||||
/// 5) (Yandex.Metrika-specific) Substitute "<layer/>" with "<layer>layer number from the hostname</layer>".
|
||||
XMLDocumentPtr processConfig(
|
||||
bool * has_zk_includes = nullptr,
|
||||
zkutil::ZooKeeperNodeCache * zk_node_cache = nullptr);
|
||||
zkutil::ZooKeeperNodeCache * zk_node_cache = nullptr,
|
||||
const zkutil::EventPtr & zk_changed_event = nullptr);
|
||||
|
||||
|
||||
/// loadConfig* functions apply processConfig and create Poco::Util::XMLConfiguration.
|
||||
@ -87,6 +89,7 @@ public:
|
||||
/// processing, load the configuration from the preprocessed file.
|
||||
LoadedConfig loadConfigWithZooKeeperIncludes(
|
||||
zkutil::ZooKeeperNodeCache & zk_node_cache,
|
||||
const zkutil::EventPtr & zk_changed_event,
|
||||
bool fallback_to_preprocessed = false);
|
||||
|
||||
/// Save preprocessed config to specified directory.
|
||||
@ -134,6 +137,7 @@ private:
|
||||
XMLDocumentPtr include_from,
|
||||
Poco::XML::Node * node,
|
||||
zkutil::ZooKeeperNodeCache * zk_node_cache,
|
||||
const zkutil::EventPtr & zk_changed_event,
|
||||
std::unordered_set<std::string> & contributing_zk_paths);
|
||||
};
|
||||
|
||||
|
@ -17,11 +17,13 @@ ConfigReloader::ConfigReloader(
|
||||
const std::string & include_from_path_,
|
||||
const std::string & preprocessed_dir_,
|
||||
zkutil::ZooKeeperNodeCache && zk_node_cache_,
|
||||
const zkutil::EventPtr & zk_changed_event_,
|
||||
Updater && updater_,
|
||||
bool already_loaded)
|
||||
: path(path_), include_from_path(include_from_path_)
|
||||
, preprocessed_dir(preprocessed_dir_)
|
||||
, zk_node_cache(std::move(zk_node_cache_))
|
||||
, zk_changed_event(zk_changed_event_)
|
||||
, updater(std::move(updater_))
|
||||
{
|
||||
if (!already_loaded)
|
||||
@ -40,7 +42,7 @@ ConfigReloader::~ConfigReloader()
|
||||
try
|
||||
{
|
||||
quit = true;
|
||||
zk_node_cache.getChangedEvent().set();
|
||||
zk_changed_event->set();
|
||||
|
||||
if (thread.joinable())
|
||||
thread.join();
|
||||
@ -60,7 +62,7 @@ void ConfigReloader::run()
|
||||
{
|
||||
try
|
||||
{
|
||||
bool zk_changed = zk_node_cache.getChangedEvent().tryWait(std::chrono::milliseconds(reload_interval).count());
|
||||
bool zk_changed = zk_changed_event->tryWait(std::chrono::milliseconds(reload_interval).count());
|
||||
if (quit)
|
||||
return;
|
||||
|
||||
@ -90,7 +92,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac
|
||||
loaded_config = config_processor.loadConfig(/* allow_zk_includes = */ true);
|
||||
if (loaded_config.has_zk_includes)
|
||||
loaded_config = config_processor.loadConfigWithZooKeeperIncludes(
|
||||
zk_node_cache, fallback_to_preprocessed);
|
||||
zk_node_cache, zk_changed_event, fallback_to_preprocessed);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -35,6 +35,7 @@ public:
|
||||
const std::string & include_from_path,
|
||||
const std::string & preprocessed_dir,
|
||||
zkutil::ZooKeeperNodeCache && zk_node_cache,
|
||||
const zkutil::EventPtr & zk_changed_event,
|
||||
Updater && updater,
|
||||
bool already_loaded);
|
||||
|
||||
@ -74,6 +75,7 @@ private:
|
||||
std::string preprocessed_dir;
|
||||
FilesChangesTracker files;
|
||||
zkutil::ZooKeeperNodeCache zk_node_cache;
|
||||
zkutil::EventPtr zk_changed_event = std::make_shared<Poco::Event>();
|
||||
|
||||
Updater updater;
|
||||
|
||||
|
@ -9,7 +9,16 @@ ZooKeeperNodeCache::ZooKeeperNodeCache(GetZooKeeper get_zookeeper_)
|
||||
{
|
||||
}
|
||||
|
||||
std::optional<std::string> ZooKeeperNodeCache::get(const std::string & path)
|
||||
ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, EventPtr watch_event)
|
||||
{
|
||||
Coordination::WatchCallback watch_callback;
|
||||
if (watch_event)
|
||||
watch_callback = [watch_event](const Coordination::WatchResponse &) { watch_event->set(); };
|
||||
|
||||
return get(path, watch_callback);
|
||||
}
|
||||
|
||||
ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, Coordination::WatchCallback caller_watch_callback)
|
||||
{
|
||||
zkutil::ZooKeeperPtr zookeeper;
|
||||
std::unordered_set<std::string> invalidated_paths;
|
||||
@ -19,8 +28,7 @@ std::optional<std::string> ZooKeeperNodeCache::get(const std::string & path)
|
||||
if (!context->zookeeper)
|
||||
{
|
||||
/// Possibly, there was a previous session and it has expired. Clear the cache.
|
||||
nonexistent_nodes.clear();
|
||||
node_cache.clear();
|
||||
path_to_cached_znode.clear();
|
||||
|
||||
context->zookeeper = get_zookeeper();
|
||||
}
|
||||
@ -33,65 +41,62 @@ std::optional<std::string> ZooKeeperNodeCache::get(const std::string & path)
|
||||
throw DB::Exception("Could not get znode: `" + path + "'. ZooKeeper not configured.", DB::ErrorCodes::NO_ZOOKEEPER);
|
||||
|
||||
for (const auto & invalidated_path : invalidated_paths)
|
||||
{
|
||||
nonexistent_nodes.erase(invalidated_path);
|
||||
node_cache.erase(invalidated_path);
|
||||
}
|
||||
path_to_cached_znode.erase(invalidated_path);
|
||||
|
||||
if (nonexistent_nodes.count(path))
|
||||
return std::nullopt;
|
||||
auto cache_it = path_to_cached_znode.find(path);
|
||||
if (cache_it != path_to_cached_znode.end())
|
||||
return cache_it->second;
|
||||
|
||||
auto watch_callback = [context=context](const Coordination::WatchResponse & response)
|
||||
std::weak_ptr<Context> weak_context(context);
|
||||
auto watch_callback = [weak_context, caller_watch_callback](const Coordination::WatchResponse & response)
|
||||
{
|
||||
if (!(response.type != Coordination::SESSION || response.state == Coordination::EXPIRED_SESSION))
|
||||
return;
|
||||
|
||||
auto owned_context = weak_context.lock();
|
||||
if (!owned_context)
|
||||
return;
|
||||
|
||||
bool changed = false;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(context->mutex);
|
||||
std::lock_guard<std::mutex> lock(owned_context->mutex);
|
||||
|
||||
if (response.type != Coordination::SESSION)
|
||||
changed = context->invalidated_paths.emplace(response.path).second;
|
||||
changed = owned_context->invalidated_paths.emplace(response.path).second;
|
||||
else if (response.state == Coordination::EXPIRED_SESSION)
|
||||
{
|
||||
context->zookeeper = nullptr;
|
||||
context->invalidated_paths.clear();
|
||||
owned_context->zookeeper = nullptr;
|
||||
owned_context->invalidated_paths.clear();
|
||||
changed = true;
|
||||
}
|
||||
}
|
||||
if (changed)
|
||||
context->changed_event.set();
|
||||
if (changed && caller_watch_callback)
|
||||
caller_watch_callback(response);
|
||||
};
|
||||
|
||||
std::string contents;
|
||||
ZNode result;
|
||||
|
||||
auto cache_it = node_cache.find(path);
|
||||
if (cache_it != node_cache.end())
|
||||
result.exists = zookeeper->tryGetWatch(path, result.contents, &result.stat, watch_callback);
|
||||
if (result.exists)
|
||||
{
|
||||
return cache_it->second;
|
||||
path_to_cached_znode.emplace(path, result);
|
||||
return result;
|
||||
}
|
||||
|
||||
if (zookeeper->tryGetWatch(path, contents, /* stat = */nullptr, watch_callback))
|
||||
/// Node doesn't exist. We must set a watch on node creation (because it wasn't set by tryGetWatch).
|
||||
|
||||
result.exists = zookeeper->existsWatch(path, &result.stat, watch_callback);
|
||||
if (!result.exists)
|
||||
{
|
||||
node_cache.emplace(path, contents);
|
||||
return contents;
|
||||
path_to_cached_znode.emplace(path, result);
|
||||
return result;
|
||||
}
|
||||
|
||||
/// Node doesn't exist. Create a watch on node creation.
|
||||
nonexistent_nodes.insert(path);
|
||||
|
||||
if (!zookeeper->existsWatch(path, /* stat = */nullptr, watch_callback))
|
||||
return std::nullopt;
|
||||
|
||||
/// Node was created between the two previous calls, try again. Watch is already set.
|
||||
if (zookeeper->tryGet(path, contents))
|
||||
{
|
||||
nonexistent_nodes.erase(path);
|
||||
node_cache.emplace(path, contents);
|
||||
return contents;
|
||||
}
|
||||
|
||||
return std::nullopt;
|
||||
result.exists = zookeeper->tryGet(path, result.contents, &result.stat);
|
||||
path_to_cached_znode.emplace(path, result);
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -22,8 +22,13 @@ namespace zkutil
|
||||
|
||||
/// This class allows querying the contents of ZooKeeper nodes and caching the results.
|
||||
/// Watches are set for cached nodes and for nodes that were nonexistent at the time of query.
|
||||
/// After a watch fires, a notification is generated for the change event.
|
||||
/// After a watch fires, the callback or event that was passed by the user is notified.
|
||||
///
|
||||
/// NOTE: methods of this class are not thread-safe.
|
||||
///
|
||||
/// Intended use case: if you need one thread to watch changes in several nodes.
|
||||
/// If instead you use simple a watch event for this, watches will accumulate for nodes that do not change
|
||||
/// or change rarely.
|
||||
class ZooKeeperNodeCache
|
||||
{
|
||||
public:
|
||||
@ -32,17 +37,21 @@ public:
|
||||
ZooKeeperNodeCache(const ZooKeeperNodeCache &) = delete;
|
||||
ZooKeeperNodeCache(ZooKeeperNodeCache &&) = default;
|
||||
|
||||
std::optional<std::string> get(const std::string & path);
|
||||
struct ZNode
|
||||
{
|
||||
bool exists = false;
|
||||
std::string contents;
|
||||
Coordination::Stat stat;
|
||||
};
|
||||
|
||||
Poco::Event & getChangedEvent() { return context->changed_event; }
|
||||
ZNode get(const std::string & path, EventPtr watch_event);
|
||||
ZNode get(const std::string & path, Coordination::WatchCallback watch_callback);
|
||||
|
||||
private:
|
||||
GetZooKeeper get_zookeeper;
|
||||
|
||||
struct Context
|
||||
{
|
||||
Poco::Event changed_event;
|
||||
|
||||
std::mutex mutex;
|
||||
zkutil::ZooKeeperPtr zookeeper;
|
||||
std::unordered_set<std::string> invalidated_paths;
|
||||
@ -50,8 +59,7 @@ private:
|
||||
|
||||
std::shared_ptr<Context> context;
|
||||
|
||||
std::unordered_set<std::string> nonexistent_nodes;
|
||||
std::unordered_map<std::string, std::string> node_cache;
|
||||
std::unordered_map<std::string, ZNode> path_to_cached_znode;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -84,6 +84,19 @@ size_t NamesAndTypesList::sizeOfDifference(const NamesAndTypesList & rhs) const
|
||||
return (std::unique(vector.begin(), vector.end()) - vector.begin()) * 2 - size() - rhs.size();
|
||||
}
|
||||
|
||||
void NamesAndTypesList::getDifference(const NamesAndTypesList & rhs, NamesAndTypesList & deleted, NamesAndTypesList & added) const
|
||||
{
|
||||
NamesAndTypes lhs_vector(begin(), end());
|
||||
std::sort(lhs_vector.begin(), lhs_vector.end());
|
||||
NamesAndTypes rhs_vector(rhs.begin(), rhs.end());
|
||||
std::sort(rhs_vector.begin(), rhs_vector.end());
|
||||
|
||||
std::set_difference(lhs_vector.begin(), lhs_vector.end(), rhs_vector.begin(), rhs_vector.end(),
|
||||
std::back_inserter(deleted));
|
||||
std::set_difference(rhs_vector.begin(), rhs_vector.end(), lhs_vector.begin(), lhs_vector.end(),
|
||||
std::back_inserter(added));
|
||||
}
|
||||
|
||||
Names NamesAndTypesList::getNames() const
|
||||
{
|
||||
Names res;
|
||||
|
@ -58,6 +58,9 @@ public:
|
||||
/// (in other words, the added and deleted columns are counted once, the columns that changed the type - twice).
|
||||
size_t sizeOfDifference(const NamesAndTypesList & rhs) const;
|
||||
|
||||
/// If an element changes type, it is present both in deleted (with the old type) and in added (with the new type).
|
||||
void getDifference(const NamesAndTypesList & rhs, NamesAndTypesList & deleted, NamesAndTypesList & added) const;
|
||||
|
||||
Names getNames() const;
|
||||
DataTypes getTypes() const;
|
||||
|
||||
|
@ -684,30 +684,14 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool only_types,
|
||||
const ASTPtr & sampling_expression, const ASTPtr & primary_expression)
|
||||
bool ExpressionAnalyzer::appendPrewhere(
|
||||
ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns)
|
||||
{
|
||||
assertSelect();
|
||||
|
||||
if (!select_query->prewhere_expression)
|
||||
return false;
|
||||
|
||||
Names additional_required_mergetree_columns;
|
||||
if (sampling_expression)
|
||||
{
|
||||
auto ast = sampling_expression;
|
||||
auto syntax_result = SyntaxAnalyzer(context, storage).analyze(ast, {});
|
||||
additional_required_mergetree_columns = ExpressionAnalyzer(ast, syntax_result, context).getRequiredSourceColumns();
|
||||
}
|
||||
if (primary_expression)
|
||||
{
|
||||
auto ast = primary_expression;
|
||||
auto syntax_result = SyntaxAnalyzer(context, storage).analyze(ast, {});
|
||||
auto required_primary_columns = ExpressionAnalyzer(ast, syntax_result, context).getRequiredSourceColumns();
|
||||
additional_required_mergetree_columns.insert(additional_required_mergetree_columns.end(),
|
||||
required_primary_columns.begin(), required_primary_columns.end());
|
||||
}
|
||||
|
||||
initChain(chain, source_columns);
|
||||
auto & step = chain.getLastStep();
|
||||
getRootActions(select_query->prewhere_expression, only_types, step.actions);
|
||||
@ -725,7 +709,7 @@ bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool onl
|
||||
|
||||
/// Add required columns to required output in order not to remove them after prewhere execution.
|
||||
/// TODO: add sampling and final execution to common chain.
|
||||
for (const auto & column : additional_required_mergetree_columns)
|
||||
for (const auto & column : additional_required_columns)
|
||||
{
|
||||
if (required_source_columns.count(column))
|
||||
{
|
||||
|
@ -174,9 +174,8 @@ public:
|
||||
bool appendArrayJoin(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendJoin(ExpressionActionsChain & chain, bool only_types);
|
||||
/// remove_filter is set in ExpressionActionsChain::finalize();
|
||||
/// sampling_expression and primary_expression are needed in order to not remove columns are used in it.
|
||||
bool appendPrewhere(ExpressionActionsChain & chain, bool only_types,
|
||||
const ASTPtr & sampling_expression, const ASTPtr & primary_expression);
|
||||
/// Columns in `additional_required_columns` will not be removed (they can be used for e.g. sampling or FINAL modifier).
|
||||
bool appendPrewhere(ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns);
|
||||
bool appendWhere(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendGroupBy(ExpressionActionsChain & chain, bool only_types);
|
||||
void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types);
|
||||
|
@ -103,7 +103,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
const ASTStorage & storage = *create.storage;
|
||||
const ASTFunction & engine = *storage.engine;
|
||||
/// Currently, there are no database engines, that support any arguments.
|
||||
if (engine.arguments || engine.parameters || storage.partition_by || storage.order_by || storage.sample_by || storage.settings)
|
||||
if (engine.arguments || engine.parameters || storage.partition_by || storage.primary_key || storage.order_by || storage.sample_by || storage.settings)
|
||||
{
|
||||
std::stringstream ostr;
|
||||
formatAST(storage, ostr, false, false);
|
||||
|
@ -339,9 +339,23 @@ InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpression
|
||||
{
|
||||
ExpressionActionsChain chain(context);
|
||||
|
||||
ASTPtr sampling_expression = (storage && query.sample_size()) ? storage->getSamplingExpression() : nullptr;
|
||||
ASTPtr primary_expression = (storage && query.final()) ? storage->getPrimaryExpression() : nullptr;
|
||||
if (query_analyzer->appendPrewhere(chain, !res.first_stage, sampling_expression, primary_expression))
|
||||
Names additional_required_columns_after_prewhere;
|
||||
|
||||
if (storage && query.sample_size())
|
||||
{
|
||||
Names columns_for_sampling = storage->getColumnsRequiredForSampling();
|
||||
additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(),
|
||||
columns_for_sampling.begin(), columns_for_sampling.end());
|
||||
}
|
||||
|
||||
if (storage && query.final())
|
||||
{
|
||||
Names columns_for_final = storage->getColumnsRequiredForFinal();
|
||||
additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(),
|
||||
columns_for_final.begin(), columns_for_final.end());
|
||||
}
|
||||
|
||||
if (query_analyzer->appendPrewhere(chain, !res.first_stage, additional_required_columns_after_prewhere))
|
||||
{
|
||||
has_prewhere = true;
|
||||
|
||||
|
@ -99,24 +99,22 @@ static NameSet getKeyColumns(const StoragePtr & storage)
|
||||
|
||||
NameSet key_columns;
|
||||
|
||||
if (merge_tree_data->partition_expr)
|
||||
for (const String & col : merge_tree_data->partition_expr->getRequiredColumns())
|
||||
if (merge_tree_data->partition_key_expr)
|
||||
for (const String & col : merge_tree_data->partition_key_expr->getRequiredColumns())
|
||||
key_columns.insert(col);
|
||||
|
||||
auto primary_expr = merge_tree_data->getPrimaryExpression();
|
||||
if (primary_expr)
|
||||
for (const String & col : primary_expr->getRequiredColumns())
|
||||
key_columns.insert(col);
|
||||
/// We don't process sampling_expression separately because it must be among the primary key columns.
|
||||
|
||||
auto secondary_sort_expr = merge_tree_data->getSecondarySortExpression();
|
||||
if (secondary_sort_expr)
|
||||
for (const String & col : secondary_sort_expr->getRequiredColumns())
|
||||
auto sorting_key_expr = merge_tree_data->sorting_key_expr;
|
||||
if (sorting_key_expr)
|
||||
for (const String & col : sorting_key_expr->getRequiredColumns())
|
||||
key_columns.insert(col);
|
||||
/// We don't process sample_by_ast separately because it must be among the primary key columns.
|
||||
|
||||
if (!merge_tree_data->merging_params.sign_column.empty())
|
||||
key_columns.insert(merge_tree_data->merging_params.sign_column);
|
||||
|
||||
if (!merge_tree_data->merging_params.version_column.empty())
|
||||
key_columns.insert(merge_tree_data->merging_params.version_column);
|
||||
|
||||
return key_columns;
|
||||
}
|
||||
|
||||
|
@ -30,6 +30,11 @@ ASTPtr ASTAlterCommand::clone() const
|
||||
res->primary_key = primary_key->clone();
|
||||
res->children.push_back(res->primary_key);
|
||||
}
|
||||
if (order_by)
|
||||
{
|
||||
res->order_by = order_by->clone();
|
||||
res->children.push_back(res->order_by);
|
||||
}
|
||||
if (partition)
|
||||
{
|
||||
res->partition = partition->clone();
|
||||
@ -80,9 +85,12 @@ void ASTAlterCommand::formatImpl(
|
||||
else if (type == ASTAlterCommand::MODIFY_PRIMARY_KEY)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY PRIMARY KEY " << (settings.hilite ? hilite_none : "");
|
||||
settings.ostr << "(";
|
||||
primary_key->formatImpl(settings, state, frame);
|
||||
settings.ostr << ")";
|
||||
}
|
||||
else if (type == ASTAlterCommand::MODIFY_ORDER_BY)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY ORDER BY " << (settings.hilite ? hilite_none : "");
|
||||
order_by->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::DROP_PARTITION)
|
||||
{
|
||||
|
@ -25,8 +25,9 @@ public:
|
||||
ADD_COLUMN,
|
||||
DROP_COLUMN,
|
||||
MODIFY_COLUMN,
|
||||
MODIFY_PRIMARY_KEY,
|
||||
COMMENT_COLUMN,
|
||||
MODIFY_PRIMARY_KEY,
|
||||
MODIFY_ORDER_BY,
|
||||
|
||||
DROP_PARTITION,
|
||||
ATTACH_PARTITION,
|
||||
@ -58,6 +59,10 @@ public:
|
||||
*/
|
||||
ASTPtr primary_key;
|
||||
|
||||
/** For MODIFY ORDER BY
|
||||
*/
|
||||
ASTPtr order_by;
|
||||
|
||||
/** Used in DROP PARTITION and ATTACH PARTITION FROM queries.
|
||||
* The value or ID of the partition is stored here.
|
||||
*/
|
||||
|
@ -16,6 +16,7 @@ class ASTStorage : public IAST
|
||||
public:
|
||||
ASTFunction * engine = nullptr;
|
||||
IAST * partition_by = nullptr;
|
||||
IAST * primary_key = nullptr;
|
||||
IAST * order_by = nullptr;
|
||||
IAST * sample_by = nullptr;
|
||||
ASTSetQuery * settings = nullptr;
|
||||
@ -31,6 +32,8 @@ public:
|
||||
res->set(res->engine, engine->clone());
|
||||
if (partition_by)
|
||||
res->set(res->partition_by, partition_by->clone());
|
||||
if (primary_key)
|
||||
res->set(res->primary_key, primary_key->clone());
|
||||
if (order_by)
|
||||
res->set(res->order_by, order_by->clone());
|
||||
if (sample_by)
|
||||
@ -53,6 +56,11 @@ public:
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "PARTITION BY " << (s.hilite ? hilite_none : "");
|
||||
partition_by->formatImpl(s, state, frame);
|
||||
}
|
||||
if (primary_key)
|
||||
{
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "PRIMARY KEY " << (s.hilite ? hilite_none : "");
|
||||
primary_key->formatImpl(s, state, frame);
|
||||
}
|
||||
if (order_by)
|
||||
{
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "ORDER BY " << (s.hilite ? hilite_none : "");
|
||||
|
@ -30,7 +30,7 @@ protected:
|
||||
|
||||
|
||||
template <typename... Args>
|
||||
ASTPtr makeASTFunction(const String & name, Args &&... args)
|
||||
std::shared_ptr<ASTFunction> makeASTFunction(const String & name, Args &&... args)
|
||||
{
|
||||
const auto function = std::make_shared<ASTFunction>();
|
||||
|
||||
|
@ -23,8 +23,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserKeyword s_drop_column("DROP COLUMN");
|
||||
ParserKeyword s_clear_column("CLEAR COLUMN");
|
||||
ParserKeyword s_modify_column("MODIFY COLUMN");
|
||||
ParserKeyword s_modify_primary_key("MODIFY PRIMARY KEY");
|
||||
ParserKeyword s_comment_column("COMMENT COLUMN");
|
||||
ParserKeyword s_modify_primary_key("MODIFY PRIMARY KEY");
|
||||
ParserKeyword s_modify_order_by("MODIFY ORDER BY");
|
||||
|
||||
ParserKeyword s_attach_partition("ATTACH PARTITION");
|
||||
ParserKeyword s_detach_partition("DETACH PARTITION");
|
||||
@ -197,19 +198,18 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_modify_primary_key.ignore(pos, expected))
|
||||
{
|
||||
if (pos->type != TokenType::OpeningRoundBracket)
|
||||
if (!parser_exp_elem.parse(pos, command->primary_key, expected))
|
||||
return false;
|
||||
++pos;
|
||||
|
||||
if (!ParserNotEmptyExpressionList(false).parse(pos, command->primary_key, expected))
|
||||
return false;
|
||||
|
||||
if (pos->type != TokenType::ClosingRoundBracket)
|
||||
return false;
|
||||
++pos;
|
||||
|
||||
command->type = ASTAlterCommand::MODIFY_PRIMARY_KEY;
|
||||
}
|
||||
else if (s_modify_order_by.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_exp_elem.parse(pos, command->order_by, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::MODIFY_ORDER_BY;
|
||||
}
|
||||
else if (s_delete_where.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_exp_elem.parse(pos, command->predicate, expected))
|
||||
|
@ -96,6 +96,7 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ParserKeyword s_engine("ENGINE");
|
||||
ParserToken s_eq(TokenType::Equals);
|
||||
ParserKeyword s_partition_by("PARTITION BY");
|
||||
ParserKeyword s_primary_key("PRIMARY KEY");
|
||||
ParserKeyword s_order_by("ORDER BY");
|
||||
ParserKeyword s_sample_by("SAMPLE BY");
|
||||
ParserKeyword s_settings("SETTINGS");
|
||||
@ -106,6 +107,7 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
|
||||
ASTPtr engine;
|
||||
ASTPtr partition_by;
|
||||
ASTPtr primary_key;
|
||||
ASTPtr order_by;
|
||||
ASTPtr sample_by;
|
||||
ASTPtr settings;
|
||||
@ -128,6 +130,14 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!primary_key && s_primary_key.ignore(pos, expected))
|
||||
{
|
||||
if (expression_p.parse(pos, primary_key, expected))
|
||||
continue;
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!order_by && s_order_by.ignore(pos, expected))
|
||||
{
|
||||
if (expression_p.parse(pos, order_by, expected))
|
||||
@ -156,6 +166,7 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
auto storage = std::make_shared<ASTStorage>();
|
||||
storage->set(storage->engine, engine);
|
||||
storage->set(storage->partition_by, partition_by);
|
||||
storage->set(storage->primary_key, primary_key);
|
||||
storage->set(storage->order_by, order_by);
|
||||
storage->set(storage->sample_by, sample_by);
|
||||
storage->set(storage->settings, settings);
|
||||
|
@ -199,7 +199,7 @@ protected:
|
||||
};
|
||||
|
||||
|
||||
/** ENGINE = name [PARTITION BY expr] [ORDER BY expr] [SAMPLE BY expr] [SETTINGS name = value, ...] */
|
||||
/** ENGINE = name [PARTITION BY expr] [ORDER BY expr] [PRIMARY KEY expr] [SAMPLE BY expr] [SETTINGS name = value, ...] */
|
||||
class ParserStorage : public IParserBase
|
||||
{
|
||||
protected:
|
||||
|
@ -13,4 +13,11 @@ void formatAST(const IAST & ast, std::ostream & s, bool hilite, bool one_line)
|
||||
ast.format(settings);
|
||||
}
|
||||
|
||||
String serializeAST(const IAST & ast, bool one_line)
|
||||
{
|
||||
std::stringstream ss;
|
||||
formatAST(ast, ss, false, one_line);
|
||||
return ss.str();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -12,6 +12,7 @@ namespace DB
|
||||
*/
|
||||
void formatAST(const IAST & ast, std::ostream & s, bool hilite = true, bool one_line = false);
|
||||
|
||||
String serializeAST(const IAST & ast, bool one_line = true);
|
||||
|
||||
inline std::ostream & operator<<(std::ostream & os, const IAST & ast)
|
||||
{
|
||||
|
@ -22,6 +22,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
@ -90,13 +91,6 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::MODIFY_PRIMARY_KEY)
|
||||
{
|
||||
AlterCommand command;
|
||||
command.type = AlterCommand::MODIFY_PRIMARY_KEY;
|
||||
command.primary_key = command_ast->primary_key;
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::COMMENT_COLUMN)
|
||||
{
|
||||
AlterCommand command;
|
||||
@ -107,6 +101,20 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
command.comment = ast_comment.value.get<String>();
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::MODIFY_PRIMARY_KEY)
|
||||
{
|
||||
AlterCommand command;
|
||||
command.type = AlterCommand::MODIFY_PRIMARY_KEY;
|
||||
command.primary_key = command_ast->primary_key;
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::MODIFY_ORDER_BY)
|
||||
{
|
||||
AlterCommand command;
|
||||
command.type = AlterCommand::MODIFY_ORDER_BY;
|
||||
command.order_by = command_ast->order_by;
|
||||
return command;
|
||||
}
|
||||
else
|
||||
return {};
|
||||
}
|
||||
@ -119,7 +127,7 @@ static bool namesEqual(const String & name_without_dot, const DB::NameAndTypePai
|
||||
return (name_with_dot == name_type.name.substr(0, name_without_dot.length() + 1) || name_without_dot == name_type.name);
|
||||
}
|
||||
|
||||
void AlterCommand::apply(ColumnsDescription & columns_description) const
|
||||
void AlterCommand::apply(ColumnsDescription & columns_description, ASTPtr & order_by_ast, ASTPtr & primary_key_ast) const
|
||||
{
|
||||
if (type == ADD_COLUMN)
|
||||
{
|
||||
@ -265,12 +273,24 @@ void AlterCommand::apply(ColumnsDescription & columns_description) const
|
||||
}
|
||||
else if (type == MODIFY_PRIMARY_KEY)
|
||||
{
|
||||
/// This have no relation to changing the list of columns.
|
||||
/// TODO Check that all columns exist, that only columns with constant defaults are added.
|
||||
if (!primary_key_ast)
|
||||
order_by_ast = primary_key;
|
||||
else
|
||||
primary_key_ast = primary_key;
|
||||
}
|
||||
else if (type == MODIFY_ORDER_BY)
|
||||
{
|
||||
if (!primary_key_ast)
|
||||
{
|
||||
/// Primary and sorting key become independent after this ALTER so we have to
|
||||
/// save the old ORDER BY expression as the new primary key.
|
||||
primary_key_ast = order_by_ast->clone();
|
||||
}
|
||||
|
||||
order_by_ast = order_by;
|
||||
}
|
||||
else if (type == COMMENT_COLUMN)
|
||||
{
|
||||
|
||||
columns_description.comments[column_name] = comment;
|
||||
}
|
||||
else
|
||||
@ -287,14 +307,18 @@ bool AlterCommand::is_mutable() const
|
||||
return true;
|
||||
}
|
||||
|
||||
void AlterCommands::apply(ColumnsDescription & columns_description) const
|
||||
void AlterCommands::apply(ColumnsDescription & columns_description, ASTPtr & order_by_ast, ASTPtr & primary_key_ast) const
|
||||
{
|
||||
auto new_columns_description = columns_description;
|
||||
auto new_order_by_ast = order_by_ast;
|
||||
auto new_primary_key_ast = primary_key_ast;
|
||||
|
||||
for (const AlterCommand & command : *this)
|
||||
command.apply(new_columns_description);
|
||||
command.apply(new_columns_description, new_order_by_ast, new_primary_key_ast);
|
||||
|
||||
columns_description = std::move(new_columns_description);
|
||||
order_by_ast = std::move(new_order_by_ast);
|
||||
primary_key_ast = std::move(new_primary_key_ast);
|
||||
}
|
||||
|
||||
void AlterCommands::validate(const IStorage & table, const Context & context)
|
||||
@ -482,6 +506,21 @@ void AlterCommands::validate(const IStorage & table, const Context & context)
|
||||
}
|
||||
}
|
||||
|
||||
void AlterCommands::apply(ColumnsDescription & columns_description) const
|
||||
{
|
||||
auto out_columns_description = columns_description;
|
||||
ASTPtr out_order_by;
|
||||
ASTPtr out_primary_key;
|
||||
apply(out_columns_description, out_order_by, out_primary_key);
|
||||
|
||||
if (out_order_by)
|
||||
throw Exception("Storage doesn't support modifying ORDER BY expression", ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (out_primary_key)
|
||||
throw Exception("Storage doesn't support modifying PRIMARY KEY expression", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
columns_description = std::move(out_columns_description);
|
||||
}
|
||||
|
||||
bool AlterCommands::is_mutable() const
|
||||
{
|
||||
for (const auto & param : *this)
|
||||
|
@ -21,8 +21,9 @@ struct AlterCommand
|
||||
ADD_COLUMN,
|
||||
DROP_COLUMN,
|
||||
MODIFY_COLUMN,
|
||||
MODIFY_PRIMARY_KEY,
|
||||
COMMENT_COLUMN,
|
||||
MODIFY_PRIMARY_KEY,
|
||||
MODIFY_ORDER_BY,
|
||||
UKNOWN_TYPE,
|
||||
};
|
||||
|
||||
@ -46,6 +47,9 @@ struct AlterCommand
|
||||
/// For MODIFY_PRIMARY_KEY
|
||||
ASTPtr primary_key;
|
||||
|
||||
/// For MODIFY_ORDER_BY
|
||||
ASTPtr order_by;
|
||||
|
||||
AlterCommand() = default;
|
||||
AlterCommand(const Type type, const String & column_name, const DataTypePtr & data_type,
|
||||
const ColumnDefaultKind default_kind, const ASTPtr & default_expression,
|
||||
@ -56,8 +60,7 @@ struct AlterCommand
|
||||
|
||||
static std::optional<AlterCommand> parse(const ASTAlterCommand * command);
|
||||
|
||||
void apply(ColumnsDescription & columns_description) const;
|
||||
|
||||
void apply(ColumnsDescription & columns_description, ASTPtr & order_by_ast, ASTPtr & primary_key_ast) const;
|
||||
/// Checks that not only metadata touched by that command
|
||||
bool is_mutable() const;
|
||||
};
|
||||
@ -68,6 +71,9 @@ class Context;
|
||||
class AlterCommands : public std::vector<AlterCommand>
|
||||
{
|
||||
public:
|
||||
void apply(ColumnsDescription & columns_description, ASTPtr & order_by_ast, ASTPtr & primary_key_ast) const;
|
||||
|
||||
/// For storages that don't support MODIFY_PRIMARY_KEY or MODIFY_ORDER_BY.
|
||||
void apply(ColumnsDescription & columns_description) const;
|
||||
|
||||
void validate(const IStorage & table, const Context & context);
|
||||
|
@ -322,29 +322,32 @@ public:
|
||||
/// Returns data path if storage supports it, empty string otherwise.
|
||||
virtual String getDataPath() const { return {}; }
|
||||
|
||||
/// Returns sampling expression for storage or nullptr if there is no.
|
||||
virtual ASTPtr getSamplingExpression() const { return nullptr; }
|
||||
/// Returns ASTExpressionList of partition key expression for storage or nullptr if there is none.
|
||||
virtual ASTPtr getPartitionKeyAST() const { return nullptr; }
|
||||
|
||||
/// Returns primary expression for storage or nullptr if there is no.
|
||||
virtual ASTPtr getPrimaryExpression() const { return nullptr; }
|
||||
/// Returns ASTExpressionList of sorting key expression for storage or nullptr if there is none.
|
||||
virtual ASTPtr getSortingKeyAST() const { return nullptr; }
|
||||
|
||||
/// Returns partition expression for storage or nullptr if there is no.
|
||||
virtual ASTPtr getPartitionExpression() const { return nullptr; }
|
||||
/// Returns ASTExpressionList of primary key expression for storage or nullptr if there is none.
|
||||
virtual ASTPtr getPrimaryKeyAST() const { return nullptr; }
|
||||
|
||||
/// Returns secondary expression for storage or nullptr if there is no.
|
||||
virtual ASTPtr getOrderExpression() const { return nullptr; }
|
||||
/// Returns sampling expression AST for storage or nullptr if there is none.
|
||||
virtual ASTPtr getSamplingKeyAST() const { return nullptr; }
|
||||
|
||||
/// Returns sampling key names for storage or empty vector if there is no.
|
||||
virtual Names getSamplingExpressionNames() const { return {}; }
|
||||
/// Returns additional columns that need to be read to calculate partition key.
|
||||
virtual Names getColumnsRequiredForPartitionKey() const { return {}; }
|
||||
|
||||
/// Returns primary key names for storage or empty vector if there is no.
|
||||
virtual Names getPrimaryExpressionNames() const { return {}; }
|
||||
/// Returns additional columns that need to be read to calculate sorting key.
|
||||
virtual Names getColumnsRequiredForSortingKey() const { return {}; }
|
||||
|
||||
/// Returns partition key names for storage or empty vector if there is no.
|
||||
virtual Names getPartitionExpressionNames() const { return {}; }
|
||||
/// Returns additional columns that need to be read to calculate primary key.
|
||||
virtual Names getColumnsRequiredForPrimaryKey() const { return {}; }
|
||||
|
||||
/// Returns order key names for storage or empty vector if there is no.
|
||||
virtual Names getOrderExpressionNames() const { return {}; }
|
||||
/// Returns additional columns that need to be read to calculate sampling key.
|
||||
virtual Names getColumnsRequiredForSampling() const { return {}; }
|
||||
|
||||
/// Returns additional columns that need to be read for FINAL to work.
|
||||
virtual Names getColumnsRequiredForFinal() const { return {}; }
|
||||
|
||||
using ITableDeclaration::ITableDeclaration;
|
||||
using std::enable_shared_from_this<IStorage>::shared_from_this;
|
||||
|
@ -89,25 +89,22 @@ MergeTreeData::MergeTreeData(
|
||||
const String & database_, const String & table_,
|
||||
const String & full_path_, const ColumnsDescription & columns_,
|
||||
Context & context_,
|
||||
const ASTPtr & primary_expr_ast_,
|
||||
const ASTPtr & secondary_sort_expr_ast_,
|
||||
const String & date_column_name,
|
||||
const ASTPtr & partition_expr_ast_,
|
||||
const ASTPtr & sampling_expression_,
|
||||
const ASTPtr & partition_by_ast_,
|
||||
const ASTPtr & order_by_ast_,
|
||||
const ASTPtr & primary_key_ast_,
|
||||
const ASTPtr & sample_by_ast_,
|
||||
const MergingParams & merging_params_,
|
||||
const MergeTreeSettings & settings_,
|
||||
bool require_part_metadata_,
|
||||
bool attach,
|
||||
BrokenPartCallback broken_part_callback_)
|
||||
: ITableDeclaration{columns_},
|
||||
context(context_),
|
||||
sampling_expression(sampling_expression_),
|
||||
index_granularity(settings_.index_granularity),
|
||||
: context(context_),
|
||||
merging_params(merging_params_),
|
||||
index_granularity(settings_.index_granularity),
|
||||
settings(settings_),
|
||||
primary_expr_ast(primary_expr_ast_),
|
||||
secondary_sort_expr_ast(secondary_sort_expr_ast_),
|
||||
partition_expr_ast(partition_expr_ast_),
|
||||
partition_by_ast(partition_by_ast_),
|
||||
sample_by_ast(sample_by_ast_),
|
||||
require_part_metadata(require_part_metadata_),
|
||||
database_name(database_), table_name(table_),
|
||||
full_path(full_path_),
|
||||
@ -116,28 +113,30 @@ MergeTreeData::MergeTreeData(
|
||||
data_parts_by_info(data_parts_indexes.get<TagByInfo>()),
|
||||
data_parts_by_state_and_info(data_parts_indexes.get<TagByStateAndInfo>())
|
||||
{
|
||||
setPrimaryKeyAndColumns(order_by_ast_, primary_key_ast_, columns_);
|
||||
|
||||
/// NOTE: using the same columns list as is read when performing actual merges.
|
||||
merging_params.check(getColumns().getAllPhysical());
|
||||
|
||||
if (!primary_expr_ast)
|
||||
throw Exception("Primary key cannot be empty", ErrorCodes::BAD_ARGUMENTS);
|
||||
if (sample_by_ast)
|
||||
{
|
||||
sampling_expr_column_name = sample_by_ast->getColumnName();
|
||||
|
||||
initPrimaryKey();
|
||||
if (!primary_key_sample.has(sampling_expr_column_name)
|
||||
&& !attach && !settings.compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility.
|
||||
throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (sampling_expression && (!primary_key_sample.has(sampling_expression->getColumnName()))
|
||||
&& !attach && !settings.compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility.
|
||||
throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS);
|
||||
auto syntax = SyntaxAnalyzer(context, {}).analyze(sample_by_ast, getColumns().getAllPhysical());
|
||||
columns_required_for_sampling = ExpressionAnalyzer(sample_by_ast, syntax, context)
|
||||
.getRequiredSourceColumns();
|
||||
}
|
||||
|
||||
MergeTreeDataFormatVersion min_format_version(0);
|
||||
if (!date_column_name.empty())
|
||||
{
|
||||
try
|
||||
{
|
||||
String partition_expr_str = "toYYYYMM(" + backQuoteIfNeed(date_column_name) + ")";
|
||||
ParserNotEmptyExpressionList parser(/* allow_alias_without_as_keyword = */ false);
|
||||
partition_expr_ast = parseQuery(
|
||||
parser, partition_expr_str.data(), partition_expr_str.data() + partition_expr_str.length(), "partition expression", 0);
|
||||
|
||||
partition_by_ast = makeASTFunction("toYYYYMM", std::make_shared<ASTIdentifier>(date_column_name));
|
||||
initPartitionKey();
|
||||
|
||||
if (minmax_idx_date_column_pos == -1)
|
||||
@ -152,6 +151,7 @@ MergeTreeData::MergeTreeData(
|
||||
}
|
||||
else
|
||||
{
|
||||
is_custom_partitioned = true;
|
||||
initPartitionKey();
|
||||
min_format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING;
|
||||
}
|
||||
@ -217,69 +217,177 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeData::initPrimaryKey()
|
||||
void MergeTreeData::setPrimaryKeyAndColumns(
|
||||
const ASTPtr & new_order_by_ast, ASTPtr new_primary_key_ast, const ColumnsDescription & new_columns, bool only_check)
|
||||
{
|
||||
auto addSortColumns = [](Names & out, const ASTPtr & expr_ast)
|
||||
if (!new_order_by_ast)
|
||||
throw Exception("ORDER BY cannot be empty", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
ASTPtr new_sorting_key_expr_list = extractKeyExpressionList(new_order_by_ast);
|
||||
ASTPtr new_primary_key_expr_list = new_primary_key_ast
|
||||
? extractKeyExpressionList(new_primary_key_ast) : new_sorting_key_expr_list->clone();
|
||||
|
||||
if (merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing)
|
||||
new_sorting_key_expr_list->children.push_back(std::make_shared<ASTIdentifier>(merging_params.version_column));
|
||||
|
||||
size_t primary_key_size = new_primary_key_expr_list->children.size();
|
||||
size_t sorting_key_size = new_sorting_key_expr_list->children.size();
|
||||
if (primary_key_size > sorting_key_size)
|
||||
throw Exception("Primary key must be a prefix of the sorting key, but its length: "
|
||||
+ toString(primary_key_size) + " is greater than the sorting key length: " + toString(sorting_key_size),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
Names new_primary_key_columns;
|
||||
Names new_sorting_key_columns;
|
||||
|
||||
for (size_t i = 0; i < sorting_key_size; ++i)
|
||||
{
|
||||
out.reserve(out.size() + expr_ast->children.size());
|
||||
for (const ASTPtr & ast : expr_ast->children)
|
||||
out.emplace_back(ast->getColumnName());
|
||||
};
|
||||
String sorting_key_column = new_sorting_key_expr_list->children[i]->getColumnName();
|
||||
new_sorting_key_columns.push_back(sorting_key_column);
|
||||
|
||||
/// Initialize description of sorting for primary key.
|
||||
primary_sort_columns.clear();
|
||||
addSortColumns(primary_sort_columns, primary_expr_ast);
|
||||
if (i < primary_key_size)
|
||||
{
|
||||
String pk_column = new_primary_key_expr_list->children[i]->getColumnName();
|
||||
if (pk_column != sorting_key_column)
|
||||
throw Exception("Primary key must be a prefix of the sorting key, but in position "
|
||||
+ toString(i) + " its column is " + pk_column + ", not " + sorting_key_column,
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
{
|
||||
auto syntax_result = SyntaxAnalyzer(context, {}).analyze(primary_expr_ast, getColumns().getAllPhysical());
|
||||
primary_expr = ExpressionAnalyzer(primary_expr_ast, syntax_result, context).getActions(false);
|
||||
|
||||
ExpressionActionsPtr projected_expr = ExpressionAnalyzer(primary_expr_ast, syntax_result, context).getActions(true);
|
||||
primary_key_sample = projected_expr->getSampleBlock();
|
||||
new_primary_key_columns.push_back(pk_column);
|
||||
}
|
||||
}
|
||||
|
||||
checkKeyExpression(*primary_expr, primary_key_sample, "Primary");
|
||||
auto all_columns = new_columns.getAllPhysical();
|
||||
|
||||
size_t primary_key_size = primary_key_sample.columns();
|
||||
primary_key_data_types.resize(primary_key_size);
|
||||
for (size_t i = 0; i < primary_key_size; ++i)
|
||||
primary_key_data_types[i] = primary_key_sample.getByPosition(i).type;
|
||||
|
||||
sort_columns = primary_sort_columns;
|
||||
if (secondary_sort_expr_ast)
|
||||
if (order_by_ast && only_check)
|
||||
{
|
||||
addSortColumns(sort_columns, secondary_sort_expr_ast);
|
||||
auto syntax_result = SyntaxAnalyzer(context, {}).analyze(secondary_sort_expr_ast, getColumns().getAllPhysical());
|
||||
secondary_sort_expr = ExpressionAnalyzer(secondary_sort_expr_ast, syntax_result, context).getActions(false);
|
||||
/// This is ALTER, not CREATE/ATTACH TABLE. Let us check that all new columns used in the sorting key
|
||||
/// expression have just been added (so that the sorting order is guaranteed to be valid with the new key).
|
||||
|
||||
ExpressionActionsPtr projected_expr = ExpressionAnalyzer(secondary_sort_expr_ast, syntax_result, context).getActions(true);
|
||||
auto secondary_key_sample = projected_expr->getSampleBlock();
|
||||
ASTPtr added_key_column_expr_list = std::make_shared<ASTExpressionList>();
|
||||
for (size_t new_i = 0, old_i = 0; new_i < sorting_key_size; ++new_i)
|
||||
{
|
||||
if (old_i < sorting_key_columns.size())
|
||||
{
|
||||
if (new_sorting_key_columns[new_i] != sorting_key_columns[old_i])
|
||||
added_key_column_expr_list->children.push_back(new_sorting_key_expr_list->children[new_i]);
|
||||
else
|
||||
++old_i;
|
||||
}
|
||||
else
|
||||
added_key_column_expr_list->children.push_back(new_sorting_key_expr_list->children[new_i]);
|
||||
}
|
||||
|
||||
checkKeyExpression(*secondary_sort_expr, secondary_key_sample, "Secondary");
|
||||
if (!added_key_column_expr_list->children.empty())
|
||||
{
|
||||
auto syntax = SyntaxAnalyzer(context, {}).analyze(added_key_column_expr_list, all_columns);
|
||||
Names used_columns = ExpressionAnalyzer(added_key_column_expr_list, syntax, context)
|
||||
.getRequiredSourceColumns();
|
||||
|
||||
NamesAndTypesList deleted_columns;
|
||||
NamesAndTypesList added_columns;
|
||||
getColumns().getAllPhysical().getDifference(all_columns, deleted_columns, added_columns);
|
||||
|
||||
for (const String & col : used_columns)
|
||||
{
|
||||
if (!added_columns.contains(col) || deleted_columns.contains(col))
|
||||
throw Exception("Existing column " + col + " is used in the expression that was "
|
||||
"added to the sorting key. You can add expressions that use only the newly added columns",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (new_columns.defaults.count(col))
|
||||
throw Exception("Newly added column " + col + " has a default expression, so adding "
|
||||
"expressions that use it to the sorting key is forbidden",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
auto new_sorting_key_syntax = SyntaxAnalyzer(context, {}).analyze(new_sorting_key_expr_list, all_columns);
|
||||
auto new_sorting_key_expr = ExpressionAnalyzer(new_sorting_key_expr_list, new_sorting_key_syntax, context)
|
||||
.getActions(false);
|
||||
auto new_sorting_key_sample =
|
||||
ExpressionAnalyzer(new_sorting_key_expr_list, new_sorting_key_syntax, context)
|
||||
.getActions(true)->getSampleBlock();
|
||||
|
||||
checkKeyExpression(*new_sorting_key_expr, new_sorting_key_sample, "Sorting");
|
||||
|
||||
auto new_primary_key_syntax = SyntaxAnalyzer(context, {}).analyze(new_primary_key_expr_list, all_columns);
|
||||
auto new_primary_key_expr = ExpressionAnalyzer(new_primary_key_expr_list, new_primary_key_syntax, context)
|
||||
.getActions(false);
|
||||
|
||||
Block new_primary_key_sample;
|
||||
DataTypes new_primary_key_data_types;
|
||||
for (size_t i = 0; i < primary_key_size; ++i)
|
||||
{
|
||||
const auto & elem = new_sorting_key_sample.getByPosition(i);
|
||||
new_primary_key_sample.insert(elem);
|
||||
new_primary_key_data_types.push_back(elem.type);
|
||||
}
|
||||
|
||||
if (!only_check)
|
||||
{
|
||||
setColumns(new_columns);
|
||||
|
||||
order_by_ast = new_order_by_ast;
|
||||
sorting_key_columns = std::move(new_sorting_key_columns);
|
||||
sorting_key_expr_ast = std::move(new_sorting_key_expr_list);
|
||||
sorting_key_expr = std::move(new_sorting_key_expr);
|
||||
|
||||
primary_key_ast = new_primary_key_ast;
|
||||
primary_key_columns = std::move(new_primary_key_columns);
|
||||
primary_key_expr_ast = std::move(new_primary_key_expr_list);
|
||||
primary_key_expr = std::move(new_primary_key_expr);
|
||||
primary_key_sample = std::move(new_primary_key_sample);
|
||||
primary_key_data_types = std::move(new_primary_key_data_types);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
ASTPtr MergeTreeData::extractKeyExpressionList(const ASTPtr & node)
|
||||
{
|
||||
if (!node)
|
||||
return std::make_shared<ASTExpressionList>();
|
||||
|
||||
const ASTFunction * expr_func = typeid_cast<const ASTFunction *>(node.get());
|
||||
|
||||
if (expr_func && expr_func->name == "tuple")
|
||||
{
|
||||
/// Primary key is specified in tuple.
|
||||
return expr_func->children.at(0);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Primary key consists of one column.
|
||||
auto res = std::make_shared<ASTExpressionList>();
|
||||
res->children.push_back(node);
|
||||
return res;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeData::initPartitionKey()
|
||||
{
|
||||
if (!partition_expr_ast || partition_expr_ast->children.empty())
|
||||
ASTPtr partition_key_expr_list = extractKeyExpressionList(partition_by_ast);
|
||||
|
||||
if (partition_key_expr_list->children.empty())
|
||||
return;
|
||||
|
||||
{
|
||||
auto syntax_result = SyntaxAnalyzer(context, {}).analyze(partition_expr_ast, getColumns().getAllPhysical());
|
||||
partition_expr = ExpressionAnalyzer(partition_expr_ast, syntax_result, context).getActions(false);
|
||||
auto syntax_result = SyntaxAnalyzer(context, {}).analyze(partition_key_expr_list, getColumns().getAllPhysical());
|
||||
partition_key_expr = ExpressionAnalyzer(partition_key_expr_list, syntax_result, context).getActions(false);
|
||||
}
|
||||
|
||||
for (const ASTPtr & ast : partition_expr_ast->children)
|
||||
for (const ASTPtr & ast : partition_key_expr_list->children)
|
||||
{
|
||||
String col_name = ast->getColumnName();
|
||||
partition_key_sample.insert(partition_expr->getSampleBlock().getByName(col_name));
|
||||
partition_key_sample.insert(partition_key_expr->getSampleBlock().getByName(col_name));
|
||||
}
|
||||
|
||||
checkKeyExpression(*partition_expr, partition_key_sample, "Partition");
|
||||
checkKeyExpression(*partition_key_expr, partition_key_sample, "Partition");
|
||||
|
||||
/// Add all columns used in the partition key to the min-max index.
|
||||
const NamesAndTypesList & minmax_idx_columns_with_types = partition_expr->getRequiredColumnsWithTypes();
|
||||
const NamesAndTypesList & minmax_idx_columns_with_types = partition_key_expr->getRequiredColumnsWithTypes();
|
||||
minmax_idx_expr = std::make_shared<ExpressionActions>(minmax_idx_columns_with_types, context);
|
||||
for (const NameAndTypePair & column : minmax_idx_columns_with_types)
|
||||
{
|
||||
@ -873,7 +981,9 @@ void MergeTreeData::checkAlter(const AlterCommands & commands)
|
||||
{
|
||||
/// Check that needed transformations can be applied to the list of columns without considering type conversions.
|
||||
auto new_columns = getColumns();
|
||||
commands.apply(new_columns);
|
||||
ASTPtr new_order_by_ast = order_by_ast;
|
||||
ASTPtr new_primary_key_ast = primary_key_ast;
|
||||
commands.apply(new_columns, new_order_by_ast, new_primary_key_ast);
|
||||
|
||||
/// Set of columns that shouldn't be altered.
|
||||
NameSet columns_alter_forbidden;
|
||||
@ -882,33 +992,28 @@ void MergeTreeData::checkAlter(const AlterCommands & commands)
|
||||
/// (and not as a part of some expression) and if the ALTER only affects column metadata.
|
||||
NameSet columns_alter_metadata_only;
|
||||
|
||||
if (partition_expr)
|
||||
if (partition_key_expr)
|
||||
{
|
||||
/// Forbid altering partition key columns because it can change partition ID format.
|
||||
/// TODO: in some cases (e.g. adding an Enum value) a partition key column can still be ALTERed.
|
||||
/// We should allow it.
|
||||
for (const String & col : partition_expr->getRequiredColumns())
|
||||
for (const String & col : partition_key_expr->getRequiredColumns())
|
||||
columns_alter_forbidden.insert(col);
|
||||
}
|
||||
|
||||
auto processSortingColumns =
|
||||
[&columns_alter_forbidden, &columns_alter_metadata_only] (const ExpressionActionsPtr & expression)
|
||||
if (sorting_key_expr)
|
||||
{
|
||||
for (const ExpressionAction & action : expression->getActions())
|
||||
for (const ExpressionAction & action : sorting_key_expr->getActions())
|
||||
{
|
||||
auto action_columns = action.getNeededColumns();
|
||||
columns_alter_forbidden.insert(action_columns.begin(), action_columns.end());
|
||||
}
|
||||
for (const String & col : expression->getRequiredColumns())
|
||||
for (const String & col : sorting_key_expr->getRequiredColumns())
|
||||
columns_alter_metadata_only.insert(col);
|
||||
};
|
||||
|
||||
if (primary_expr)
|
||||
processSortingColumns(primary_expr);
|
||||
/// We don't process sampling_expression separately because it must be among the primary key columns.
|
||||
|
||||
if (secondary_sort_expr)
|
||||
processSortingColumns(secondary_sort_expr);
|
||||
/// We don't process sample_by_ast separately because it must be among the primary key columns
|
||||
/// and we don't process primary_key_expr separately because it is a prefix of sorting_key_expr.
|
||||
}
|
||||
|
||||
if (!merging_params.sign_column.empty())
|
||||
columns_alter_forbidden.insert(merging_params.sign_column);
|
||||
@ -940,8 +1045,18 @@ void MergeTreeData::checkAlter(const AlterCommands & commands)
|
||||
"ALTER of key column " + command.column_name + " must be metadata-only",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
if (command.type == AlterCommand::MODIFY_ORDER_BY)
|
||||
{
|
||||
if (!is_custom_partitioned)
|
||||
throw Exception(
|
||||
"ALTER MODIFY ORDER BY is not supported for default-partitioned tables created with the old syntax",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
}
|
||||
|
||||
setPrimaryKeyAndColumns(new_order_by_ast, new_primary_key_ast, new_columns, /* only_check = */ true);
|
||||
|
||||
/// Check that type conversions are possible.
|
||||
ExpressionActionsPtr unused_expression;
|
||||
NameToNameMap unused_map;
|
||||
@ -1095,7 +1210,7 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name
|
||||
MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart(
|
||||
const DataPartPtr & part,
|
||||
const NamesAndTypesList & new_columns,
|
||||
const ASTPtr & new_primary_key,
|
||||
const ASTPtr & new_primary_key_expr_list,
|
||||
bool skip_sanity_checks)
|
||||
{
|
||||
ExpressionActionsPtr expression;
|
||||
@ -1160,10 +1275,9 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart(
|
||||
size_t new_primary_key_file_size{};
|
||||
MergeTreeDataPartChecksum::uint128 new_primary_key_hash{};
|
||||
|
||||
/// TODO: Check the order of secondary sorting key columns.
|
||||
if (new_primary_key.get() != primary_expr_ast.get())
|
||||
if (new_primary_key_expr_list)
|
||||
{
|
||||
ASTPtr query = new_primary_key;
|
||||
ASTPtr query = new_primary_key_expr_list;
|
||||
auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, new_columns);
|
||||
ExpressionActionsPtr new_primary_expr = ExpressionAnalyzer(query, syntax_result, context).getActions(true);
|
||||
Block new_primary_key_sample = new_primary_expr->getSampleBlock();
|
||||
@ -2335,7 +2449,7 @@ bool MergeTreeData::isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const A
|
||||
{
|
||||
const String column_name = node->getColumnName();
|
||||
|
||||
for (const auto & name : primary_sort_columns)
|
||||
for (const auto & name : primary_key_columns)
|
||||
if (column_name == name)
|
||||
return true;
|
||||
|
||||
@ -2392,10 +2506,10 @@ MergeTreeData * MergeTreeData::checkStructureAndGetMergeTreeData(const StoragePt
|
||||
return ast ? queryToString(ast) : "";
|
||||
};
|
||||
|
||||
if (query_to_string(secondary_sort_expr_ast) != query_to_string(src_data->secondary_sort_expr_ast))
|
||||
if (query_to_string(order_by_ast) != query_to_string(src_data->order_by_ast))
|
||||
throw Exception("Tables have different ordering", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (query_to_string(partition_expr_ast) != query_to_string(src_data->partition_expr_ast))
|
||||
if (query_to_string(partition_by_ast) != query_to_string(src_data->partition_by_ast))
|
||||
throw Exception("Tables have different partition key", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (format_version != src_data->format_version)
|
||||
|
@ -284,20 +284,28 @@ public:
|
||||
/// Attach the table corresponding to the directory in full_path (must end with /), with the given columns.
|
||||
/// Correctness of names and paths is not checked.
|
||||
///
|
||||
/// primary_expr_ast - expression used for sorting;
|
||||
/// date_column_name - if not empty, the name of the Date column used for partitioning by month.
|
||||
/// Otherwise, partition_expr_ast is used for partitioning.
|
||||
/// Otherwise, partition_by_ast is used for partitioning.
|
||||
///
|
||||
/// order_by_ast - a single expression or a tuple. It is used as a sorting key
|
||||
/// (an ASTExpressionList used for sorting data in parts);
|
||||
/// primary_key_ast - can be nullptr, an expression, or a tuple.
|
||||
/// Used to determine an ASTExpressionList values of which are written in the primary.idx file
|
||||
/// for one row in every `index_granularity` rows to speed up range queries.
|
||||
/// Primary key must be a prefix of the sorting key;
|
||||
/// If it is nullptr, then it will be determined from order_by_ast.
|
||||
///
|
||||
/// require_part_metadata - should checksums.txt and columns.txt exist in the part directory.
|
||||
/// attach - whether the existing table is attached or the new table is created.
|
||||
MergeTreeData(const String & database_, const String & table_,
|
||||
const String & full_path_,
|
||||
const ColumnsDescription & columns_,
|
||||
Context & context_,
|
||||
const ASTPtr & primary_expr_ast_,
|
||||
const ASTPtr & secondary_sort_expr_ast_,
|
||||
const String & date_column_name,
|
||||
const ASTPtr & partition_expr_ast_,
|
||||
const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported.
|
||||
const ASTPtr & partition_by_ast_,
|
||||
const ASTPtr & order_by_ast_,
|
||||
const ASTPtr & primary_key_ast_,
|
||||
const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported.
|
||||
const MergingParams & merging_params_,
|
||||
const MergeTreeSettings & settings_,
|
||||
bool require_part_metadata_,
|
||||
@ -307,7 +315,6 @@ public:
|
||||
/// Load the set of data parts from disk. Call once - immediately after the object is created.
|
||||
void loadDataParts(bool skip_sanity_checks);
|
||||
|
||||
bool supportsSampling() const { return sampling_expression != nullptr; }
|
||||
bool supportsPrewhere() const { return true; }
|
||||
|
||||
bool supportsFinal() const
|
||||
@ -472,12 +479,13 @@ public:
|
||||
|
||||
/// Performs ALTER of the data part, writes the result to temporary files.
|
||||
/// Returns an object allowing to rename temporary files to permanent files.
|
||||
/// If new_primary_key_expr_list is not nullptr, will prepare the new primary.idx file.
|
||||
/// If the number of affected columns is suspiciously high and skip_sanity_checks is false, throws an exception.
|
||||
/// If no data transformations are necessary, returns nullptr.
|
||||
AlterDataPartTransactionPtr alterDataPart(
|
||||
const DataPartPtr & part,
|
||||
const NamesAndTypesList & new_columns,
|
||||
const ASTPtr & new_primary_key,
|
||||
const ASTPtr & new_primary_key_expr_list,
|
||||
bool skip_sanity_checks);
|
||||
|
||||
/// Freezes all parts.
|
||||
@ -489,11 +497,26 @@ public:
|
||||
broken_part_callback(name);
|
||||
}
|
||||
|
||||
bool hasPrimaryKey() const { return !primary_sort_columns.empty(); }
|
||||
ExpressionActionsPtr getPrimaryExpression() const { return primary_expr; }
|
||||
ExpressionActionsPtr getSecondarySortExpression() const { return secondary_sort_expr; } /// may return nullptr
|
||||
Names getPrimarySortColumns() const { return primary_sort_columns; }
|
||||
Names getSortColumns() const { return sort_columns; }
|
||||
/** Get the key expression AST as an ASTExpressionList.
|
||||
* It can be specified in the tuple: (CounterID, Date),
|
||||
* or as one column: CounterID.
|
||||
*/
|
||||
static ASTPtr extractKeyExpressionList(const ASTPtr & node);
|
||||
|
||||
Names getColumnsRequiredForPartitionKey() const { return (partition_key_expr ? partition_key_expr->getRequiredColumns() : Names{}); }
|
||||
|
||||
bool hasSortingKey() const { return !sorting_key_columns.empty(); }
|
||||
bool hasPrimaryKey() const { return !primary_key_columns.empty(); }
|
||||
|
||||
ASTPtr getSortingKeyAST() const { return sorting_key_expr_ast; }
|
||||
ASTPtr getPrimaryKeyAST() const { return primary_key_expr_ast; }
|
||||
|
||||
Names getColumnsRequiredForSortingKey() const { return sorting_key_expr->getRequiredColumns(); }
|
||||
Names getColumnsRequiredForPrimaryKey() const { return primary_key_expr->getRequiredColumns(); }
|
||||
|
||||
bool supportsSampling() const { return sample_by_ast != nullptr; }
|
||||
ASTPtr getSamplingExpression() const { return sample_by_ast; }
|
||||
Names getColumnsRequiredForSampling() const { return columns_required_for_sampling; }
|
||||
|
||||
/// Check that the part is not broken and calculate the checksums for it if they are not present.
|
||||
MutableDataPartPtr loadPartAndFixMetadata(const String & relative_path);
|
||||
@ -543,21 +566,12 @@ public:
|
||||
MergeTreeDataFormatVersion format_version;
|
||||
|
||||
Context & context;
|
||||
const ASTPtr sampling_expression;
|
||||
const size_t index_granularity;
|
||||
|
||||
/// Merging params - what additional actions to perform during merge.
|
||||
const MergingParams merging_params;
|
||||
|
||||
const MergeTreeSettings settings;
|
||||
|
||||
ASTPtr primary_expr_ast;
|
||||
ASTPtr secondary_sort_expr_ast;
|
||||
Block primary_key_sample;
|
||||
DataTypes primary_key_data_types;
|
||||
|
||||
ASTPtr partition_expr_ast;
|
||||
ExpressionActionsPtr partition_expr;
|
||||
bool is_custom_partitioned = false;
|
||||
ExpressionActionsPtr partition_key_expr;
|
||||
Block partition_key_sample;
|
||||
|
||||
ExpressionActionsPtr minmax_idx_expr;
|
||||
@ -565,6 +579,24 @@ public:
|
||||
DataTypes minmax_idx_column_types;
|
||||
Int64 minmax_idx_date_column_pos = -1; /// In a common case minmax index includes a date column.
|
||||
|
||||
/// Names of columns for primary key + secondary sorting columns.
|
||||
Names sorting_key_columns;
|
||||
ASTPtr sorting_key_expr_ast;
|
||||
ExpressionActionsPtr sorting_key_expr;
|
||||
|
||||
/// Names of columns for primary key.
|
||||
Names primary_key_columns;
|
||||
ASTPtr primary_key_expr_ast;
|
||||
ExpressionActionsPtr primary_key_expr;
|
||||
Block primary_key_sample;
|
||||
DataTypes primary_key_data_types;
|
||||
|
||||
String sampling_expr_column_name;
|
||||
Names columns_required_for_sampling;
|
||||
|
||||
const size_t index_granularity;
|
||||
const MergeTreeSettings settings;
|
||||
|
||||
/// Limiting parallel sends per one table, used in DataPartsExchange
|
||||
std::atomic_uint current_table_sends {0};
|
||||
|
||||
@ -574,21 +606,18 @@ public:
|
||||
private:
|
||||
friend struct MergeTreeDataPart;
|
||||
friend class StorageMergeTree;
|
||||
friend class ReplicatedMergeTreeAlterThread;
|
||||
friend class MergeTreeDataMergerMutator;
|
||||
friend class StorageMergeTree;
|
||||
friend class StorageReplicatedMergeTree;
|
||||
friend class MergeTreeDataMergerMutator;
|
||||
friend class ReplicatedMergeTreeAlterThread;
|
||||
friend struct ReplicatedMergeTreeTableMetadata;
|
||||
|
||||
ASTPtr partition_by_ast;
|
||||
ASTPtr order_by_ast;
|
||||
ASTPtr primary_key_ast;
|
||||
ASTPtr sample_by_ast;
|
||||
|
||||
bool require_part_metadata;
|
||||
|
||||
ExpressionActionsPtr primary_expr;
|
||||
/// Additional expression for sorting (of rows with the same primary keys).
|
||||
ExpressionActionsPtr secondary_sort_expr;
|
||||
/// Names of columns for primary key. Is the prefix of sort_columns.
|
||||
Names primary_sort_columns;
|
||||
/// Names of columns for primary key + secondary sorting columns.
|
||||
Names sort_columns;
|
||||
|
||||
String database_name;
|
||||
String table_name;
|
||||
String full_path;
|
||||
@ -690,7 +719,7 @@ private:
|
||||
/// The same for clearOldTemporaryDirectories.
|
||||
std::mutex clear_old_temporary_directories_mutex;
|
||||
|
||||
void initPrimaryKey();
|
||||
void setPrimaryKeyAndColumns(const ASTPtr & new_order_by_ast, ASTPtr new_primary_key_ast, const ColumnsDescription & new_columns, bool only_check = false);
|
||||
|
||||
void initPartitionKey();
|
||||
|
||||
|
@ -333,20 +333,15 @@ MergeTreeData::DataPartsVector MergeTreeDataMergerMutator::selectAllPartsFromPar
|
||||
|
||||
|
||||
/// PK columns are sorted and merged, ordinary columns are gathered using info from merge step
|
||||
static void extractMergingAndGatheringColumns(const NamesAndTypesList & all_columns,
|
||||
const ExpressionActionsPtr & primary_key_expressions, const ExpressionActionsPtr & secondary_key_expressions,
|
||||
static void extractMergingAndGatheringColumns(
|
||||
const NamesAndTypesList & all_columns,
|
||||
const ExpressionActionsPtr & sorting_key_expr,
|
||||
const MergeTreeData::MergingParams & merging_params,
|
||||
NamesAndTypesList & gathering_columns, Names & gathering_column_names,
|
||||
NamesAndTypesList & merging_columns, Names & merging_column_names
|
||||
)
|
||||
NamesAndTypesList & merging_columns, Names & merging_column_names)
|
||||
{
|
||||
Names primary_key_columns_vec = primary_key_expressions->getRequiredColumns();
|
||||
std::set<String> key_columns(primary_key_columns_vec.cbegin(), primary_key_columns_vec.cend());
|
||||
if (secondary_key_expressions)
|
||||
{
|
||||
Names secondary_key_columns_vec = secondary_key_expressions->getRequiredColumns();
|
||||
key_columns.insert(secondary_key_columns_vec.begin(), secondary_key_columns_vec.end());
|
||||
}
|
||||
Names sort_key_columns_vec = sorting_key_expr->getRequiredColumns();
|
||||
std::set<String> key_columns(sort_key_columns_vec.cbegin(), sort_key_columns_vec.cend());
|
||||
|
||||
/// Force sign column for Collapsing mode
|
||||
if (merging_params.mode == MergeTreeData::MergingParams::Collapsing)
|
||||
@ -366,20 +361,18 @@ static void extractMergingAndGatheringColumns(const NamesAndTypesList & all_colu
|
||||
|
||||
/// TODO: also force "summing" and "aggregating" columns to make Horizontal merge only for such columns
|
||||
|
||||
for (auto & column : all_columns)
|
||||
for (const auto & column : all_columns)
|
||||
{
|
||||
auto it = std::find(key_columns.cbegin(), key_columns.cend(), column.name);
|
||||
|
||||
if (key_columns.end() == it)
|
||||
{
|
||||
gathering_columns.emplace_back(column);
|
||||
gathering_column_names.emplace_back(column.name);
|
||||
}
|
||||
else
|
||||
if (key_columns.count(column.name))
|
||||
{
|
||||
merging_columns.emplace_back(column);
|
||||
merging_column_names.emplace_back(column.name);
|
||||
}
|
||||
else
|
||||
{
|
||||
gathering_columns.emplace_back(column);
|
||||
gathering_column_names.emplace_back(column.name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -555,8 +548,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
|
||||
NamesAndTypesList gathering_columns, merging_columns;
|
||||
Names gathering_column_names, merging_column_names;
|
||||
extractMergingAndGatheringColumns(all_columns, data.getPrimaryExpression(), data.getSecondarySortExpression()
|
||||
, data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names);
|
||||
extractMergingAndGatheringColumns(
|
||||
all_columns, data.sorting_key_expr,
|
||||
data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names);
|
||||
|
||||
MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared<MergeTreeData::DataPart>(
|
||||
data, future_part.name, future_part.part_info);
|
||||
@ -636,12 +630,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
|
||||
if (data.hasPrimaryKey())
|
||||
src_streams.emplace_back(std::make_shared<MaterializingBlockInputStream>(
|
||||
std::make_shared<ExpressionBlockInputStream>(BlockInputStreamPtr(std::move(input)), data.getPrimaryExpression())));
|
||||
std::make_shared<ExpressionBlockInputStream>(BlockInputStreamPtr(std::move(input)), data.sorting_key_expr)));
|
||||
else
|
||||
src_streams.emplace_back(std::move(input));
|
||||
}
|
||||
|
||||
Names sort_columns = data.getSortColumns();
|
||||
Names sort_columns = data.sorting_key_columns;
|
||||
SortDescription sort_description;
|
||||
size_t sort_columns_size = sort_columns.size();
|
||||
sort_description.reserve(sort_columns_size);
|
||||
@ -908,7 +902,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
|
||||
if (data.hasPrimaryKey())
|
||||
in = std::make_shared<MaterializingBlockInputStream>(
|
||||
std::make_shared<ExpressionBlockInputStream>(in, data.getPrimaryExpression()));
|
||||
std::make_shared<ExpressionBlockInputStream>(in, data.primary_key_expr));
|
||||
|
||||
MergeTreeDataPart::MinMaxIndex minmax_idx;
|
||||
|
||||
|
@ -448,7 +448,7 @@ void MergeTreeDataPart::loadIndex()
|
||||
.getSize() / MERGE_TREE_MARK_SIZE;
|
||||
}
|
||||
|
||||
size_t key_size = storage.primary_sort_columns.size();
|
||||
size_t key_size = storage.primary_key_columns.size();
|
||||
|
||||
if (key_size)
|
||||
{
|
||||
@ -631,7 +631,7 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata)
|
||||
|
||||
if (!checksums.empty())
|
||||
{
|
||||
if (!storage.primary_sort_columns.empty() && !checksums.files.count("primary.idx"))
|
||||
if (!storage.primary_key_columns.empty() && !checksums.files.count("primary.idx"))
|
||||
throw Exception("No checksum for primary.idx", ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
if (require_part_metadata)
|
||||
@ -659,7 +659,7 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata)
|
||||
if (!checksums.files.count("count.txt"))
|
||||
throw Exception("No checksum for count.txt", ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
if (storage.partition_expr && !checksums.files.count("partition.dat"))
|
||||
if (storage.partition_key_expr && !checksums.files.count("partition.dat"))
|
||||
throw Exception("No checksum for partition.dat", ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
if (!isEmpty())
|
||||
@ -685,14 +685,14 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata)
|
||||
};
|
||||
|
||||
/// Check that the primary key index is not empty.
|
||||
if (!storage.primary_sort_columns.empty())
|
||||
if (!storage.primary_key_columns.empty())
|
||||
check_file_not_empty(path + "primary.idx");
|
||||
|
||||
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
{
|
||||
check_file_not_empty(path + "count.txt");
|
||||
|
||||
if (storage.partition_expr)
|
||||
if (storage.partition_key_expr)
|
||||
check_file_not_empty(path + "partition.dat");
|
||||
|
||||
for (const String & col_name : storage.minmax_idx_columns)
|
||||
|
@ -208,16 +208,16 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts(
|
||||
data.check(real_column_names);
|
||||
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
Names primary_sort_columns = data.getPrimarySortColumns();
|
||||
Names primary_key_columns = data.primary_key_columns;
|
||||
|
||||
KeyCondition key_condition(query_info, context, primary_sort_columns, data.getPrimaryExpression());
|
||||
KeyCondition key_condition(query_info, context, primary_key_columns, data.primary_key_expr);
|
||||
|
||||
if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue())
|
||||
{
|
||||
std::stringstream exception_message;
|
||||
exception_message << "Primary key (";
|
||||
for (size_t i = 0, size = primary_sort_columns.size(); i < size; ++i)
|
||||
exception_message << (i == 0 ? "" : ", ") << primary_sort_columns[i];
|
||||
for (size_t i = 0, size = primary_key_columns.size(); i < size; ++i)
|
||||
exception_message << (i == 0 ? "" : ", ") << primary_key_columns[i];
|
||||
exception_message << ") is not used and setting 'force_primary_key' is set.";
|
||||
|
||||
throw Exception(exception_message.str(), ErrorCodes::INDEX_NOT_USED);
|
||||
@ -369,14 +369,14 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts(
|
||||
|
||||
if (use_sampling)
|
||||
{
|
||||
if (!data.sampling_expression)
|
||||
if (!data.supportsSampling())
|
||||
throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED);
|
||||
|
||||
if (sample_factor_column_queried && relative_sample_size != RelativeSize(0))
|
||||
used_sample_factor = 1.0 / boost::rational_cast<Float64>(relative_sample_size);
|
||||
|
||||
RelativeSize size_of_universum = 0;
|
||||
DataTypePtr type = data.getPrimaryExpression()->getSampleBlock().getByName(data.sampling_expression->getColumnName()).type;
|
||||
DataTypePtr type = data.primary_key_sample.getByName(data.sampling_expr_column_name).type;
|
||||
|
||||
if (typeid_cast<const DataTypeUInt64 *>(type.get()))
|
||||
size_of_universum = RelativeSize(std::numeric_limits<UInt64>::max()) + RelativeSize(1);
|
||||
@ -443,11 +443,11 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts(
|
||||
|
||||
if (has_lower_limit)
|
||||
{
|
||||
if (!key_condition.addCondition(data.sampling_expression->getColumnName(), Range::createLeftBounded(lower, true)))
|
||||
if (!key_condition.addCondition(data.sampling_expr_column_name, Range::createLeftBounded(lower, true)))
|
||||
throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
ASTPtr args = std::make_shared<ASTExpressionList>();
|
||||
args->children.push_back(data.sampling_expression);
|
||||
args->children.push_back(data.getSamplingExpression());
|
||||
args->children.push_back(std::make_shared<ASTLiteral>(lower));
|
||||
|
||||
lower_function = std::make_shared<ASTFunction>();
|
||||
@ -460,11 +460,11 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts(
|
||||
|
||||
if (has_upper_limit)
|
||||
{
|
||||
if (!key_condition.addCondition(data.sampling_expression->getColumnName(), Range::createRightBounded(upper, false)))
|
||||
if (!key_condition.addCondition(data.sampling_expr_column_name, Range::createRightBounded(upper, false)))
|
||||
throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
ASTPtr args = std::make_shared<ASTExpressionList>();
|
||||
args->children.push_back(data.sampling_expression);
|
||||
args->children.push_back(data.getSamplingExpression());
|
||||
args->children.push_back(std::make_shared<ASTLiteral>(upper));
|
||||
|
||||
upper_function = std::make_shared<ASTFunction>();
|
||||
@ -491,7 +491,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts(
|
||||
auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, available_real_columns);
|
||||
filter_expression = ExpressionAnalyzer(filter_function, syntax_result, context).getActions(false);
|
||||
|
||||
/// Add columns needed for `sampling_expression` to `column_names_to_read`.
|
||||
/// Add columns needed for `sample_by_ast` to `column_names_to_read`.
|
||||
std::vector<String> add_columns = filter_expression->getRequiredColumns();
|
||||
column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end());
|
||||
std::sort(column_names_to_read.begin(), column_names_to_read.end());
|
||||
@ -552,8 +552,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts(
|
||||
|
||||
if (select.final())
|
||||
{
|
||||
/// Add columns needed to calculate primary key and the sign.
|
||||
std::vector<String> add_columns = data.getPrimaryExpression()->getRequiredColumns();
|
||||
/// Add columns needed to calculate the sorting expression and the sign.
|
||||
std::vector<String> add_columns = data.sorting_key_expr->getRequiredColumns();
|
||||
column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end());
|
||||
|
||||
if (!data.merging_params.sign_column.empty())
|
||||
@ -781,10 +781,10 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal
|
||||
prewhere_info, true, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size, true,
|
||||
virt_columns, part.part_index_in_query);
|
||||
|
||||
to_merge.emplace_back(std::make_shared<ExpressionBlockInputStream>(source_stream, data.getPrimaryExpression()));
|
||||
to_merge.emplace_back(std::make_shared<ExpressionBlockInputStream>(source_stream, data.sorting_key_expr));
|
||||
}
|
||||
|
||||
Names sort_columns = data.getSortColumns();
|
||||
Names sort_columns = data.sorting_key_columns;
|
||||
SortDescription sort_description;
|
||||
size_t sort_columns_size = sort_columns.size();
|
||||
sort_description.reserve(sort_columns_size);
|
||||
|
@ -76,14 +76,14 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block
|
||||
data.check(block, true);
|
||||
block.checkNumberOfRows();
|
||||
|
||||
if (!data.partition_expr) /// Table is not partitioned.
|
||||
if (!data.partition_key_expr) /// Table is not partitioned.
|
||||
{
|
||||
result.emplace_back(Block(block), Row());
|
||||
return result;
|
||||
}
|
||||
|
||||
Block block_copy = block;
|
||||
data.partition_expr->execute(block_copy);
|
||||
data.partition_key_expr->execute(block_copy);
|
||||
|
||||
ColumnRawPtrs partition_columns;
|
||||
partition_columns.reserve(data.partition_key_sample.columns());
|
||||
@ -180,15 +180,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
||||
dir.createDirectories();
|
||||
|
||||
/// If we need to calculate some columns to sort.
|
||||
if (data.hasPrimaryKey())
|
||||
{
|
||||
data.getPrimaryExpression()->execute(block);
|
||||
auto secondary_sort_expr = data.getSecondarySortExpression();
|
||||
if (secondary_sort_expr)
|
||||
secondary_sort_expr->execute(block);
|
||||
}
|
||||
if (data.hasSortingKey())
|
||||
data.sorting_key_expr->execute(block);
|
||||
|
||||
Names sort_columns = data.getSortColumns();
|
||||
Names sort_columns = data.sorting_key_columns;
|
||||
SortDescription sort_description;
|
||||
size_t sort_columns_size = sort_columns.size();
|
||||
sort_description.reserve(sort_columns_size);
|
||||
@ -201,7 +196,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
||||
/// Sort.
|
||||
IColumn::Permutation * perm_ptr = nullptr;
|
||||
IColumn::Permutation perm;
|
||||
if (data.hasPrimaryKey())
|
||||
if (!sort_description.empty())
|
||||
{
|
||||
if (!isAlreadySorted(block, sort_description))
|
||||
{
|
||||
|
@ -115,7 +115,7 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe
|
||||
|
||||
void MergeTreePartition::load(const MergeTreeData & storage, const String & part_path)
|
||||
{
|
||||
if (!storage.partition_expr)
|
||||
if (!storage.partition_key_expr)
|
||||
return;
|
||||
|
||||
ReadBufferFromFile file = openForReading(part_path + "partition.dat");
|
||||
@ -126,7 +126,7 @@ void MergeTreePartition::load(const MergeTreeData & storage, const String & part
|
||||
|
||||
void MergeTreePartition::store(const MergeTreeData & storage, const String & part_path, MergeTreeDataPartChecksums & checksums) const
|
||||
{
|
||||
if (!storage.partition_expr)
|
||||
if (!storage.partition_key_expr)
|
||||
return;
|
||||
|
||||
WriteBufferFromFile out(part_path + "partition.dat");
|
||||
|
@ -40,7 +40,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer(
|
||||
const MergeTreeData & data,
|
||||
const Names & column_names,
|
||||
Logger * log)
|
||||
: primary_key_columns{ext::collection_cast<std::unordered_set>(data.getPrimarySortColumns())},
|
||||
: primary_key_columns{ext::collection_cast<std::unordered_set>(data.primary_key_columns)},
|
||||
table_columns{ext::map<std::unordered_set>(data.getColumns().getAllPhysical(),
|
||||
[] (const NameAndTypePair & col) { return col.name; })},
|
||||
block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)},
|
||||
|
@ -402,31 +402,31 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
|
||||
/// The set of written offset columns so that you do not write shared offsets of nested structures columns several times
|
||||
WrittenOffsetColumns offset_columns;
|
||||
|
||||
auto sort_columns = storage.getPrimarySortColumns();
|
||||
auto primary_key_column_names = storage.primary_key_columns;
|
||||
|
||||
/// Here we will add the columns related to the Primary Key, then write the index.
|
||||
std::vector<ColumnWithTypeAndName> primary_columns(sort_columns.size());
|
||||
std::map<String, size_t> primary_columns_name_to_position;
|
||||
std::vector<ColumnWithTypeAndName> primary_key_columns(primary_key_column_names.size());
|
||||
std::map<String, size_t> primary_key_column_name_to_position;
|
||||
|
||||
for (size_t i = 0, size = sort_columns.size(); i < size; ++i)
|
||||
for (size_t i = 0, size = primary_key_column_names.size(); i < size; ++i)
|
||||
{
|
||||
const auto & name = sort_columns[i];
|
||||
const auto & name = primary_key_column_names[i];
|
||||
|
||||
if (!primary_columns_name_to_position.emplace(name, i).second)
|
||||
if (!primary_key_column_name_to_position.emplace(name, i).second)
|
||||
throw Exception("Primary key contains duplicate columns", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
primary_columns[i] = block.getByName(name);
|
||||
primary_key_columns[i] = block.getByName(name);
|
||||
|
||||
/// Reorder primary key columns in advance and add them to `primary_columns`.
|
||||
/// Reorder primary key columns in advance and add them to `primary_key_columns`.
|
||||
if (permutation)
|
||||
primary_columns[i].column = primary_columns[i].column->permute(*permutation, 0);
|
||||
primary_key_columns[i].column = primary_key_columns[i].column->permute(*permutation, 0);
|
||||
}
|
||||
|
||||
if (index_columns.empty())
|
||||
{
|
||||
index_columns.resize(sort_columns.size());
|
||||
for (size_t i = 0, size = sort_columns.size(); i < size; ++i)
|
||||
index_columns[i] = primary_columns[i].column->cloneEmpty();
|
||||
index_columns.resize(primary_key_column_names.size());
|
||||
for (size_t i = 0, size = primary_key_column_names.size(); i < size; ++i)
|
||||
index_columns[i] = primary_key_columns[i].column->cloneEmpty();
|
||||
}
|
||||
|
||||
if (serialization_states.empty())
|
||||
@ -451,10 +451,10 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
|
||||
|
||||
if (permutation)
|
||||
{
|
||||
auto primary_column_it = primary_columns_name_to_position.find(it->name);
|
||||
if (primary_columns_name_to_position.end() != primary_column_it)
|
||||
auto primary_column_it = primary_key_column_name_to_position.find(it->name);
|
||||
if (primary_key_column_name_to_position.end() != primary_column_it)
|
||||
{
|
||||
auto & primary_column = *primary_columns[primary_column_it->second].column;
|
||||
auto & primary_column = *primary_key_columns[primary_column_it->second].column;
|
||||
writeData(column.name, *column.type, primary_column, offset_columns, false, serialization_states[i]);
|
||||
}
|
||||
else
|
||||
@ -486,11 +486,11 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
|
||||
{
|
||||
if (storage.hasPrimaryKey())
|
||||
{
|
||||
for (size_t j = 0, size = primary_columns.size(); j < size; ++j)
|
||||
for (size_t j = 0, size = primary_key_columns.size(); j < size; ++j)
|
||||
{
|
||||
const IColumn & primary_column = *primary_columns[j].column.get();
|
||||
const IColumn & primary_column = *primary_key_columns[j].column.get();
|
||||
index_columns[j]->insertFrom(primary_column, i);
|
||||
primary_columns[j].type->serializeBinary(primary_column, i, *index_stream);
|
||||
primary_key_columns[j].type->serializeBinary(primary_column, i, *index_stream);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1,21 +1,29 @@
|
||||
#include <memory>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeAlterThread.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
#include <Interpreters/InterpreterAlterQuery.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeAlterThread.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
|
||||
#include <memory>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_FOUND_NODE;
|
||||
}
|
||||
|
||||
static const auto ALTER_ERROR_SLEEP_MS = 10 * 1000;
|
||||
|
||||
|
||||
ReplicatedMergeTreeAlterThread::ReplicatedMergeTreeAlterThread(StorageReplicatedMergeTree & storage_)
|
||||
: storage(storage_)
|
||||
, zk_node_cache([&] { return storage.getZooKeeper(); })
|
||||
, log_name(storage.database_name + "." + storage.table_name + " (ReplicatedMergeTreeAlterThread)")
|
||||
, log(&Logger::get(log_name))
|
||||
{
|
||||
@ -53,136 +61,147 @@ void ReplicatedMergeTreeAlterThread::run()
|
||||
|
||||
auto zookeeper = storage.getZooKeeper();
|
||||
|
||||
Coordination::Stat stat;
|
||||
const String columns_str = zookeeper->getWatch(storage.zookeeper_path + "/columns", &stat, task->getWatchCallback());
|
||||
String columns_path = storage.zookeeper_path + "/columns";
|
||||
auto columns_znode = zk_node_cache.get(columns_path, task->getWatchCallback());
|
||||
if (!columns_znode.exists)
|
||||
throw Exception(columns_path + " doesn't exist", ErrorCodes::NOT_FOUND_NODE);
|
||||
int32_t columns_version = columns_znode.stat.version;
|
||||
|
||||
String metadata_path = storage.zookeeper_path + "/metadata";
|
||||
auto metadata_znode = zk_node_cache.get(metadata_path, task->getWatchCallback());
|
||||
if (!metadata_znode.exists)
|
||||
throw Exception(metadata_path + " doesn't exist", ErrorCodes::NOT_FOUND_NODE);
|
||||
int32_t metadata_version = metadata_znode.stat.version;
|
||||
|
||||
const bool changed_columns_version = (columns_version != storage.columns_version);
|
||||
const bool changed_metadata_version = (metadata_version != storage.metadata_version);
|
||||
|
||||
if (!(changed_columns_version || changed_metadata_version || force_recheck_parts))
|
||||
return;
|
||||
|
||||
const String & columns_str = columns_znode.contents;
|
||||
auto columns_in_zk = ColumnsDescription::parse(columns_str);
|
||||
|
||||
bool changed_version = (stat.version != storage.columns_version);
|
||||
const String & metadata_str = metadata_znode.contents;
|
||||
auto metadata_in_zk = ReplicatedMergeTreeTableMetadata::parse(metadata_str);
|
||||
auto metadata_diff = ReplicatedMergeTreeTableMetadata(storage.data).checkAndFindDiff(metadata_in_zk, /* allow_alter = */ true);
|
||||
|
||||
/// If you need to lock table structure, then suspend merges.
|
||||
ActionLock merge_blocker = storage.merger_mutator.actions_blocker.cancel();
|
||||
|
||||
MergeTreeData::DataParts parts;
|
||||
|
||||
/// If metadata nodes have changed, we will update table structure locally.
|
||||
if (changed_columns_version || changed_metadata_version)
|
||||
{
|
||||
/// If you need to lock table structure, then suspend merges.
|
||||
ActionLock merge_blocker;
|
||||
/// Temporarily cancel part checks to avoid locking for long time.
|
||||
auto temporarily_stop_part_checks = storage.part_check_thread.temporarilyStop();
|
||||
|
||||
if (changed_version || force_recheck_parts)
|
||||
merge_blocker = storage.merger_mutator.actions_blocker.cancel();
|
||||
/// Temporarily cancel parts sending
|
||||
ActionLock data_parts_exchange_blocker;
|
||||
if (storage.data_parts_exchange_endpoint_holder)
|
||||
data_parts_exchange_blocker = storage.data_parts_exchange_endpoint_holder->getBlocker().cancel();
|
||||
|
||||
MergeTreeData::DataParts parts;
|
||||
/// Temporarily cancel part fetches
|
||||
auto fetches_blocker = storage.fetcher.blocker.cancel();
|
||||
|
||||
/// If columns description has changed, we will update table structure locally.
|
||||
if (changed_version)
|
||||
LOG_INFO(log, "Version of metadata nodes in ZooKeeper changed. Waiting for structure write lock.");
|
||||
|
||||
auto table_lock = storage.lockStructureForAlter();
|
||||
|
||||
if (columns_in_zk == storage.getColumns() && metadata_diff.empty())
|
||||
{
|
||||
/// Temporarily cancel part checks to avoid locking for long time.
|
||||
auto temporarily_stop_part_checks = storage.part_check_thread.temporarilyStop();
|
||||
LOG_INFO(log, "Metadata nodes changed in ZooKeeper, but their contents didn't change. "
|
||||
"Most probably it is a cyclic ALTER.");
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_INFO(log, "Metadata changed in ZooKeeper. Applying changes locally.");
|
||||
|
||||
/// Temporarily cancel parts sending
|
||||
ActionLock data_parts_exchange_blocker;
|
||||
if (storage.data_parts_exchange_endpoint_holder)
|
||||
data_parts_exchange_blocker = storage.data_parts_exchange_endpoint_holder->getBlocker().cancel();
|
||||
storage.setTableStructure(std::move(columns_in_zk), metadata_diff);
|
||||
|
||||
/// Temporarily cancel part fetches
|
||||
auto fetches_blocker = storage.fetcher.blocker.cancel();
|
||||
LOG_INFO(log, "Applied changes to the metadata of the table.");
|
||||
}
|
||||
|
||||
LOG_INFO(log, "Changed version of 'columns' node in ZooKeeper. Waiting for structure write lock.");
|
||||
/// You need to get a list of parts under table lock to avoid race condition with merge.
|
||||
parts = storage.data.getDataParts();
|
||||
|
||||
auto table_lock = storage.lockStructureForAlter();
|
||||
storage.columns_version = columns_version;
|
||||
storage.metadata_version = metadata_version;
|
||||
}
|
||||
|
||||
if (columns_in_zk != storage.getColumns())
|
||||
{
|
||||
LOG_INFO(log, "Columns list changed in ZooKeeper. Applying changes locally.");
|
||||
/// Update parts.
|
||||
if (changed_columns_version || force_recheck_parts)
|
||||
{
|
||||
auto table_lock = storage.lockStructure(false);
|
||||
|
||||
storage.context.getDatabase(storage.database_name)->alterTable(
|
||||
storage.context, storage.table_name, columns_in_zk, {});
|
||||
storage.setColumns(std::move(columns_in_zk));
|
||||
if (changed_columns_version)
|
||||
LOG_INFO(log, "ALTER-ing parts");
|
||||
|
||||
/// Reinitialize primary key because primary key column types might have changed.
|
||||
storage.data.initPrimaryKey();
|
||||
int changed_parts = 0;
|
||||
|
||||
LOG_INFO(log, "Applied changes to table.");
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_INFO(log, "Columns version changed in ZooKeeper, but data wasn't changed. It's like cyclic ALTERs.");
|
||||
}
|
||||
|
||||
/// You need to get a list of parts under table lock to avoid race condition with merge.
|
||||
if (!changed_columns_version)
|
||||
parts = storage.data.getDataParts();
|
||||
|
||||
storage.columns_version = stat.version;
|
||||
}
|
||||
const auto columns_for_parts = storage.getColumns().getAllPhysical();
|
||||
|
||||
/// Update parts.
|
||||
if (changed_version || force_recheck_parts)
|
||||
for (const MergeTreeData::DataPartPtr & part : parts)
|
||||
{
|
||||
auto table_lock = storage.lockStructure(false);
|
||||
/// Update the part and write result to temporary files.
|
||||
/// TODO: You can skip checking for too large changes if ZooKeeper has, for example,
|
||||
/// node /flags/force_alter.
|
||||
auto transaction = storage.data.alterDataPart(part, columns_for_parts, nullptr, false);
|
||||
|
||||
if (changed_version)
|
||||
LOG_INFO(log, "ALTER-ing parts");
|
||||
if (!transaction)
|
||||
continue;
|
||||
|
||||
int changed_parts = 0;
|
||||
++changed_parts;
|
||||
|
||||
if (!changed_version)
|
||||
parts = storage.data.getDataParts();
|
||||
/// Update part metadata in ZooKeeper.
|
||||
Coordination::Requests ops;
|
||||
ops.emplace_back(zkutil::makeSetRequest(
|
||||
storage.replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
|
||||
ops.emplace_back(zkutil::makeSetRequest(
|
||||
storage.replica_path + "/parts/" + part->name + "/checksums",
|
||||
storage.getChecksumsForZooKeeper(transaction->getNewChecksums()),
|
||||
-1));
|
||||
|
||||
const auto columns_for_parts = storage.getColumns().getAllPhysical();
|
||||
|
||||
for (const MergeTreeData::DataPartPtr & part : parts)
|
||||
try
|
||||
{
|
||||
/// Update the part and write result to temporary files.
|
||||
/// TODO: You can skip checking for too large changes if ZooKeeper has, for example,
|
||||
/// node /flags/force_alter.
|
||||
auto transaction = storage.data.alterDataPart(
|
||||
part, columns_for_parts, storage.data.primary_expr_ast, false);
|
||||
zookeeper->multi(ops);
|
||||
}
|
||||
catch (const Coordination::Exception & e)
|
||||
{
|
||||
/// The part does not exist in ZK. We will add to queue for verification - maybe the part is superfluous, and it must be removed locally.
|
||||
if (e.code == Coordination::ZNONODE)
|
||||
storage.enqueuePartForCheck(part->name);
|
||||
|
||||
if (!transaction)
|
||||
continue;
|
||||
|
||||
++changed_parts;
|
||||
|
||||
/// Update part metadata in ZooKeeper.
|
||||
Coordination::Requests ops;
|
||||
ops.emplace_back(zkutil::makeSetRequest(
|
||||
storage.replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
|
||||
ops.emplace_back(zkutil::makeSetRequest(
|
||||
storage.replica_path + "/parts/" + part->name + "/checksums",
|
||||
storage.getChecksumsForZooKeeper(transaction->getNewChecksums()),
|
||||
-1));
|
||||
|
||||
try
|
||||
{
|
||||
zookeeper->multi(ops);
|
||||
}
|
||||
catch (const Coordination::Exception & e)
|
||||
{
|
||||
/// The part does not exist in ZK. We will add to queue for verification - maybe the part is superfluous, and it must be removed locally.
|
||||
if (e.code == Coordination::ZNONODE)
|
||||
storage.enqueuePartForCheck(part->name);
|
||||
|
||||
throw;
|
||||
}
|
||||
|
||||
/// Apply file changes.
|
||||
transaction->commit();
|
||||
throw;
|
||||
}
|
||||
|
||||
/// Columns sizes could be quietly changed in case of MODIFY/ADD COLUMN
|
||||
storage.data.recalculateColumnSizes();
|
||||
|
||||
/// List of columns for a specific replica.
|
||||
zookeeper->set(storage.replica_path + "/columns", columns_str);
|
||||
|
||||
if (changed_version)
|
||||
{
|
||||
if (changed_parts != 0)
|
||||
LOG_INFO(log, "ALTER-ed " << changed_parts << " parts");
|
||||
else
|
||||
LOG_INFO(log, "No parts ALTER-ed");
|
||||
}
|
||||
|
||||
force_recheck_parts = false;
|
||||
/// Apply file changes.
|
||||
transaction->commit();
|
||||
}
|
||||
|
||||
/// It's important that parts and merge_blocker are destroyed before the wait.
|
||||
/// Columns sizes could be quietly changed in case of MODIFY/ADD COLUMN
|
||||
storage.data.recalculateColumnSizes();
|
||||
|
||||
if (changed_columns_version)
|
||||
{
|
||||
if (changed_parts != 0)
|
||||
LOG_INFO(log, "ALTER-ed " << changed_parts << " parts");
|
||||
else
|
||||
LOG_INFO(log, "No parts ALTER-ed");
|
||||
}
|
||||
}
|
||||
|
||||
/// Update metadata ZK nodes for a specific replica.
|
||||
if (changed_columns_version || force_recheck_parts)
|
||||
zookeeper->set(storage.replica_path + "/columns", columns_str);
|
||||
if (changed_metadata_version || force_recheck_parts)
|
||||
zookeeper->set(storage.replica_path + "/metadata", metadata_str);
|
||||
|
||||
force_recheck_parts = false;
|
||||
}
|
||||
catch (const Coordination::Exception & e)
|
||||
{
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <thread>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Common/ZooKeeper/Types.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperNodeCache.h>
|
||||
#include <Core/Types.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
@ -31,6 +32,7 @@ private:
|
||||
void run();
|
||||
|
||||
StorageReplicatedMergeTree & storage;
|
||||
zkutil::ZooKeeperNodeCache zk_node_cache;
|
||||
String log_name;
|
||||
Logger * log;
|
||||
BackgroundSchedulePool::TaskHolder task;
|
||||
|
181
dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp
Normal file
181
dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp
Normal file
@ -0,0 +1,181 @@
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int METADATA_MISMATCH;
|
||||
}
|
||||
|
||||
static String formattedAST(const ASTPtr & ast)
|
||||
{
|
||||
if (!ast)
|
||||
return "";
|
||||
std::stringstream ss;
|
||||
formatAST(*ast, ss, false, true);
|
||||
return ss.str();
|
||||
}
|
||||
|
||||
ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTreeData & data)
|
||||
{
|
||||
if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos];
|
||||
|
||||
sampling_expression = formattedAST(data.sample_by_ast);
|
||||
index_granularity = data.index_granularity;
|
||||
merging_params_mode = static_cast<int>(data.merging_params.mode);
|
||||
sign_column = data.merging_params.sign_column;
|
||||
|
||||
if (!data.primary_key_ast)
|
||||
primary_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.order_by_ast));
|
||||
else
|
||||
{
|
||||
primary_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.primary_key_ast));
|
||||
sorting_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.order_by_ast));
|
||||
}
|
||||
|
||||
data_format_version = data.format_version;
|
||||
|
||||
if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
partition_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.partition_by_ast));
|
||||
}
|
||||
|
||||
void ReplicatedMergeTreeTableMetadata::write(WriteBuffer & out) const
|
||||
{
|
||||
out << "metadata format version: 1" << "\n"
|
||||
<< "date column: " << date_column << "\n"
|
||||
<< "sampling expression: " << sampling_expression << "\n"
|
||||
<< "index granularity: " << index_granularity << "\n"
|
||||
<< "mode: " << merging_params_mode << "\n"
|
||||
<< "sign column: " << sign_column << "\n"
|
||||
<< "primary key: " << primary_key << "\n";
|
||||
|
||||
if (data_format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
{
|
||||
out << "data format version: " << data_format_version.toUnderType() << "\n"
|
||||
<< "partition key: " << partition_key << "\n";
|
||||
}
|
||||
|
||||
if (!sorting_key.empty())
|
||||
out << "sorting key: " << sorting_key << "\n";
|
||||
}
|
||||
|
||||
String ReplicatedMergeTreeTableMetadata::toString() const
|
||||
{
|
||||
WriteBufferFromOwnString out;
|
||||
write(out);
|
||||
return out.str();
|
||||
}
|
||||
|
||||
void ReplicatedMergeTreeTableMetadata::read(ReadBuffer & in)
|
||||
{
|
||||
in >> "metadata format version: 1\n";
|
||||
in >> "date column: " >> date_column >> "\n";
|
||||
in >> "sampling expression: " >> sampling_expression >> "\n";
|
||||
in >> "index granularity: " >> index_granularity >> "\n";
|
||||
in >> "mode: " >> merging_params_mode >> "\n";
|
||||
in >> "sign column: " >> sign_column >> "\n";
|
||||
in >> "primary key: " >> primary_key >> "\n";
|
||||
|
||||
if (in.eof())
|
||||
data_format_version = 0;
|
||||
else
|
||||
in >> "data format version: " >> data_format_version.toUnderType() >> "\n";
|
||||
|
||||
if (data_format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
in >> "partition key: " >> partition_key >> "\n";
|
||||
|
||||
if (checkString("sorting key: ", in))
|
||||
in >> sorting_key >> "\n";
|
||||
}
|
||||
|
||||
ReplicatedMergeTreeTableMetadata ReplicatedMergeTreeTableMetadata::parse(const String & s)
|
||||
{
|
||||
ReplicatedMergeTreeTableMetadata metadata;
|
||||
ReadBufferFromString buf(s);
|
||||
metadata.read(buf);
|
||||
return metadata;
|
||||
}
|
||||
|
||||
ReplicatedMergeTreeTableMetadata::Diff
|
||||
ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTableMetadata & from_zk, bool allow_alter) const
|
||||
{
|
||||
Diff diff;
|
||||
|
||||
if (data_format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
{
|
||||
if (date_column != from_zk.date_column)
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in date index column."
|
||||
" Stored in ZooKeeper: " + from_zk.date_column + ", local: " + date_column,
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
}
|
||||
else if (!from_zk.date_column.empty())
|
||||
throw Exception(
|
||||
"Existing table metadata in ZooKeeper differs in date index column."
|
||||
" Stored in ZooKeeper: " + from_zk.date_column + ", local is custom-partitioned.",
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
|
||||
if (sampling_expression != from_zk.sampling_expression)
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in sample expression."
|
||||
" Stored in ZooKeeper: " + from_zk.sampling_expression + ", local: " + sampling_expression,
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
|
||||
if (index_granularity != from_zk.index_granularity)
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in index granularity."
|
||||
" Stored in ZooKeeper: " + DB::toString(from_zk.index_granularity) + ", local: " + DB::toString(index_granularity),
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
|
||||
if (merging_params_mode != from_zk.merging_params_mode)
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in mode of merge operation."
|
||||
" Stored in ZooKeeper: " + DB::toString(from_zk.merging_params_mode) + ", local: "
|
||||
+ DB::toString(merging_params_mode),
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
|
||||
if (sign_column != from_zk.sign_column)
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in sign column."
|
||||
" Stored in ZooKeeper: " + from_zk.sign_column + ", local: " + sign_column,
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
|
||||
/// NOTE: You can make a less strict check of match expressions so that tables do not break from small changes
|
||||
/// in formatAST code.
|
||||
if (primary_key != from_zk.primary_key)
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in primary key."
|
||||
" Stored in ZooKeeper: " + from_zk.primary_key + ", local: " + primary_key,
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
|
||||
if (data_format_version != from_zk.data_format_version)
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in data format version."
|
||||
" Stored in ZooKeeper: " + DB::toString(from_zk.data_format_version.toUnderType()) +
|
||||
", local: " + DB::toString(data_format_version.toUnderType()),
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
|
||||
if (partition_key != from_zk.partition_key)
|
||||
throw Exception(
|
||||
"Existing table metadata in ZooKeeper differs in partition key expression."
|
||||
" Stored in ZooKeeper: " + from_zk.partition_key + ", local: " + partition_key,
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
|
||||
if (sorting_key != from_zk.sorting_key)
|
||||
{
|
||||
if (allow_alter)
|
||||
{
|
||||
diff.sorting_key_changed = true;
|
||||
diff.new_sorting_key = from_zk.sorting_key;
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Existing table metadata in ZooKeeper differs in sorting key expression."
|
||||
" Stored in ZooKeeper: " + from_zk.sorting_key + ", local: " + sorting_key,
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
}
|
||||
|
||||
return diff;
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,49 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataFormatVersion.h>
|
||||
#include <Core/Types.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class MergeTreeData;
|
||||
class WriteBuffer;
|
||||
class ReadBuffer;
|
||||
|
||||
/** The basic parameters of ReplicatedMergeTree table engine for saving in ZooKeeper.
|
||||
* Lets you verify that they match local ones.
|
||||
*/
|
||||
struct ReplicatedMergeTreeTableMetadata
|
||||
{
|
||||
String date_column;
|
||||
String sampling_expression;
|
||||
UInt64 index_granularity;
|
||||
int merging_params_mode;
|
||||
String sign_column;
|
||||
String primary_key;
|
||||
MergeTreeDataFormatVersion data_format_version;
|
||||
String partition_key;
|
||||
String sorting_key;
|
||||
|
||||
ReplicatedMergeTreeTableMetadata() = default;
|
||||
explicit ReplicatedMergeTreeTableMetadata(const MergeTreeData & data);
|
||||
|
||||
void read(ReadBuffer & in);
|
||||
static ReplicatedMergeTreeTableMetadata parse(const String & s);
|
||||
|
||||
void write(WriteBuffer & out) const;
|
||||
String toString() const;
|
||||
|
||||
struct Diff
|
||||
{
|
||||
bool sorting_key_changed = false;
|
||||
String new_sorting_key;
|
||||
|
||||
bool empty() const { return !sorting_key_changed; }
|
||||
};
|
||||
|
||||
Diff checkAndFindDiff(const ReplicatedMergeTreeTableMetadata & from_zk, bool allow_alter) const;
|
||||
};
|
||||
|
||||
}
|
@ -28,28 +28,6 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
/** Get the key expression AST as an ASTExpressionList.
|
||||
* It can be specified in the tuple: (CounterID, Date),
|
||||
* or as one column: CounterID.
|
||||
*/
|
||||
static ASTPtr extractKeyExpressionList(IAST & node)
|
||||
{
|
||||
const ASTFunction * expr_func = typeid_cast<const ASTFunction *>(&node);
|
||||
|
||||
if (expr_func && expr_func->name == "tuple")
|
||||
{
|
||||
/// Primary key is specified in tuple.
|
||||
return expr_func->children.at(0);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Primary key consists of one column.
|
||||
auto res = std::make_shared<ASTExpressionList>();
|
||||
res->children.push_back(node.ptr());
|
||||
return res;
|
||||
}
|
||||
}
|
||||
|
||||
/** Get the list of column names.
|
||||
* It can be specified in the tuple: (Clicks, Cost),
|
||||
* or as one column: Clicks.
|
||||
@ -267,9 +245,13 @@ A common partitioning expression is some function of the event date column e.g.
|
||||
Rows with different partition expression values are never merged together. That allows manipulating partitions with ALTER commands.
|
||||
Also it acts as a kind of index.
|
||||
|
||||
Primary key is specified in the ORDER BY clause. It is mandatory for all MergeTree types.
|
||||
It is like (CounterID, EventDate, intHash64(UserID)) - a list of column names or functional expressions in round brackets.
|
||||
If your primary key has just one element, you may omit round brackets.
|
||||
Sorting key is specified in the ORDER BY clause. It is mandatory for all MergeTree types.
|
||||
It is like (CounterID, EventDate, intHash64(UserID)) - a list of column names or functional expressions
|
||||
in round brackets.
|
||||
If your sorting key has just one element, you may omit round brackets.
|
||||
|
||||
By default primary key is equal to the sorting key. You can specify a primary key that is a prefix of the
|
||||
sorting key in the PRIMARY KEY clause.
|
||||
|
||||
Careful choice of the primary key is extremely important for processing short-time queries.
|
||||
|
||||
@ -285,6 +267,8 @@ MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate) SETTI
|
||||
|
||||
MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) SAMPLE BY intHash32(UserID)
|
||||
|
||||
MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) PRIMARY KEY (CounterID, EventDate) SAMPLE BY intHash32(UserID)
|
||||
|
||||
CollapsingMergeTree(Sign) PARTITION BY StartDate SAMPLE BY intHash32(UserID) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID)
|
||||
|
||||
SummingMergeTree PARTITION BY toMonday(EventDate) ORDER BY (OrderID, EventDate, BannerID, PhraseID, ContextType, RegionID, PageID, IsFlat, TypeID, ResourceNo)
|
||||
@ -345,13 +329,14 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
*
|
||||
* Alternatively, you can specify:
|
||||
* - Partitioning expression in the PARTITION BY clause;
|
||||
* - Primary key in the ORDER BY clause;
|
||||
* - Sorting key in the ORDER BY clause;
|
||||
* - Primary key (if it is different from the sorting key) in the PRIMARY KEY clause;
|
||||
* - Sampling expression in the SAMPLE BY clause;
|
||||
* - Additional MergeTreeSettings in the SETTINGS clause;
|
||||
*/
|
||||
|
||||
bool is_extended_storage_def =
|
||||
args.storage_def->partition_by || args.storage_def->order_by || args.storage_def->sample_by || args.storage_def->settings;
|
||||
args.storage_def->partition_by || args.storage_def->primary_key || args.storage_def->order_by || args.storage_def->sample_by || args.storage_def->settings;
|
||||
|
||||
String name_part = args.engine_name.substr(0, args.engine_name.size() - strlen("MergeTree"));
|
||||
|
||||
@ -494,8 +479,6 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
engine_args.erase(engine_args.begin(), engine_args.begin() + 2);
|
||||
}
|
||||
|
||||
ASTPtr secondary_sorting_expr_list;
|
||||
|
||||
if (merging_params.mode == MergeTreeData::MergingParams::Collapsing)
|
||||
{
|
||||
if (auto ast = typeid_cast<const ASTIdentifier *>(engine_args.back().get()))
|
||||
@ -552,12 +535,8 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
}
|
||||
else if (merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing)
|
||||
{
|
||||
if (auto ast = typeid_cast<const ASTIdentifier *>(engine_args.back().get()))
|
||||
{
|
||||
if (auto ast = typeid_cast<ASTIdentifier *>(engine_args.back().get()))
|
||||
merging_params.version_column = ast->name;
|
||||
secondary_sorting_expr_list = std::make_shared<ASTExpressionList>();
|
||||
secondary_sorting_expr_list->children.push_back(engine_args.back());
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Version column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
|
||||
@ -576,25 +555,29 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
}
|
||||
|
||||
String date_column_name;
|
||||
ASTPtr partition_expr_list;
|
||||
ASTPtr primary_expr_list;
|
||||
ASTPtr sampling_expression;
|
||||
ASTPtr partition_by_ast;
|
||||
ASTPtr order_by_ast;
|
||||
ASTPtr primary_key_ast;
|
||||
ASTPtr sample_by_ast;
|
||||
MergeTreeSettings storage_settings = args.context.getMergeTreeSettings();
|
||||
|
||||
if (is_extended_storage_def)
|
||||
{
|
||||
if (args.storage_def->partition_by)
|
||||
partition_expr_list = extractKeyExpressionList(*args.storage_def->partition_by);
|
||||
partition_by_ast = args.storage_def->partition_by->ptr();
|
||||
|
||||
if (args.storage_def->order_by)
|
||||
primary_expr_list = extractKeyExpressionList(*args.storage_def->order_by);
|
||||
else
|
||||
if (!args.storage_def->order_by)
|
||||
throw Exception("You must provide an ORDER BY expression in the table definition. "
|
||||
"If you don't want this table to be sorted, use ORDER BY tuple()",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
order_by_ast = args.storage_def->order_by->ptr();
|
||||
|
||||
if (args.storage_def->primary_key)
|
||||
primary_key_ast = args.storage_def->primary_key->ptr();
|
||||
|
||||
if (args.storage_def->sample_by)
|
||||
sampling_expression = args.storage_def->sample_by->ptr();
|
||||
sample_by_ast = args.storage_def->sample_by->ptr();
|
||||
|
||||
storage_settings.loadFromQuery(*args.storage_def);
|
||||
}
|
||||
@ -603,7 +586,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
/// If there is an expression for sampling. MergeTree(date, [sample_key], primary_key, index_granularity)
|
||||
if (engine_args.size() == 4)
|
||||
{
|
||||
sampling_expression = engine_args[1];
|
||||
sample_by_ast = engine_args[1];
|
||||
engine_args.erase(engine_args.begin() + 1);
|
||||
}
|
||||
|
||||
@ -616,7 +599,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
"Date column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
primary_expr_list = extractKeyExpressionList(*engine_args[1]);
|
||||
order_by_ast = engine_args[1];
|
||||
|
||||
auto ast = typeid_cast<const ASTLiteral *>(engine_args.back().get());
|
||||
if (ast && ast->value.getType() == Field::Types::UInt64)
|
||||
@ -631,14 +614,14 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
return StorageReplicatedMergeTree::create(
|
||||
zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name,
|
||||
args.columns,
|
||||
args.context, primary_expr_list, secondary_sorting_expr_list, date_column_name, partition_expr_list,
|
||||
sampling_expression, merging_params, storage_settings,
|
||||
args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast,
|
||||
sample_by_ast, merging_params, storage_settings,
|
||||
args.has_force_restore_data_flag);
|
||||
else
|
||||
return StorageMergeTree::create(
|
||||
args.data_path, args.database_name, args.table_name, args.columns, args.attach,
|
||||
args.context, primary_expr_list, secondary_sorting_expr_list, date_column_name, partition_expr_list,
|
||||
sampling_expression, merging_params, storage_settings,
|
||||
args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast,
|
||||
sample_by_ast, merging_params, storage_settings,
|
||||
args.has_force_restore_data_flag);
|
||||
}
|
||||
|
||||
|
@ -678,10 +678,6 @@ void StorageBuffer::flushThread()
|
||||
|
||||
void StorageBuffer::alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context)
|
||||
{
|
||||
for (const auto & param : params)
|
||||
if (param.type == AlterCommand::MODIFY_PRIMARY_KEY)
|
||||
throw Exception("Storage engine " + getName() + " doesn't support primary key.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
auto lock = lockStructureForAlter();
|
||||
|
||||
/// So that no blocks of the old structure remain.
|
||||
|
@ -302,10 +302,6 @@ BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const Settings &
|
||||
|
||||
void StorageDistributed::alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context)
|
||||
{
|
||||
for (const auto & param : params)
|
||||
if (param.type == AlterCommand::MODIFY_PRIMARY_KEY)
|
||||
throw Exception("Storage engine " + getName() + " doesn't support primary key.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
auto lock = lockStructureForAlter();
|
||||
|
||||
ColumnsDescription new_columns = getColumns();
|
||||
|
@ -94,11 +94,11 @@ StoragePtr StorageFactory::get(
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
if ((storage_def->partition_by || storage_def->order_by || storage_def->sample_by)
|
||||
if ((storage_def->partition_by || storage_def->primary_key || storage_def->order_by || storage_def->sample_by)
|
||||
&& !endsWith(name, "MergeTree"))
|
||||
{
|
||||
throw Exception(
|
||||
"Engine " + name + " doesn't support PARTITION BY, ORDER BY or SAMPLE BY clauses. "
|
||||
"Engine " + name + " doesn't support PARTITION BY, PRIMARY KEY, ORDER BY or SAMPLE BY clauses. "
|
||||
"Currently only the MergeTree family of engines supports them", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
|
@ -396,10 +396,6 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr
|
||||
|
||||
void StorageMerge::alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context)
|
||||
{
|
||||
for (const auto & param : params)
|
||||
if (param.type == AlterCommand::MODIFY_PRIMARY_KEY)
|
||||
throw Exception("Storage engine " + getName() + " doesn't support primary key.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
auto lock = lockStructureForAlter();
|
||||
|
||||
ColumnsDescription new_columns = getColumns();
|
||||
|
@ -46,11 +46,11 @@ StorageMergeTree::StorageMergeTree(
|
||||
const ColumnsDescription & columns_,
|
||||
bool attach,
|
||||
Context & context_,
|
||||
const ASTPtr & primary_expr_ast_,
|
||||
const ASTPtr & secondary_sorting_expr_list_,
|
||||
const String & date_column_name,
|
||||
const ASTPtr & partition_expr_ast_,
|
||||
const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported.
|
||||
const ASTPtr & partition_by_ast_,
|
||||
const ASTPtr & order_by_ast_,
|
||||
const ASTPtr & primary_key_ast_,
|
||||
const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported.
|
||||
const MergeTreeData::MergingParams & merging_params_,
|
||||
const MergeTreeSettings & settings_,
|
||||
bool has_force_restore_data_flag)
|
||||
@ -58,8 +58,8 @@ StorageMergeTree::StorageMergeTree(
|
||||
context(context_), background_pool(context_.getBackgroundPool()),
|
||||
data(database_name, table_name,
|
||||
full_path, columns_,
|
||||
context_, primary_expr_ast_, secondary_sorting_expr_list_, date_column_name, partition_expr_ast_,
|
||||
sampling_expression_, merging_params_,
|
||||
context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_,
|
||||
sample_by_ast_, merging_params_,
|
||||
settings_, false, attach),
|
||||
reader(data), writer(data), merger_mutator(data, context.getBackgroundPool()),
|
||||
log(&Logger::get(database_name_ + "." + table_name + " (StorageMergeTree)"))
|
||||
@ -206,62 +206,60 @@ void StorageMergeTree::alter(
|
||||
data.checkAlter(params);
|
||||
|
||||
auto new_columns = data.getColumns();
|
||||
params.apply(new_columns);
|
||||
|
||||
std::vector<MergeTreeData::AlterDataPartTransactionPtr> transactions;
|
||||
|
||||
bool primary_key_is_modified = false;
|
||||
|
||||
ASTPtr new_primary_key_ast = data.primary_expr_ast;
|
||||
ASTPtr new_order_by_ast = data.order_by_ast;
|
||||
ASTPtr new_primary_key_ast = data.primary_key_ast;
|
||||
params.apply(new_columns, new_order_by_ast, new_primary_key_ast);
|
||||
|
||||
ASTPtr primary_expr_list_for_altering_parts;
|
||||
for (const AlterCommand & param : params)
|
||||
{
|
||||
if (param.type == AlterCommand::MODIFY_PRIMARY_KEY)
|
||||
{
|
||||
primary_key_is_modified = true;
|
||||
new_primary_key_ast = param.primary_key;
|
||||
if (supportsSampling())
|
||||
throw Exception("MODIFY PRIMARY KEY only supported for tables without sampling key", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
primary_expr_list_for_altering_parts = MergeTreeData::extractKeyExpressionList(param.primary_key);
|
||||
}
|
||||
}
|
||||
|
||||
if (primary_key_is_modified && supportsSampling())
|
||||
throw Exception("MODIFY PRIMARY KEY only supported for tables without sampling key", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
auto parts = data.getDataParts({MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});
|
||||
auto columns_for_parts = new_columns.getAllPhysical();
|
||||
std::vector<MergeTreeData::AlterDataPartTransactionPtr> transactions;
|
||||
for (const MergeTreeData::DataPartPtr & part : parts)
|
||||
{
|
||||
if (auto transaction = data.alterDataPart(part, columns_for_parts, new_primary_key_ast, false))
|
||||
if (auto transaction = data.alterDataPart(part, columns_for_parts, primary_expr_list_for_altering_parts, false))
|
||||
transactions.push_back(std::move(transaction));
|
||||
}
|
||||
|
||||
auto table_hard_lock = lockStructureForAlter();
|
||||
|
||||
IDatabase::ASTModifier storage_modifier;
|
||||
if (primary_key_is_modified)
|
||||
IDatabase::ASTModifier storage_modifier = [&] (IAST & ast)
|
||||
{
|
||||
storage_modifier = [&new_primary_key_ast] (IAST & ast)
|
||||
{
|
||||
auto tuple = std::make_shared<ASTFunction>();
|
||||
tuple->name = "tuple";
|
||||
tuple->arguments = new_primary_key_ast;
|
||||
tuple->children.push_back(tuple->arguments);
|
||||
auto & storage_ast = typeid_cast<ASTStorage &>(ast);
|
||||
|
||||
/// Primary key is in the second place in table engine description and can be represented as a tuple.
|
||||
/// TODO: Not always in second place. If there is a sampling key, then the third one. Fix it.
|
||||
auto & storage_ast = typeid_cast<ASTStorage &>(ast);
|
||||
typeid_cast<ASTExpressionList &>(*storage_ast.engine->arguments).children.at(1) = tuple;
|
||||
};
|
||||
}
|
||||
if (new_order_by_ast.get() != data.order_by_ast.get())
|
||||
{
|
||||
if (storage_ast.order_by)
|
||||
{
|
||||
/// The table was created using the "new" syntax (with key expressions in separate clauses).
|
||||
storage_ast.set(storage_ast.order_by, new_order_by_ast);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Primary key is in the second place in table engine description and can be represented as a tuple.
|
||||
/// TODO: Not always in second place. If there is a sampling key, then the third one. Fix it.
|
||||
storage_ast.engine->arguments->children.at(1) = new_order_by_ast;
|
||||
}
|
||||
}
|
||||
|
||||
if (new_primary_key_ast.get() != data.primary_key_ast.get())
|
||||
storage_ast.set(storage_ast.primary_key, new_primary_key_ast);
|
||||
};
|
||||
|
||||
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, storage_modifier);
|
||||
setColumns(std::move(new_columns));
|
||||
|
||||
if (primary_key_is_modified)
|
||||
{
|
||||
data.primary_expr_ast = new_primary_key_ast;
|
||||
}
|
||||
/// Reinitialize primary key because primary key column types might have changed.
|
||||
data.initPrimaryKey();
|
||||
data.setPrimaryKeyAndColumns(new_order_by_ast, new_primary_key_ast, new_columns);
|
||||
|
||||
for (auto & transaction : transactions)
|
||||
transaction->commit();
|
||||
@ -269,7 +267,7 @@ void StorageMergeTree::alter(
|
||||
/// Columns sizes could be changed
|
||||
data.recalculateColumnSizes();
|
||||
|
||||
if (primary_key_is_modified)
|
||||
if (primary_expr_list_for_altering_parts)
|
||||
data.loadDataParts(false);
|
||||
}
|
||||
|
||||
@ -717,7 +715,9 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi
|
||||
alter_command.column_name = get<String>(column_name);
|
||||
|
||||
auto new_columns = getColumns();
|
||||
alter_command.apply(new_columns);
|
||||
ASTPtr ignored_order_by_ast;
|
||||
ASTPtr ignored_primary_key_ast;
|
||||
alter_command.apply(new_columns, ignored_order_by_ast, ignored_primary_key_ast);
|
||||
|
||||
auto columns_for_parts = new_columns.getAllPhysical();
|
||||
for (const auto & part : parts)
|
||||
@ -725,7 +725,7 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi
|
||||
if (part->info.partition_id != partition_id)
|
||||
throw Exception("Unexpected partition ID " + part->info.partition_id + ". This is a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (auto transaction = data.alterDataPart(part, columns_for_parts, data.primary_expr_ast, false))
|
||||
if (auto transaction = data.alterDataPart(part, columns_for_parts, nullptr, false))
|
||||
transactions.push_back(std::move(transaction));
|
||||
|
||||
LOG_DEBUG(log, "Removing column " << get<String>(column_name) << " from part " << part->name);
|
||||
@ -992,34 +992,4 @@ ActionLock StorageMergeTree::getActionLock(StorageActionBlockType action_type)
|
||||
return {};
|
||||
}
|
||||
|
||||
Names StorageMergeTree::getSamplingExpressionNames() const
|
||||
{
|
||||
NameOrderedSet names;
|
||||
const auto & expr = data.sampling_expression;
|
||||
if (expr)
|
||||
expr->collectIdentifierNames(names);
|
||||
|
||||
return Names(names.begin(), names.end());
|
||||
}
|
||||
|
||||
Names StorageMergeTree::getPrimaryExpressionNames() const
|
||||
{
|
||||
return data.getPrimarySortColumns();
|
||||
}
|
||||
|
||||
Names StorageMergeTree::getPartitionExpressionNames() const
|
||||
{
|
||||
NameOrderedSet names;
|
||||
const auto & expr = data.partition_expr_ast;
|
||||
if (expr)
|
||||
expr->collectIdentifierNames(names);
|
||||
|
||||
return Names(names.cbegin(), names.cend());
|
||||
}
|
||||
|
||||
Names StorageMergeTree::getOrderExpressionNames() const
|
||||
{
|
||||
return data.getSortColumns();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -84,21 +84,17 @@ public:
|
||||
|
||||
String getDataPath() const override { return full_path; }
|
||||
|
||||
ASTPtr getSamplingExpression() const override { return data.sampling_expression; }
|
||||
ASTPtr getPartitionKeyAST() const override { return data.partition_by_ast; }
|
||||
ASTPtr getSortingKeyAST() const override { return data.getSortingKeyAST(); }
|
||||
ASTPtr getPrimaryKeyAST() const override { return data.getPrimaryKeyAST(); }
|
||||
ASTPtr getSamplingKeyAST() const override { return data.getSamplingExpression(); }
|
||||
|
||||
ASTPtr getPrimaryExpression() const override { return data.primary_expr_ast; }
|
||||
Names getColumnsRequiredForPartitionKey() const override { return data.getColumnsRequiredForPartitionKey(); }
|
||||
Names getColumnsRequiredForSortingKey() const override { return data.getColumnsRequiredForSortingKey(); }
|
||||
Names getColumnsRequiredForPrimaryKey() const override { return data.getColumnsRequiredForPrimaryKey(); }
|
||||
Names getColumnsRequiredForSampling() const override { return data.getColumnsRequiredForSampling(); }
|
||||
Names getColumnsRequiredForFinal() const override { return data.getColumnsRequiredForSortingKey(); }
|
||||
|
||||
ASTPtr getPartitionExpression() const override { return data.partition_expr_ast; }
|
||||
|
||||
ASTPtr getOrderExpression() const override { return data.secondary_sort_expr_ast; }
|
||||
|
||||
Names getSamplingExpressionNames() const override;
|
||||
|
||||
Names getPrimaryExpressionNames() const override;
|
||||
|
||||
Names getPartitionExpressionNames() const override;
|
||||
|
||||
Names getOrderExpressionNames() const override;
|
||||
private:
|
||||
String path;
|
||||
String database_name;
|
||||
@ -164,9 +160,7 @@ protected:
|
||||
* (correctness of names and paths are not checked)
|
||||
* consisting of the specified columns.
|
||||
*
|
||||
* primary_expr_ast - expression for sorting;
|
||||
* date_column_name - if not empty, the name of the column with the date used for partitioning by month;
|
||||
otherwise, partition_expr_ast is used as the partitioning expression;
|
||||
* See MergeTreeData constructor for comments on parameters.
|
||||
*/
|
||||
StorageMergeTree(
|
||||
const String & path_,
|
||||
@ -175,11 +169,11 @@ protected:
|
||||
const ColumnsDescription & columns_,
|
||||
bool attach,
|
||||
Context & context_,
|
||||
const ASTPtr & primary_expr_ast_,
|
||||
const ASTPtr & secondary_sorting_expr_list_,
|
||||
const String & date_column_name,
|
||||
const ASTPtr & partition_expr_ast_,
|
||||
const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported.
|
||||
const ASTPtr & partition_by_ast_,
|
||||
const ASTPtr & order_by_ast_,
|
||||
const ASTPtr & primary_key_ast_,
|
||||
const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported.
|
||||
const MergeTreeData::MergingParams & merging_params_,
|
||||
const MergeTreeSettings & settings_,
|
||||
bool has_force_restore_data_flag);
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h>
|
||||
@ -100,7 +101,6 @@ namespace ErrorCodes
|
||||
extern const int UNEXPECTED_FILE_IN_DATA_PART;
|
||||
extern const int NO_FILE_IN_DATA_PART;
|
||||
extern const int UNFINISHED;
|
||||
extern const int METADATA_MISMATCH;
|
||||
extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS;
|
||||
extern const int TOO_MANY_FETCHES;
|
||||
extern const int BAD_DATA_PART_NAME;
|
||||
@ -198,11 +198,11 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
|
||||
const String & path_, const String & database_name_, const String & name_,
|
||||
const ColumnsDescription & columns_,
|
||||
Context & context_,
|
||||
const ASTPtr & primary_expr_ast_,
|
||||
const ASTPtr & secondary_sorting_expr_list_,
|
||||
const String & date_column_name,
|
||||
const ASTPtr & partition_expr_ast_,
|
||||
const ASTPtr & sampling_expression_,
|
||||
const ASTPtr & partition_by_ast_,
|
||||
const ASTPtr & order_by_ast_,
|
||||
const ASTPtr & primary_key_ast_,
|
||||
const ASTPtr & sample_by_ast_,
|
||||
const MergeTreeData::MergingParams & merging_params_,
|
||||
const MergeTreeSettings & settings_,
|
||||
bool has_force_restore_data_flag)
|
||||
@ -213,8 +213,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
|
||||
replica_name(context.getMacros()->expand(replica_name_, database_name, table_name)),
|
||||
data(database_name, table_name,
|
||||
full_path, columns_,
|
||||
context_, primary_expr_ast_, secondary_sorting_expr_list_, date_column_name, partition_expr_ast_,
|
||||
sampling_expression_, merging_params_,
|
||||
context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_,
|
||||
sample_by_ast_, merging_params_,
|
||||
settings_, true, attach,
|
||||
[this] (const std::string & name) { enqueuePartForCheck(name); }),
|
||||
reader(data), writer(data), merger_mutator(data, context.getBackgroundPool()), queue(*this),
|
||||
@ -315,176 +315,9 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes()
|
||||
/// Mutations
|
||||
zookeeper->createIfNotExists(zookeeper_path + "/mutations", String());
|
||||
zookeeper->createIfNotExists(replica_path + "/mutation_pointer", String());
|
||||
}
|
||||
|
||||
|
||||
static String formattedAST(const ASTPtr & ast)
|
||||
{
|
||||
if (!ast)
|
||||
return "";
|
||||
std::stringstream ss;
|
||||
formatAST(*ast, ss, false, true);
|
||||
return ss.str();
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
/** The basic parameters of table engine for saving in ZooKeeper.
|
||||
* Lets you verify that they match local ones.
|
||||
*/
|
||||
struct TableMetadata
|
||||
{
|
||||
const MergeTreeData & data;
|
||||
|
||||
explicit TableMetadata(const MergeTreeData & data_)
|
||||
: data(data_) {}
|
||||
|
||||
void write(WriteBuffer & out) const
|
||||
{
|
||||
out << "metadata format version: 1" << "\n"
|
||||
<< "date column: ";
|
||||
|
||||
if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
out << data.minmax_idx_columns[data.minmax_idx_date_column_pos] << "\n";
|
||||
else
|
||||
out << "\n";
|
||||
|
||||
out << "sampling expression: " << formattedAST(data.sampling_expression) << "\n"
|
||||
<< "index granularity: " << data.index_granularity << "\n"
|
||||
<< "mode: " << static_cast<int>(data.merging_params.mode) << "\n"
|
||||
<< "sign column: " << data.merging_params.sign_column << "\n"
|
||||
<< "primary key: " << formattedAST(data.primary_expr_ast) << "\n";
|
||||
|
||||
if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
{
|
||||
out << "data format version: " << data.format_version.toUnderType() << "\n";
|
||||
out << "partition key: " << formattedAST(data.partition_expr_ast) << "\n";
|
||||
}
|
||||
}
|
||||
|
||||
String toString() const
|
||||
{
|
||||
WriteBufferFromOwnString out;
|
||||
write(out);
|
||||
return out.str();
|
||||
}
|
||||
|
||||
void check(ReadBuffer & in) const
|
||||
{
|
||||
/// TODO Can be made less cumbersome.
|
||||
|
||||
in >> "metadata format version: 1";
|
||||
|
||||
in >> "\ndate column: ";
|
||||
String read_date_column;
|
||||
in >> read_date_column;
|
||||
|
||||
if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
{
|
||||
const String & local_date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos];
|
||||
if (local_date_column != read_date_column)
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in date index column."
|
||||
" Stored in ZooKeeper: " + read_date_column + ", local: " + local_date_column,
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
}
|
||||
else if (!read_date_column.empty())
|
||||
throw Exception(
|
||||
"Existing table metadata in ZooKeeper differs in date index column."
|
||||
" Stored in ZooKeeper: " + read_date_column + ", local is custom-partitioned.",
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
|
||||
in >> "\nsampling expression: ";
|
||||
String read_sample_expression;
|
||||
String local_sample_expression = formattedAST(data.sampling_expression);
|
||||
in >> read_sample_expression;
|
||||
|
||||
if (read_sample_expression != local_sample_expression)
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in sample expression."
|
||||
" Stored in ZooKeeper: " + read_sample_expression + ", local: " + local_sample_expression,
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
|
||||
in >> "\nindex granularity: ";
|
||||
size_t read_index_granularity = 0;
|
||||
in >> read_index_granularity;
|
||||
|
||||
if (read_index_granularity != data.index_granularity)
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in index granularity."
|
||||
" Stored in ZooKeeper: " + DB::toString(read_index_granularity) + ", local: " + DB::toString(data.index_granularity),
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
|
||||
in >> "\nmode: ";
|
||||
int read_mode = 0;
|
||||
in >> read_mode;
|
||||
|
||||
if (read_mode != static_cast<int>(data.merging_params.mode))
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in mode of merge operation."
|
||||
" Stored in ZooKeeper: " + DB::toString(read_mode) + ", local: "
|
||||
+ DB::toString(static_cast<int>(data.merging_params.mode)),
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
|
||||
in >> "\nsign column: ";
|
||||
String read_sign_column;
|
||||
in >> read_sign_column;
|
||||
|
||||
if (read_sign_column != data.merging_params.sign_column)
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in sign column."
|
||||
" Stored in ZooKeeper: " + read_sign_column + ", local: " + data.merging_params.sign_column,
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
|
||||
in >> "\nprimary key: ";
|
||||
String read_primary_key;
|
||||
String local_primary_key = formattedAST(data.primary_expr_ast);
|
||||
in >> read_primary_key;
|
||||
|
||||
/// NOTE: You can make a less strict check of match expressions so that tables do not break from small changes
|
||||
/// in formatAST code.
|
||||
if (read_primary_key != local_primary_key)
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in primary key."
|
||||
" Stored in ZooKeeper: " + read_primary_key + ", local: " + local_primary_key,
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
|
||||
in >> "\n";
|
||||
MergeTreeDataFormatVersion read_data_format_version;
|
||||
if (in.eof())
|
||||
read_data_format_version = 0;
|
||||
else
|
||||
{
|
||||
in >> "data format version: ";
|
||||
in >> read_data_format_version.toUnderType();
|
||||
}
|
||||
|
||||
if (read_data_format_version != data.format_version)
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in data format version."
|
||||
" Stored in ZooKeeper: " + DB::toString(read_data_format_version.toUnderType()) +
|
||||
", local: " + DB::toString(data.format_version.toUnderType()),
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
|
||||
if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
{
|
||||
in >> "\npartition key: ";
|
||||
String read_partition_key;
|
||||
String local_partition_key = formattedAST(data.partition_expr_ast);
|
||||
in >> read_partition_key;
|
||||
|
||||
if (read_partition_key != local_partition_key)
|
||||
throw Exception(
|
||||
"Existing table metadata in ZooKeeper differs in partition key expression."
|
||||
" Stored in ZooKeeper: " + read_partition_key + ", local: " + local_partition_key,
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
|
||||
in >> "\n";
|
||||
}
|
||||
|
||||
assertEOF(in);
|
||||
}
|
||||
|
||||
void check(const String & s) const
|
||||
{
|
||||
ReadBufferFromString in(s);
|
||||
check(in);
|
||||
}
|
||||
};
|
||||
/// ALTERs of the metadata node.
|
||||
zookeeper->createIfNotExists(replica_path + "/metadata", String());
|
||||
}
|
||||
|
||||
|
||||
@ -500,7 +333,7 @@ void StorageReplicatedMergeTree::createTableIfNotExists()
|
||||
zookeeper->createAncestors(zookeeper_path);
|
||||
|
||||
/// We write metadata of table so that the replicas can check table parameters with them.
|
||||
String metadata = TableMetadata(data).toString();
|
||||
String metadata = ReplicatedMergeTreeTableMetadata(data).toString();
|
||||
|
||||
Coordination::Requests ops;
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "",
|
||||
@ -538,15 +371,20 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo
|
||||
{
|
||||
auto zookeeper = getZooKeeper();
|
||||
|
||||
String metadata_str = zookeeper->get(zookeeper_path + "/metadata");
|
||||
TableMetadata(data).check(metadata_str);
|
||||
ReplicatedMergeTreeTableMetadata old_metadata(data);
|
||||
|
||||
Coordination::Stat stat;
|
||||
auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_path + "/columns", &stat));
|
||||
columns_version = stat.version;
|
||||
Coordination::Stat metadata_stat;
|
||||
String metadata_str = zookeeper->get(zookeeper_path + "/metadata", &metadata_stat);
|
||||
auto metadata_from_zk = ReplicatedMergeTreeTableMetadata::parse(metadata_str);
|
||||
auto metadata_diff = old_metadata.checkAndFindDiff(metadata_from_zk, allow_alter);
|
||||
metadata_version = metadata_stat.version;
|
||||
|
||||
Coordination::Stat columns_stat;
|
||||
auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_path + "/columns", &columns_stat));
|
||||
columns_version = columns_stat.version;
|
||||
|
||||
const ColumnsDescription & old_columns = getColumns();
|
||||
if (columns_from_zk != old_columns)
|
||||
if (columns_from_zk != old_columns || !metadata_diff.empty())
|
||||
{
|
||||
if (allow_alter &&
|
||||
(skip_sanity_checks ||
|
||||
@ -555,10 +393,13 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo
|
||||
{
|
||||
LOG_WARNING(log, "Table structure in ZooKeeper is a little different from local table structure. Assuming ALTER.");
|
||||
|
||||
/// Without any locks, because table has not been created yet.
|
||||
context.getDatabase(database_name)->alterTable(context, table_name, columns_from_zk, {});
|
||||
|
||||
setColumns(std::move(columns_from_zk));
|
||||
/// We delay setting table structure till startup() because otherwise new table metadata file can
|
||||
/// be overwritten in DatabaseOrdinary::createTable.
|
||||
set_table_structure_at_startup = [columns_from_zk, metadata_diff, this]()
|
||||
{
|
||||
/// Without any locks, because table has not been created yet.
|
||||
setTableStructure(std::move(columns_from_zk), metadata_diff);
|
||||
};
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -569,6 +410,56 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo
|
||||
}
|
||||
|
||||
|
||||
void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & metadata_diff)
|
||||
{
|
||||
ASTPtr new_primary_key_ast = data.primary_key_ast;
|
||||
ASTPtr new_order_by_ast = data.order_by_ast;
|
||||
IDatabase::ASTModifier storage_modifier;
|
||||
if (!metadata_diff.empty())
|
||||
{
|
||||
ParserNotEmptyExpressionList parser(false);
|
||||
auto new_sorting_key_expr_list = parseQuery(parser, metadata_diff.new_sorting_key, 0);
|
||||
|
||||
if (new_sorting_key_expr_list->children.size() == 1)
|
||||
new_order_by_ast = new_sorting_key_expr_list->children[0];
|
||||
else
|
||||
{
|
||||
auto tuple = makeASTFunction("tuple");
|
||||
tuple->arguments->children = new_sorting_key_expr_list->children;
|
||||
new_order_by_ast = tuple;
|
||||
}
|
||||
|
||||
if (!data.primary_key_ast)
|
||||
{
|
||||
/// Primary and sorting key become independent after this ALTER so we have to
|
||||
/// save the old ORDER BY expression as the new primary key.
|
||||
new_primary_key_ast = data.order_by_ast->clone();
|
||||
}
|
||||
|
||||
storage_modifier = [&](IAST & ast)
|
||||
{
|
||||
auto & storage_ast = typeid_cast<ASTStorage &>(ast);
|
||||
|
||||
if (!storage_ast.order_by)
|
||||
throw Exception(
|
||||
"ALTER MODIFY ORDER BY of default-partitioned tables is not supported",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (new_primary_key_ast.get() != data.primary_key_ast.get())
|
||||
storage_ast.set(storage_ast.primary_key, new_primary_key_ast);
|
||||
|
||||
storage_ast.set(storage_ast.order_by, new_order_by_ast);
|
||||
};
|
||||
}
|
||||
|
||||
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, storage_modifier);
|
||||
|
||||
/// Even if the primary/sorting keys didn't change we must reinitialize it
|
||||
/// because primary key column types might have changed.
|
||||
data.setPrimaryKeyAndColumns(new_order_by_ast, new_primary_key_ast, new_columns);
|
||||
}
|
||||
|
||||
|
||||
/** If necessary, restore a part, replica itself adds a record for its receipt.
|
||||
* What time should I put for this entry in the queue? Time is taken into account when calculating lag of replica.
|
||||
* For these purposes, it makes sense to use creation time of missing part
|
||||
@ -1591,7 +1482,9 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry &
|
||||
alter_command.column_name = entry.column_name;
|
||||
|
||||
auto new_columns = getColumns();
|
||||
alter_command.apply(new_columns);
|
||||
ASTPtr ignored_order_by_ast;
|
||||
ASTPtr ignored_primary_key_ast;
|
||||
alter_command.apply(new_columns, ignored_order_by_ast, ignored_primary_key_ast);
|
||||
|
||||
size_t modified_parts = 0;
|
||||
auto parts = data.getDataParts();
|
||||
@ -1611,7 +1504,7 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry &
|
||||
|
||||
LOG_DEBUG(log, "Clearing column " << entry.column_name << " in part " << part->name);
|
||||
|
||||
auto transaction = data.alterDataPart(part, columns_for_parts, data.primary_expr_ast, false);
|
||||
auto transaction = data.alterDataPart(part, columns_for_parts, nullptr, false);
|
||||
if (!transaction)
|
||||
continue;
|
||||
|
||||
@ -2897,6 +2790,9 @@ void StorageReplicatedMergeTree::startup()
|
||||
if (is_readonly)
|
||||
return;
|
||||
|
||||
if (set_table_structure_at_startup)
|
||||
set_table_structure_at_startup();
|
||||
|
||||
queue.initialize(
|
||||
zookeeper_path, replica_path,
|
||||
database_name + "." + table_name + " (ReplicatedMergeTreeQueue)",
|
||||
@ -3124,9 +3020,33 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params,
|
||||
|
||||
LOG_DEBUG(log, "Doing ALTER");
|
||||
|
||||
int new_columns_version = -1; /// Initialization is to suppress (useless) false positive warning found by cppcheck.
|
||||
String new_columns_str;
|
||||
Coordination::Stat stat;
|
||||
/// Alter is done by modifying the metadata nodes in ZK that are shared between all replicas
|
||||
/// (/columns, /metadata). We set contents of the shared nodes to the new values and wait while
|
||||
/// replicas asynchronously apply changes (see ReplicatedMergeTreeAlterThread.cpp) and modify
|
||||
/// their respective replica metadata nodes (/replicas/<replica>/columns, /replicas/<replica>/metadata).
|
||||
|
||||
struct ChangedNode
|
||||
{
|
||||
ChangedNode(const String & table_path_, String name_, String new_value_)
|
||||
: table_path(table_path_), name(std::move(name_)), shared_path(table_path + "/" + name)
|
||||
, new_value(std::move(new_value_))
|
||||
{}
|
||||
|
||||
const String & table_path;
|
||||
String name;
|
||||
|
||||
String shared_path;
|
||||
|
||||
String getReplicaPath(const String & replica) const
|
||||
{
|
||||
return table_path + "/replicas/" + replica + "/" + name;
|
||||
}
|
||||
|
||||
String new_value;
|
||||
int32_t new_version = -1; /// Initialization is to suppress (useless) false positive warning found by cppcheck.
|
||||
};
|
||||
|
||||
std::vector<ChangedNode> changed_nodes;
|
||||
|
||||
{
|
||||
/// Just to read current structure. Alter will be done in separate thread.
|
||||
@ -3138,33 +3058,56 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params,
|
||||
data.checkAlter(params);
|
||||
|
||||
for (const AlterCommand & param : params)
|
||||
{
|
||||
if (param.type == AlterCommand::MODIFY_PRIMARY_KEY)
|
||||
throw Exception("Modification of primary key is not supported for replicated tables", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
ColumnsDescription new_columns = data.getColumns();
|
||||
params.apply(new_columns);
|
||||
ASTPtr new_order_by_ast = data.order_by_ast;
|
||||
ASTPtr new_primary_key_ast = data.primary_key_ast;
|
||||
params.apply(new_columns, new_order_by_ast, new_primary_key_ast);
|
||||
|
||||
new_columns_str = new_columns.toString();
|
||||
String new_columns_str = new_columns.toString();
|
||||
if (new_columns_str != data.getColumns().toString())
|
||||
changed_nodes.emplace_back(zookeeper_path, "columns", new_columns_str);
|
||||
|
||||
/// Do ALTER.
|
||||
getZooKeeper()->set(zookeeper_path + "/columns", new_columns_str, -1, &stat);
|
||||
ReplicatedMergeTreeTableMetadata new_metadata(data);
|
||||
if (new_order_by_ast.get() != data.order_by_ast.get())
|
||||
new_metadata.sorting_key = serializeAST(*MergeTreeData::extractKeyExpressionList(new_order_by_ast));
|
||||
|
||||
new_columns_version = stat.version;
|
||||
String new_metadata_str = new_metadata.toString();
|
||||
if (new_metadata_str != ReplicatedMergeTreeTableMetadata(data).toString())
|
||||
changed_nodes.emplace_back(zookeeper_path, "metadata", new_metadata_str);
|
||||
|
||||
/// Modify shared metadata nodes in ZooKeeper.
|
||||
Coordination::Requests ops;
|
||||
for (const auto & node : changed_nodes)
|
||||
ops.emplace_back(zkutil::makeSetRequest(node.shared_path, node.new_value, -1));
|
||||
|
||||
Coordination::Responses results = getZooKeeper()->multi(ops);
|
||||
|
||||
for (size_t i = 0; i < changed_nodes.size(); ++i)
|
||||
changed_nodes[i].new_version = dynamic_cast<const Coordination::SetResponse &>(*results[i]).stat.version;
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Updated columns in ZooKeeper. Waiting for replicas to apply changes.");
|
||||
LOG_DEBUG(log, "Updated shared metadata nodes in ZooKeeper. Waiting for replicas to apply changes.");
|
||||
|
||||
/// Wait until all replicas will apply ALTER.
|
||||
|
||||
/// Subscribe to change of columns, to finish waiting if someone will do another ALTER.
|
||||
if (!getZooKeeper()->exists(zookeeper_path + "/columns", &stat, alter_query_event))
|
||||
throw Exception(zookeeper_path + "/columns doesn't exist", ErrorCodes::NOT_FOUND_NODE);
|
||||
|
||||
if (stat.version != new_columns_version)
|
||||
for (const auto & node : changed_nodes)
|
||||
{
|
||||
LOG_WARNING(log, zookeeper_path + "/columns changed before this ALTER finished; "
|
||||
"overlapping ALTER-s are fine but use caution with nontransitive changes");
|
||||
return;
|
||||
Coordination::Stat stat;
|
||||
/// Subscribe to change of shared ZK metadata nodes, to finish waiting if someone will do another ALTER.
|
||||
if (!getZooKeeper()->exists(node.shared_path, &stat, alter_query_event))
|
||||
throw Exception(node.shared_path + " doesn't exist", ErrorCodes::NOT_FOUND_NODE);
|
||||
|
||||
if (stat.version != node.new_version)
|
||||
{
|
||||
LOG_WARNING(log, node.shared_path + " changed before this ALTER finished; " +
|
||||
"overlapping ALTER-s are fine but use caution with nontransitive changes");
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
Strings replicas = getZooKeeper()->getChildren(zookeeper_path + "/replicas");
|
||||
@ -3180,8 +3123,10 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params,
|
||||
|
||||
while (!partial_shutdown_called)
|
||||
{
|
||||
auto zookeeper = getZooKeeper();
|
||||
|
||||
/// Replica could be inactive.
|
||||
if (!getZooKeeper()->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
|
||||
if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
|
||||
{
|
||||
LOG_WARNING(log, "Replica " << replica << " is not active during ALTER query."
|
||||
" ALTER will be done asynchronously when replica becomes active.");
|
||||
@ -3190,40 +3135,92 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params,
|
||||
break;
|
||||
}
|
||||
|
||||
String replica_columns_str;
|
||||
|
||||
/// Replica could has been removed.
|
||||
if (!getZooKeeper()->tryGet(zookeeper_path + "/replicas/" + replica + "/columns", replica_columns_str, &stat))
|
||||
struct ReplicaNode
|
||||
{
|
||||
LOG_WARNING(log, replica << " was removed");
|
||||
break;
|
||||
explicit ReplicaNode(String path_) : path(std::move(path_)) {}
|
||||
|
||||
String path;
|
||||
String value;
|
||||
int32_t version = -1;
|
||||
};
|
||||
|
||||
std::vector<ReplicaNode> replica_nodes;
|
||||
for (const auto & node : changed_nodes)
|
||||
replica_nodes.emplace_back(node.getReplicaPath(replica));
|
||||
|
||||
bool replica_was_removed = false;
|
||||
for (auto & node : replica_nodes)
|
||||
{
|
||||
Coordination::Stat stat;
|
||||
|
||||
/// Replica could has been removed.
|
||||
if (!zookeeper->tryGet(node.path, node.value, &stat))
|
||||
{
|
||||
LOG_WARNING(log, replica << " was removed");
|
||||
replica_was_removed = true;
|
||||
break;
|
||||
}
|
||||
|
||||
node.version = stat.version;
|
||||
}
|
||||
|
||||
int replica_columns_version = stat.version;
|
||||
if (replica_was_removed)
|
||||
break;
|
||||
|
||||
bool alter_was_applied = true;
|
||||
for (size_t i = 0; i < replica_nodes.size(); ++i)
|
||||
{
|
||||
if (replica_nodes[i].value != changed_nodes[i].new_value)
|
||||
{
|
||||
alter_was_applied = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
/// The ALTER has been successfully applied.
|
||||
if (replica_columns_str == new_columns_str)
|
||||
if (alter_was_applied)
|
||||
break;
|
||||
|
||||
if (!getZooKeeper()->exists(zookeeper_path + "/columns", &stat))
|
||||
throw Exception(zookeeper_path + "/columns doesn't exist", ErrorCodes::NOT_FOUND_NODE);
|
||||
|
||||
if (stat.version != new_columns_version)
|
||||
for (const auto & node : changed_nodes)
|
||||
{
|
||||
LOG_WARNING(log, zookeeper_path + "/columns changed before ALTER finished; "
|
||||
"overlapping ALTER-s are fine but use caution with nontransitive changes");
|
||||
return;
|
||||
Coordination::Stat stat;
|
||||
if (!zookeeper->exists(node.shared_path, &stat))
|
||||
throw Exception(node.shared_path + " doesn't exist", ErrorCodes::NOT_FOUND_NODE);
|
||||
|
||||
if (stat.version != node.new_version)
|
||||
{
|
||||
LOG_WARNING(log, node.shared_path + " changed before this ALTER finished; "
|
||||
"overlapping ALTER-s are fine but use caution with nontransitive changes");
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
if (!getZooKeeper()->exists(zookeeper_path + "/replicas/" + replica + "/columns", &stat, alter_query_event))
|
||||
bool replica_nodes_changed_concurrently = false;
|
||||
for (const auto & replica_node : replica_nodes)
|
||||
{
|
||||
LOG_WARNING(log, replica << " was removed");
|
||||
Coordination::Stat stat;
|
||||
if (!zookeeper->exists(replica_node.path, &stat, alter_query_event))
|
||||
{
|
||||
LOG_WARNING(log, replica << " was removed");
|
||||
replica_was_removed = true;
|
||||
break;
|
||||
}
|
||||
|
||||
if (stat.version != replica_node.version)
|
||||
{
|
||||
replica_nodes_changed_concurrently = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (replica_was_removed)
|
||||
break;
|
||||
}
|
||||
|
||||
if (stat.version != replica_columns_version)
|
||||
if (replica_nodes_changed_concurrently)
|
||||
continue;
|
||||
|
||||
/// Now wait for replica nodes to change.
|
||||
|
||||
if (!replication_alter_columns_timeout)
|
||||
{
|
||||
alter_query_event->wait();
|
||||
@ -3934,7 +3931,9 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query
|
||||
leader_address.database,
|
||||
context.getClientInfo().current_user, context.getClientInfo().current_password, timeouts, "ClickHouse replica");
|
||||
|
||||
RemoteBlockInputStream stream(connection, formattedAST(new_query), {}, context, &settings);
|
||||
std::stringstream new_query_ss;
|
||||
formatAST(*new_query, new_query_ss, false, true);
|
||||
RemoteBlockInputStream stream(connection, new_query_ss.str(), {}, context, &settings);
|
||||
NullBlockOutputStream output({});
|
||||
|
||||
copyData(stream, output);
|
||||
@ -4906,34 +4905,4 @@ bool StorageReplicatedMergeTree::dropPartsInPartition(
|
||||
return true;
|
||||
}
|
||||
|
||||
Names StorageReplicatedMergeTree::getSamplingExpressionNames() const
|
||||
{
|
||||
NameOrderedSet names;
|
||||
const auto & expr = data.sampling_expression;
|
||||
if (expr)
|
||||
expr->collectIdentifierNames(names);
|
||||
|
||||
return Names(names.begin(), names.end());
|
||||
}
|
||||
|
||||
Names StorageReplicatedMergeTree::getPrimaryExpressionNames() const
|
||||
{
|
||||
return data.getPrimarySortColumns();
|
||||
}
|
||||
|
||||
Names StorageReplicatedMergeTree::getOrderExpressionNames() const
|
||||
{
|
||||
return data.getSortColumns();
|
||||
}
|
||||
|
||||
Names StorageReplicatedMergeTree::getPartitionExpressionNames() const
|
||||
{
|
||||
NameOrderedSet names;
|
||||
const auto & expr = data.partition_expr_ast;
|
||||
if (expr)
|
||||
expr->collectIdentifierNames(names);
|
||||
|
||||
return Names(names.cbegin(), names.cend());
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -14,6 +14,7 @@
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeAlterThread.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h>
|
||||
#include <Storages/MergeTree/EphemeralLockInZooKeeper.h>
|
||||
#include <Storages/MergeTree/BackgroundProcessingPool.h>
|
||||
#include <Storages/MergeTree/DataPartsExchange.h>
|
||||
@ -188,17 +189,16 @@ public:
|
||||
|
||||
String getDataPath() const override { return full_path; }
|
||||
|
||||
ASTPtr getSamplingExpression() const override { return data.sampling_expression; }
|
||||
ASTPtr getPartitionKeyAST() const override { return data.partition_by_ast; }
|
||||
ASTPtr getSortingKeyAST() const override { return data.getSortingKeyAST(); }
|
||||
ASTPtr getPrimaryKeyAST() const override { return data.getPrimaryKeyAST(); }
|
||||
ASTPtr getSamplingKeyAST() const override { return data.getSamplingExpression(); }
|
||||
|
||||
ASTPtr getPrimaryExpression() const override { return data.primary_expr_ast; }
|
||||
|
||||
Names getSamplingExpressionNames() const override;
|
||||
|
||||
Names getPrimaryExpressionNames() const override;
|
||||
|
||||
Names getOrderExpressionNames() const override;
|
||||
|
||||
Names getPartitionExpressionNames() const override;
|
||||
Names getColumnsRequiredForPartitionKey() const override { return data.getColumnsRequiredForPartitionKey(); }
|
||||
Names getColumnsRequiredForSortingKey() const override { return data.getColumnsRequiredForSortingKey(); }
|
||||
Names getColumnsRequiredForPrimaryKey() const override { return data.getColumnsRequiredForPrimaryKey(); }
|
||||
Names getColumnsRequiredForSampling() const override { return data.getColumnsRequiredForSampling(); }
|
||||
Names getColumnsRequiredForFinal() const override { return data.getColumnsRequiredForSortingKey(); }
|
||||
|
||||
private:
|
||||
/// Delete old parts from disk and from ZooKeeper.
|
||||
@ -241,11 +241,17 @@ private:
|
||||
*/
|
||||
zkutil::EphemeralNodeHolderPtr replica_is_active_node;
|
||||
|
||||
/** Version node /columns in ZooKeeper corresponding to the current data.columns.
|
||||
/** Version of the /columns node in ZooKeeper corresponding to the current data.columns.
|
||||
* Read and modify along with the data.columns - under TableStructureLock.
|
||||
*/
|
||||
int columns_version = -1;
|
||||
|
||||
/// Version of the /metadata node in ZooKeeper.
|
||||
int metadata_version = -1;
|
||||
|
||||
/// Used to delay setting table structure till startup() in case of an offline ALTER.
|
||||
std::function<void()> set_table_structure_at_startup;
|
||||
|
||||
/** Is this replica "leading". The leader replica selects the parts to merge.
|
||||
*/
|
||||
std::atomic<bool> is_leader {false};
|
||||
@ -330,9 +336,14 @@ private:
|
||||
|
||||
/** Verify that the list of columns and table settings match those specified in ZK (/metadata).
|
||||
* If not, throw an exception.
|
||||
* Must be called before startup().
|
||||
*/
|
||||
void checkTableStructure(bool skip_sanity_checks, bool allow_alter);
|
||||
|
||||
/// A part of ALTER: apply metadata changes only (data parts are altered separately).
|
||||
/// Must be called under IStorage::lockStructureForAlter() lock.
|
||||
void setTableStructure(ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & metadata_diff);
|
||||
|
||||
/** Check that the set of parts corresponds to that in ZK (/replicas/me/parts/).
|
||||
* If any parts described in ZK are not locally, throw an exception.
|
||||
* If any local parts are not mentioned in ZK, remove them.
|
||||
@ -532,11 +543,11 @@ protected:
|
||||
const String & path_, const String & database_name_, const String & name_,
|
||||
const ColumnsDescription & columns_,
|
||||
Context & context_,
|
||||
const ASTPtr & primary_expr_ast_,
|
||||
const ASTPtr & secondary_sorting_expr_list_,
|
||||
const String & date_column_name,
|
||||
const ASTPtr & partition_expr_ast_,
|
||||
const ASTPtr & sampling_expression_,
|
||||
const ASTPtr & partition_by_ast_,
|
||||
const ASTPtr & order_by_ast_,
|
||||
const ASTPtr & primary_key_ast_,
|
||||
const ASTPtr & sample_by_ast_,
|
||||
const MergeTreeData::MergingParams & merging_params_,
|
||||
const MergeTreeSettings & settings_,
|
||||
bool has_force_restore_data_flag);
|
||||
|
@ -37,10 +37,10 @@ StorageSystemColumns::StorageSystemColumns(const std::string & name_)
|
||||
{ "data_uncompressed_bytes", std::make_shared<DataTypeUInt64>() },
|
||||
{ "marks_bytes", std::make_shared<DataTypeUInt64>() },
|
||||
{ "comment", std::make_shared<DataTypeString>() },
|
||||
{ "is_in_primary_key", std::make_shared<DataTypeUInt8>() },
|
||||
{ "is_in_order_key", std::make_shared<DataTypeUInt8>() },
|
||||
{ "is_in_partition_key", std::make_shared<DataTypeUInt8>() },
|
||||
{ "is_in_sample_key", std::make_shared<DataTypeUInt8>() },
|
||||
{ "is_in_sorting_key", std::make_shared<DataTypeUInt8>() },
|
||||
{ "is_in_primary_key", std::make_shared<DataTypeUInt8>() },
|
||||
{ "is_in_sampling_key", std::make_shared<DataTypeUInt8>() },
|
||||
}));
|
||||
}
|
||||
|
||||
@ -86,10 +86,10 @@ protected:
|
||||
NamesAndTypesList columns;
|
||||
ColumnDefaults column_defaults;
|
||||
ColumnComments column_comments;
|
||||
Names partition_key_names;
|
||||
Names order_key_names;
|
||||
Names primary_key_names;
|
||||
Names sampling_key_names;
|
||||
Names cols_required_for_partition_key;
|
||||
Names cols_required_for_sorting_key;
|
||||
Names cols_required_for_primary_key;
|
||||
Names cols_required_for_sampling;
|
||||
MergeTreeData::ColumnSizeByName column_sizes;
|
||||
|
||||
{
|
||||
@ -117,10 +117,10 @@ protected:
|
||||
column_defaults = storage->getColumns().defaults;
|
||||
column_comments = storage->getColumns().comments;
|
||||
|
||||
partition_key_names = storage->getPartitionExpressionNames();
|
||||
order_key_names = storage->getOrderExpressionNames();
|
||||
primary_key_names = storage->getPrimaryExpressionNames();
|
||||
sampling_key_names = storage->getSamplingExpressionNames();
|
||||
cols_required_for_partition_key = storage->getColumnsRequiredForPartitionKey();
|
||||
cols_required_for_sorting_key = storage->getColumnsRequiredForSortingKey();
|
||||
cols_required_for_primary_key = storage->getColumnsRequiredForPrimaryKey();
|
||||
cols_required_for_sampling = storage->getColumnsRequiredForSampling();
|
||||
|
||||
/** Info about sizes of columns for tables of MergeTree family.
|
||||
* NOTE: It is possible to add getter for this info to IStorage interface.
|
||||
@ -210,13 +210,13 @@ protected:
|
||||
};
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(find_in_vector(primary_key_names));
|
||||
res_columns[res_index++]->insert(find_in_vector(cols_required_for_partition_key));
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(find_in_vector(order_key_names));
|
||||
res_columns[res_index++]->insert(find_in_vector(cols_required_for_sorting_key));
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(find_in_vector(partition_key_names));
|
||||
res_columns[res_index++]->insert(find_in_vector(cols_required_for_primary_key));
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(find_in_vector(sampling_key_names));
|
||||
res_columns[res_index++]->insert(find_in_vector(cols_required_for_sampling));
|
||||
}
|
||||
|
||||
++rows_count;
|
||||
|
@ -40,10 +40,10 @@ StorageSystemTables::StorageSystemTables(const std::string & name_)
|
||||
{"dependencies_table", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
{"create_table_query", std::make_shared<DataTypeString>()},
|
||||
{"engine_full", std::make_shared<DataTypeString>()},
|
||||
{"primary_key", std::make_shared<DataTypeString>()},
|
||||
{"order_key", std::make_shared<DataTypeString>()},
|
||||
{"partition_key", std::make_shared<DataTypeString>()},
|
||||
{"sample_key", std::make_shared<DataTypeString>()},
|
||||
{"sorting_key", std::make_shared<DataTypeString>()},
|
||||
{"primary_key", std::make_shared<DataTypeString>()},
|
||||
{"sampling_key", std::make_shared<DataTypeString>()},
|
||||
}));
|
||||
}
|
||||
|
||||
@ -257,7 +257,7 @@ protected:
|
||||
ASTPtr expression_ptr;
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
if ((expression_ptr = table_it->getPrimaryExpression()))
|
||||
if ((expression_ptr = table_it->getPartitionKeyAST()))
|
||||
res_columns[res_index++]->insert(queryToString(expression_ptr));
|
||||
else
|
||||
res_columns[res_index++]->insertDefault();
|
||||
@ -265,7 +265,7 @@ protected:
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
if ((expression_ptr = table_it->getOrderExpression()))
|
||||
if ((expression_ptr = table_it->getSortingKeyAST()))
|
||||
res_columns[res_index++]->insert(queryToString(expression_ptr));
|
||||
else
|
||||
res_columns[res_index++]->insertDefault();
|
||||
@ -273,7 +273,7 @@ protected:
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
if ((expression_ptr = table_it->getPartitionExpression()))
|
||||
if ((expression_ptr = table_it->getPrimaryKeyAST()))
|
||||
res_columns[res_index++]->insert(queryToString(expression_ptr));
|
||||
else
|
||||
res_columns[res_index++]->insertDefault();
|
||||
@ -281,7 +281,7 @@ protected:
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
if ((expression_ptr = table_it->getSamplingExpression()))
|
||||
if ((expression_ptr = table_it->getSamplingKeyAST()))
|
||||
res_columns[res_index++]->insert(queryToString(expression_ptr));
|
||||
else
|
||||
res_columns[res_index++]->insertDefault();
|
||||
|
@ -121,3 +121,10 @@
|
||||
1
|
||||
2
|
||||
3
|
||||
*** Check table creation statement ***
|
||||
CREATE TABLE test.pk2 ( x UInt32, y UInt32, z UInt32) ENGINE = MergeTree PRIMARY KEY (x, y) ORDER BY (x, y, z) SETTINGS index_granularity = 8192
|
||||
*** Check that the inserted values were correctly sorted ***
|
||||
100 20 1
|
||||
100 20 2
|
||||
100 30 1
|
||||
100 30 2
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET send_logs_level = 'none';
|
||||
|
||||
DROP TABLE IF EXISTS test.pk;
|
||||
CREATE TABLE test.pk (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 1);
|
||||
|
||||
@ -64,3 +66,18 @@ SELECT x, y FROM test.pk WHERE x > 2 AND y > 'z' ORDER BY x, y;
|
||||
SELECT x, y FROM test.pk WHERE y < 'A' ORDER BY x, y;
|
||||
|
||||
DROP TABLE test.pk;
|
||||
|
||||
DROP TABLE IF EXISTS test.pk2;
|
||||
CREATE TABLE test.pk2 (x UInt32) ENGINE MergeTree ORDER BY x;
|
||||
|
||||
ALTER TABLE test.pk2 ADD COLUMN y UInt32, ADD COLUMN z UInt32, MODIFY ORDER BY (x, y, z);
|
||||
ALTER TABLE test.pk2 MODIFY PRIMARY KEY (y); -- { serverError 36 }
|
||||
ALTER TABLE test.pk2 MODIFY PRIMARY KEY (x, y);
|
||||
SELECT '*** Check table creation statement ***';
|
||||
SHOW CREATE TABLE test.pk2;
|
||||
|
||||
INSERT INTO test.pk2 VALUES (100, 30, 2), (100, 30, 1), (100, 20, 2), (100, 20, 1);
|
||||
SELECT '*** Check that the inserted values were correctly sorted ***';
|
||||
SELECT * FROM test.pk2;
|
||||
|
||||
DROP TABLE test.pk2;
|
||||
|
@ -34,5 +34,5 @@ wait
|
||||
$ch "SELECT DISTINCT * FROM test.clear_column1 WHERE d != toDate('2000-03-01') ORDER BY d, i, s"
|
||||
$ch "SELECT DISTINCT * FROM test.clear_column2 WHERE d != toDate('2000-03-01') ORDER BY d, i, s"
|
||||
|
||||
#$ch "DROP TABLE IF EXISTS test.clear_column1"
|
||||
#$ch "DROP TABLE IF EXISTS test.clear_column2"
|
||||
$ch "DROP TABLE IF EXISTS test.clear_column1"
|
||||
$ch "DROP TABLE IF EXISTS test.clear_column2"
|
||||
|
@ -1 +1,2 @@
|
||||
1
|
||||
123
|
||||
|
@ -1,13 +1,12 @@
|
||||
drop table if exists test.trepl;
|
||||
create table test.trepl
|
||||
(
|
||||
d Date,
|
||||
a Int32,
|
||||
b Int32
|
||||
) engine = ReplacingMergeTree(d, (a,b), 8192);
|
||||
|
||||
|
||||
create table test.trepl(d Date,a Int32, b Int32) engine = ReplacingMergeTree(d, (a,b), 8192);
|
||||
insert into test.trepl values ('2018-09-19', 1, 1);
|
||||
select b from test.trepl FINAL prewhere a < 1000;
|
||||
drop table test.trepl;
|
||||
|
||||
|
||||
drop table if exists test.versioned_collapsing;
|
||||
create table test.versioned_collapsing(d Date, x UInt32, sign Int8, version UInt32) engine = VersionedCollapsingMergeTree(d, x, 8192, sign, version);
|
||||
insert into test.versioned_collapsing values ('2018-09-19', 123, 1, 0);
|
||||
select x from test.versioned_collapsing FINAL prewhere version < 1000;
|
||||
drop table test.versioned_collapsing;
|
||||
|
@ -1,25 +1,25 @@
|
||||
┌─name────────────────┬─primary_key─┬─order_key─┬─partition_key─┬─sample_key─┐
|
||||
│ check_system_tables │ name1 │ │ name2 │ name1 │
|
||||
└─────────────────────┴─────────────┴───────────┴───────────────┴────────────┘
|
||||
┌─name──┬─is_in_primary_key─┬─is_in_order_key─┬─is_in_partition_key─┬─is_in_sample_key─┐
|
||||
│ name1 │ 1 │ 1 │ 0 │ 1 │
|
||||
│ name2 │ 0 │ 0 │ 1 │ 0 │
|
||||
│ name3 │ 0 │ 0 │ 0 │ 0 │
|
||||
└───────┴───────────────────┴─────────────────┴─────────────────────┴──────────────────┘
|
||||
┌─name────────────────┬─primary_key─┬─order_key─┬─partition_key─┬─sample_key─┐
|
||||
│ check_system_tables │ date │ version │ date │ │
|
||||
└─────────────────────┴─────────────┴───────────┴───────────────┴────────────┘
|
||||
┌─name────┬─is_in_primary_key─┬─is_in_order_key─┬─is_in_partition_key─┬─is_in_sample_key─┐
|
||||
│ date │ 1 │ 1 │ 1 │ 0 │
|
||||
│ value │ 0 │ 0 │ 0 │ 0 │
|
||||
│ version │ 0 │ 1 │ 0 │ 0 │
|
||||
│ sign │ 0 │ 0 │ 0 │ 0 │
|
||||
└─────────┴───────────────────┴─────────────────┴─────────────────────┴──────────────────┘
|
||||
┌─name────────────────┬─primary_key───────────────────────┬─order_key─┬─partition_key───┬─sample_key────────┐
|
||||
│ check_system_tables │ Counter, Event, intHash32(UserId) │ │ toYYYYMM(Event) │ intHash32(UserId) │
|
||||
└─────────────────────┴───────────────────────────────────┴───────────┴─────────────────┴───────────────────┘
|
||||
┌─name────┬─is_in_primary_key─┬─is_in_order_key─┬─is_in_partition_key─┬─is_in_sample_key─┐
|
||||
│ Event │ 1 │ 1 │ 1 │ 0 │
|
||||
│ UserId │ 0 │ 0 │ 0 │ 1 │
|
||||
│ Counter │ 1 │ 1 │ 0 │ 0 │
|
||||
└─────────┴───────────────────┴─────────────────┴─────────────────────┴──────────────────┘
|
||||
┌─name────────────────┬─partition_key─┬─sorting_key─┬─primary_key─┬─sampling_key─┐
|
||||
│ check_system_tables │ name2 │ name1 │ name1 │ name1 │
|
||||
└─────────────────────┴───────────────┴─────────────┴─────────────┴──────────────┘
|
||||
┌─name──┬─is_in_partition_key─┬─is_in_sorting_key─┬─is_in_primary_key─┬─is_in_sampling_key─┐
|
||||
│ name1 │ 0 │ 1 │ 1 │ 1 │
|
||||
│ name2 │ 1 │ 0 │ 0 │ 0 │
|
||||
│ name3 │ 0 │ 0 │ 0 │ 0 │
|
||||
└───────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘
|
||||
┌─name────────────────┬─partition_key─┬─sorting_key───┬─primary_key─┬─sampling_key─┐
|
||||
│ check_system_tables │ date │ date, version │ date │ │
|
||||
└─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘
|
||||
┌─name────┬─is_in_partition_key─┬─is_in_sorting_key─┬─is_in_primary_key─┬─is_in_sampling_key─┐
|
||||
│ date │ 1 │ 1 │ 1 │ 0 │
|
||||
│ value │ 0 │ 0 │ 0 │ 0 │
|
||||
│ version │ 0 │ 1 │ 0 │ 0 │
|
||||
│ sign │ 0 │ 0 │ 0 │ 0 │
|
||||
└─────────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘
|
||||
┌─name────────────────┬─partition_key───┬─sorting_key───────────────────────┬─primary_key───────────────────────┬─sampling_key──────┐
|
||||
│ check_system_tables │ toYYYYMM(Event) │ Counter, Event, intHash32(UserId) │ Counter, Event, intHash32(UserId) │ intHash32(UserId) │
|
||||
└─────────────────────┴─────────────────┴───────────────────────────────────┴───────────────────────────────────┴───────────────────┘
|
||||
┌─name────┬─is_in_partition_key─┬─is_in_sorting_key─┬─is_in_primary_key─┬─is_in_sampling_key─┐
|
||||
│ Event │ 1 │ 1 │ 1 │ 0 │
|
||||
│ UserId │ 0 │ 1 │ 1 │ 1 │
|
||||
│ Counter │ 0 │ 1 │ 1 │ 0 │
|
||||
└─────────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘
|
||||
|
@ -12,12 +12,12 @@ CREATE TABLE test.check_system_tables
|
||||
PARTITION BY name2
|
||||
SAMPLE BY name1;
|
||||
|
||||
SELECT name, primary_key, order_key, partition_key, sample_key
|
||||
SELECT name, partition_key, sorting_key, primary_key, sampling_key
|
||||
FROM system.tables
|
||||
WHERE name = 'check_system_tables'
|
||||
FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
SELECT name, is_in_primary_key, is_in_order_key, is_in_partition_key, is_in_sample_key
|
||||
SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key
|
||||
FROM system.columns
|
||||
WHERE table = 'check_system_tables'
|
||||
FORMAT PrettyCompactNoEscapes;
|
||||
@ -35,12 +35,12 @@ CREATE TABLE test.check_system_tables
|
||||
PARTITION BY date
|
||||
ORDER BY date;
|
||||
|
||||
SELECT name, primary_key, order_key, partition_key, sample_key
|
||||
SELECT name, partition_key, sorting_key, primary_key, sampling_key
|
||||
FROM system.tables
|
||||
WHERE name = 'check_system_tables'
|
||||
FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
SELECT name, is_in_primary_key, is_in_order_key, is_in_partition_key, is_in_sample_key
|
||||
SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key
|
||||
FROM system.columns
|
||||
WHERE table = 'check_system_tables'
|
||||
FORMAT PrettyCompactNoEscapes;
|
||||
@ -55,12 +55,12 @@ CREATE TABLE test.check_system_tables
|
||||
Counter UInt32
|
||||
) ENGINE = MergeTree(Event, intHash32(UserId), (Counter, Event, intHash32(UserId)), 8192);
|
||||
|
||||
SELECT name, primary_key, order_key, partition_key, sample_key
|
||||
SELECT name, partition_key, sorting_key, primary_key, sampling_key
|
||||
FROM system.tables
|
||||
WHERE name = 'check_system_tables'
|
||||
FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
SELECT name, is_in_primary_key, is_in_order_key, is_in_partition_key, is_in_sample_key
|
||||
SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key
|
||||
FROM system.columns
|
||||
WHERE table = 'check_system_tables'
|
||||
FORMAT PrettyCompactNoEscapes;
|
||||
|
@ -0,0 +1,12 @@
|
||||
*** Check that the parts are sorted according to the new key. ***
|
||||
1 2 0 10
|
||||
1 2 0 20
|
||||
1 2 2 40
|
||||
1 2 2 50
|
||||
1 2 1 30
|
||||
*** Check that the rows are collapsed according to the new key. ***
|
||||
1 2 0 30
|
||||
1 2 1 30
|
||||
1 2 4 90
|
||||
*** Check SHOW CREATE TABLE ***
|
||||
CREATE TABLE test.summing ( x UInt32, y UInt32, z UInt32, val UInt32) ENGINE = SummingMergeTree PRIMARY KEY (x, y) ORDER BY (x, y, -z) SETTINGS index_granularity = 8192
|
@ -0,0 +1,38 @@
|
||||
SET send_logs_level = 'none';
|
||||
|
||||
DROP TABLE IF EXISTS test.old_style;
|
||||
CREATE TABLE test.old_style(d Date, x UInt32) ENGINE MergeTree(d, x, 8192);
|
||||
ALTER TABLE test.old_style ADD COLUMN y UInt32, MODIFY ORDER BY (x, y); -- { serverError 36}
|
||||
DROP TABLE test.old_style;
|
||||
|
||||
DROP TABLE IF EXISTS test.summing;
|
||||
CREATE TABLE test.summing(x UInt32, y UInt32, val UInt32) ENGINE SummingMergeTree ORDER BY (x, y);
|
||||
|
||||
/* Can't add an expression with existing column to ORDER BY. */
|
||||
ALTER TABLE test.summing MODIFY ORDER BY (x, y, -val); -- { serverError 36}
|
||||
|
||||
/* Can't add an expression with existing column to ORDER BY. */
|
||||
ALTER TABLE test.summing ADD COLUMN z UInt32 DEFAULT x + 1, MODIFY ORDER BY (x, y, -z); -- { serverError 36}
|
||||
|
||||
/* Can't add nonexistent column to ORDER BY. */
|
||||
ALTER TABLE test.summing MODIFY ORDER BY (x, y, nonexistent); -- { serverError 47}
|
||||
|
||||
/* Can't modyfy ORDER BY so that it is no longer a prefix of the PRIMARY KEY. */
|
||||
ALTER TABLE test.summing MODIFY ORDER BY x; -- { serverError 36}
|
||||
|
||||
INSERT INTO test.summing(x, y, val) VALUES (1, 2, 10), (1, 2, 20);
|
||||
|
||||
ALTER TABLE test.summing ADD COLUMN z UInt32 AFTER y, MODIFY ORDER BY (x, y, -z);
|
||||
|
||||
INSERT INTO test.summing(x, y, z, val) values (1, 2, 1, 30), (1, 2, 2, 40), (1, 2, 2, 50);
|
||||
|
||||
SELECT '*** Check that the parts are sorted according to the new key. ***';
|
||||
SELECT * FROM test.summing ORDER BY _part;
|
||||
|
||||
SELECT '*** Check that the rows are collapsed according to the new key. ***';
|
||||
SELECT * FROM test.summing FINAL ORDER BY x, y, z;
|
||||
|
||||
SELECT '*** Check SHOW CREATE TABLE ***';
|
||||
SHOW CREATE TABLE test.summing;
|
||||
|
||||
DROP TABLE test.summing;
|
@ -0,0 +1,14 @@
|
||||
*** Check that the parts are sorted according to the new key. ***
|
||||
1 2 0 10
|
||||
1 2 0 20
|
||||
1 2 2 40
|
||||
1 2 2 50
|
||||
1 2 1 30
|
||||
*** Check that the rows are collapsed according to the new key. ***
|
||||
1 2 0 30
|
||||
1 2 1 30
|
||||
1 2 4 90
|
||||
*** Check SHOW CREATE TABLE ***
|
||||
CREATE TABLE test.summing_r2 ( x UInt32, y UInt32, z UInt32, val UInt32) ENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test/summing\', \'r2\') PRIMARY KEY (x, y) ORDER BY (x, y, -z) SETTINGS index_granularity = 8192
|
||||
*** Check SHOW CREATE TABLE after offline ALTER ***
|
||||
CREATE TABLE test.summing_r2 ( x UInt32, y UInt32, z UInt32, t UInt32, val UInt32) ENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test/summing\', \'r2\') PRIMARY KEY (x, y) ORDER BY (x, y, t * t) SETTINGS index_granularity = 8192
|
@ -0,0 +1,49 @@
|
||||
SET send_logs_level = 'none';
|
||||
|
||||
DROP TABLE IF EXISTS test.old_style;
|
||||
CREATE TABLE test.old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/old_style', 'r1', d, x, 8192);
|
||||
ALTER TABLE test.old_style ADD COLUMN y UInt32, MODIFY ORDER BY (x, y); -- { serverError 36 }
|
||||
DROP TABLE test.old_style;
|
||||
|
||||
DROP TABLE IF EXISTS test.summing_r1;
|
||||
DROP TABLE IF EXISTS test.summing_r2;
|
||||
CREATE TABLE test.summing_r1(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/test/summing', 'r1') ORDER BY (x, y);
|
||||
CREATE TABLE test.summing_r2(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/test/summing', 'r2') ORDER BY (x, y);
|
||||
|
||||
/* Can't add an expression with existing column to ORDER BY. */
|
||||
ALTER TABLE test.summing_r1 MODIFY ORDER BY (x, y, -val); -- { serverError 36 }
|
||||
|
||||
/* Can't add an expression with existing column to ORDER BY. */
|
||||
ALTER TABLE test.summing_r1 ADD COLUMN z UInt32 DEFAULT x + 1, MODIFY ORDER BY (x, y, -z); -- { serverError 36 }
|
||||
|
||||
/* Can't add nonexistent column to ORDER BY. */
|
||||
ALTER TABLE test.summing_r1 MODIFY ORDER BY (x, y, nonexistent); -- { serverError 47 }
|
||||
|
||||
/* Can't modyfy ORDER BY so that it is no longer a prefix of the PRIMARY KEY. */
|
||||
ALTER TABLE test.summing_r1 MODIFY ORDER BY x; -- { serverError 36 }
|
||||
|
||||
INSERT INTO test.summing_r1(x, y, val) VALUES (1, 2, 10), (1, 2, 20);
|
||||
SYSTEM SYNC REPLICA test.summing_r2;
|
||||
|
||||
ALTER TABLE test.summing_r1 ADD COLUMN z UInt32 AFTER y, MODIFY ORDER BY (x, y, -z);
|
||||
|
||||
INSERT INTO test.summing_r1(x, y, z, val) values (1, 2, 1, 30), (1, 2, 2, 40), (1, 2, 2, 50);
|
||||
SYSTEM SYNC REPLICA test.summing_r2;
|
||||
|
||||
SELECT '*** Check that the parts are sorted according to the new key. ***';
|
||||
SELECT * FROM test.summing_r2 ORDER BY _part;
|
||||
|
||||
SELECT '*** Check that the rows are collapsed according to the new key. ***';
|
||||
SELECT * FROM test.summing_r2 FINAL ORDER BY x, y, z;
|
||||
|
||||
SELECT '*** Check SHOW CREATE TABLE ***';
|
||||
SHOW CREATE TABLE test.summing_r2;
|
||||
|
||||
DETACH TABLE test.summing_r2;
|
||||
ALTER TABLE test.summing_r1 ADD COLUMN t UInt32 AFTER z, MODIFY ORDER BY (x, y, t * t); -- { serverError 341 }
|
||||
ATTACH TABLE test.summing_r2;
|
||||
SELECT '*** Check SHOW CREATE TABLE after offline ALTER ***';
|
||||
SHOW CREATE TABLE test.summing_r2;
|
||||
|
||||
DROP TABLE test.summing_r1;
|
||||
DROP TABLE test.summing_r2;
|
Loading…
Reference in New Issue
Block a user