Merge remote-tracking branch 'origin/master' into sanych73-prepared_statements

This commit is contained in:
Alexey Milovidov 2019-06-16 02:40:17 +03:00
commit 7514674447
187 changed files with 4151 additions and 2738 deletions

View File

@ -33,6 +33,7 @@
#include <IO/UseSSL.h>
#include <Interpreters/AsynchronousMetrics.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/ExternalDictionaries.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/loadMetadata.h>
#include <Interpreters/DNSCacheUpdater.h>
@ -832,7 +833,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
if (!config().getBool("dictionaries_lazy_load", true))
{
global_context->tryCreateEmbeddedDictionaries();
global_context->tryCreateExternalDictionaries();
global_context->getExternalDictionaries().enableAlwaysLoadEverything(true);
}
}
catch (...)

View File

@ -0,0 +1,70 @@
#include <Common/Config/AbstractConfigurationComparison.h>
#include <unordered_set>
#include <common/StringRef.h>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
namespace
{
String concatKeyAndSubKey(const String & key, const String & subkey)
{
// Copied from Poco::Util::ConfigurationView::translateKey():
String result = key;
if (!result.empty() && !subkey.empty() && subkey[0] != '[')
result += '.';
result += subkey;
return result;
};
}
bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const Poco::Util::AbstractConfiguration & right)
{
return isSameConfiguration(left, String(), right, String());
}
bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const String & left_key,
const Poco::Util::AbstractConfiguration & right, const String & right_key)
{
if (&left == &right && left_key == right_key)
return true;
bool has_property = left.hasProperty(left_key);
if (has_property != right.hasProperty(right_key))
return false;
if (has_property)
{
/// The left and right configurations contains values so we can compare them.
if (left.getRawString(left_key) != right.getRawString(right_key))
return false;
}
/// Get the subkeys of the left and right configurations.
Poco::Util::AbstractConfiguration::Keys subkeys;
left.keys(left_key, subkeys);
{
/// Check that the right configuration has the same set of subkeys as the left configuration.
Poco::Util::AbstractConfiguration::Keys right_subkeys;
right.keys(right_key, right_subkeys);
std::unordered_set<StringRef> left_subkeys{subkeys.begin(), subkeys.end()};
if ((left_subkeys.size() != right_subkeys.size()) || (left_subkeys.size() != subkeys.size()))
return false;
for (const auto & right_subkey : right_subkeys)
if (!left_subkeys.count(right_subkey))
return false;
}
/// Go through all the subkeys and compare corresponding parts of the configurations.
for (const auto & subkey : subkeys)
if (!isSameConfiguration(left, concatKeyAndSubKey(left_key, subkey), right, concatKeyAndSubKey(right_key, subkey)))
return false;
return true;
}
}

View File

@ -0,0 +1,29 @@
#pragma once
#include <Core/Types.h>
namespace Poco::Util
{
class AbstractConfiguration;
}
namespace DB
{
/// Returns true if two configurations contains the same keys and values.
bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left,
const Poco::Util::AbstractConfiguration & right);
/// Returns true if specified subviews of the two configurations contains the same keys and values.
bool isSameConfiguration(const Poco::Util::AbstractConfiguration & left, const String & left_key,
const Poco::Util::AbstractConfiguration & right, const String & right_key);
inline bool operator==(const Poco::Util::AbstractConfiguration & left, const Poco::Util::AbstractConfiguration & right)
{
return isSameConfiguration(left, right);
}
inline bool operator!=(const Poco::Util::AbstractConfiguration & left, const Poco::Util::AbstractConfiguration & right)
{
return !isSameConfiguration(left, right);
}
}

View File

@ -31,24 +31,29 @@ void DatabaseDictionary::loadTables(Context &, ThreadPool *, bool)
{
}
Tables DatabaseDictionary::listTables(const Context & context)
Tables DatabaseDictionary::listTables(const Context & context, const FilterByNameFunction & filter_by_name)
{
auto objects_map = context.getExternalDictionaries().getObjectsMap();
const auto & dictionaries = objects_map.get();
Tables tables;
for (const auto & pair : dictionaries)
ExternalLoader::Loadables loadables;
if (filter_by_name)
{
auto dict_ptr = std::static_pointer_cast<IDictionaryBase>(pair.second.loadable);
if (dict_ptr)
/// If `filter_by_name` is set, we iterate through all dictionaries with such names. That's why we need to load all of them.
loadables = context.getExternalDictionaries().loadAndGet(filter_by_name);
}
else
{
/// If `filter_by_name` isn't set, we iterate through only already loaded dictionaries. We don't try to load all dictionaries in this case.
loadables = context.getExternalDictionaries().getCurrentlyLoadedObjects();
}
for (const auto & loadable : loadables)
{
auto dict_ptr = std::static_pointer_cast<const IDictionaryBase>(loadable);
auto dict_name = dict_ptr->getName();
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
const std::string & dict_name = pair.first;
tables[dict_name] = StorageDictionary::create(dict_name, ColumnsDescription{columns}, context, true, dict_name);
}
}
return tables;
}
@ -56,9 +61,7 @@ bool DatabaseDictionary::isTableExist(
const Context & context,
const String & table_name) const
{
auto objects_map = context.getExternalDictionaries().getObjectsMap();
const auto & dictionaries = objects_map.get();
return dictionaries.count(table_name);
return context.getExternalDictionaries().getCurrentStatus(table_name) != ExternalLoader::Status::NOT_EXIST;
}
StoragePtr DatabaseDictionary::tryGetTable(
@ -76,19 +79,14 @@ StoragePtr DatabaseDictionary::tryGetTable(
return {};
}
DatabaseIteratorPtr DatabaseDictionary::getIterator(const Context & context)
DatabaseIteratorPtr DatabaseDictionary::getIterator(const Context & context, const FilterByNameFunction & filter_by_name)
{
return std::make_unique<DatabaseSnapshotIterator>(listTables(context));
return std::make_unique<DatabaseSnapshotIterator>(listTables(context, filter_by_name));
}
bool DatabaseDictionary::empty(const Context & context) const
{
auto objects_map = context.getExternalDictionaries().getObjectsMap();
const auto & dictionaries = objects_map.get();
for (const auto & pair : dictionaries)
if (pair.second.loadable)
return false;
return true;
return context.getExternalDictionaries().getNumberOfNames() == 0;
}
StoragePtr DatabaseDictionary::detachTable(const String & /*table_name*/)

View File

@ -44,7 +44,7 @@ public:
const Context & context,
const String & table_name) const override;
DatabaseIteratorPtr getIterator(const Context & context) override;
DatabaseIteratorPtr getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
bool empty(const Context & context) const override;
@ -96,7 +96,7 @@ private:
Poco::Logger * log;
Tables listTables(const Context & context);
Tables listTables(const Context & context, const FilterByNameFunction & filter_by_name);
ASTPtr getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const;
};

View File

@ -107,10 +107,16 @@ StoragePtr DatabaseWithOwnTablesBase::tryGetTable(
return it->second;
}
DatabaseIteratorPtr DatabaseWithOwnTablesBase::getIterator(const Context & /*context*/)
DatabaseIteratorPtr DatabaseWithOwnTablesBase::getIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_table_name)
{
std::lock_guard lock(mutex);
if (!filter_by_table_name)
return std::make_unique<DatabaseSnapshotIterator>(tables);
Tables filtered_tables;
for (const auto & [table_name, storage] : tables)
if (filter_by_table_name(table_name))
filtered_tables.emplace(table_name, storage);
return std::make_unique<DatabaseSnapshotIterator>(std::move(filtered_tables));
}
bool DatabaseWithOwnTablesBase::empty(const Context & /*context*/) const

View File

@ -89,7 +89,7 @@ public:
StoragePtr detachTable(const String & table_name) override;
DatabaseIteratorPtr getIterator(const Context & context) override;
DatabaseIteratorPtr getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
void shutdown() override;

View File

@ -73,9 +73,11 @@ public:
const Context & context,
const String & name) const = 0;
using FilterByNameFunction = std::function<bool(const String &)>;
/// Get an iterator that allows you to pass through all the tables.
/// It is possible to have "hidden" tables that are not visible when passing through, but are visible if you get them by name using the functions above.
virtual DatabaseIteratorPtr getIterator(const Context & context) = 0;
virtual DatabaseIteratorPtr getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) = 0;
/// Is the database empty.
virtual bool empty(const Context & context) const = 0;

View File

@ -51,9 +51,9 @@ public:
bool isCached() const override { return true; }
std::unique_ptr<IExternalLoadable> clone() const override
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_unique<CacheDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, size);
return std::make_shared<CacheDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, size);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }

View File

@ -75,9 +75,9 @@ public:
bool isCached() const override { return true; }
std::unique_ptr<IExternalLoadable> clone() const override
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_unique<ComplexKeyCacheDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, size);
return std::make_shared<ComplexKeyCacheDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, size);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }

View File

@ -50,9 +50,9 @@ public:
bool isCached() const override { return false; }
std::unique_ptr<IExternalLoadable> clone() const override
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_unique<ComplexKeyHashedDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block);
return std::make_shared<ComplexKeyHashedDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }

View File

@ -88,18 +88,15 @@ DictionarySourcePtr DictionarySourceFactory::create(
throw Exception{name + ": element dictionary.source should have exactly one child element",
ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG};
auto sample_block = createSampleBlock(dict_struct);
const auto & source_type = keys.front();
{
const auto found = registered_sources.find(source_type);
if (found != registered_sources.end())
{
const auto & create_source = found->second;
auto sample_block = createSampleBlock(dict_struct);
return create_source(dict_struct, config, config_prefix, sample_block, context);
}
}
throw Exception{name + ": unknown dictionary source type: " + source_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
}

View File

@ -47,9 +47,9 @@ public:
bool isCached() const override { return false; }
std::unique_ptr<IExternalLoadable> clone() const override
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_unique<FlatDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block);
return std::make_shared<FlatDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }

View File

@ -46,9 +46,9 @@ public:
bool isCached() const override { return false; }
std::unique_ptr<IExternalLoadable> clone() const override
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_unique<HashedDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block);
return std::make_shared<HashedDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }

View File

@ -42,9 +42,9 @@ public:
bool isCached() const override { return false; }
std::unique_ptr<IExternalLoadable> clone() const override
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_unique<RangeHashedDictionary>(dictionary_name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty);
return std::make_shared<RangeHashedDictionary>(dictionary_name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }

View File

@ -51,9 +51,9 @@ public:
bool isCached() const override { return false; }
std::unique_ptr<IExternalLoadable> clone() const override
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_unique<TrieDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty);
return std::make_shared<TrieDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }

View File

@ -11,6 +11,7 @@
#include <Common/ColumnsHashing.h>
#include <Common/HashTable/ClearableHashMap.h>
// for better debug: #include <Core/iostream_debug_helpers.h>
/** The function will enumerate distinct values of the passed multidimensional arrays looking inside at the specified depths.
* This is very unusual function made as a special order for Yandex.Metrica.
@ -335,6 +336,7 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeMethodImpl(
if (prev_off == off)
{
want_clear = true;
if (arrays_depths.max_array_depth > 1)
++indices_by_depth[0];
for (ssize_t depth = current_offset_depth - 1; depth >= 0; --depth)

View File

@ -537,9 +537,9 @@ bool CatBoostModel::isModified() const
return true;
}
std::unique_ptr<IExternalLoadable> CatBoostModel::clone() const
std::shared_ptr<const IExternalLoadable> CatBoostModel::clone() const
{
return std::make_unique<CatBoostModel>(name, model_path, lib_path, lifetime);
return std::make_shared<CatBoostModel>(name, model_path, lib_path, lifetime);
}
size_t CatBoostModel::getFloatFeaturesCount() const

View File

@ -66,7 +66,7 @@ public:
bool isModified() const override;
std::unique_ptr<IExternalLoadable> clone() const override;
std::shared_ptr<const IExternalLoadable> clone() const override;
std::chrono::time_point<std::chrono::system_clock> getCreationTime() const override { return creation_time; }
std::exception_ptr getCreationException() const override { return creation_exception; }

View File

@ -105,7 +105,7 @@ struct ContextShared
mutable std::recursive_mutex mutex;
/// Separate mutex for access of dictionaries. Separate mutex to avoid locks when server doing request to itself.
mutable std::mutex embedded_dictionaries_mutex;
mutable std::recursive_mutex external_dictionaries_mutex;
mutable std::mutex external_dictionaries_mutex;
mutable std::mutex external_models_mutex;
/// Separate mutex for re-initialization of zookeer session. This operation could take a long time and must not interfere with another operations.
mutable std::mutex zookeeper_mutex;
@ -892,6 +892,10 @@ StoragePtr Context::tryGetTable(const String & database_name, const String & tab
StoragePtr Context::getTableImpl(const String & database_name, const String & table_name, Exception * exception) const
{
String db;
DatabasePtr database;
{
auto lock = getLock();
if (database_name.empty())
@ -901,7 +905,7 @@ StoragePtr Context::getTableImpl(const String & database_name, const String & ta
return res;
}
String db = resolveDatabase(database_name, current_database);
db = resolveDatabase(database_name, current_database);
checkDatabaseAccessRightsImpl(db);
Databases::const_iterator it = shared->databases.find(db);
@ -912,7 +916,10 @@ StoragePtr Context::getTableImpl(const String & database_name, const String & ta
return {};
}
auto table = it->second->tryGetTable(*this, table_name);
database = it->second;
}
auto table = database->tryGetTable(*this, table_name);
if (!table)
{
if (exception)
@ -1253,23 +1260,48 @@ EmbeddedDictionaries & Context::getEmbeddedDictionaries()
const ExternalDictionaries & Context::getExternalDictionaries() const
{
return getExternalDictionariesImpl(false);
{
std::lock_guard lock(shared->external_dictionaries_mutex);
if (shared->external_dictionaries)
return *shared->external_dictionaries;
}
const auto & config = getConfigRef();
std::lock_guard lock(shared->external_dictionaries_mutex);
if (!shared->external_dictionaries)
{
if (!this->global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
auto config_repository = shared->runtime_components_factory->createExternalDictionariesConfigRepository();
shared->external_dictionaries.emplace(std::move(config_repository), config, *this->global_context);
}
return *shared->external_dictionaries;
}
ExternalDictionaries & Context::getExternalDictionaries()
{
return getExternalDictionariesImpl(false);
return const_cast<ExternalDictionaries &>(const_cast<const Context *>(this)->getExternalDictionaries());
}
const ExternalModels & Context::getExternalModels() const
{
return getExternalModelsImpl(false);
std::lock_guard lock(shared->external_models_mutex);
if (!shared->external_models)
{
if (!this->global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
auto config_repository = shared->runtime_components_factory->createExternalModelsConfigRepository();
shared->external_models.emplace(std::move(config_repository), *this->global_context);
}
return *shared->external_models;
}
ExternalModels & Context::getExternalModels()
{
return getExternalModelsImpl(false);
return const_cast<ExternalModels &>(const_cast<const Context *>(this)->getExternalModels());
}
@ -1291,61 +1323,12 @@ EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_
}
ExternalDictionaries & Context::getExternalDictionariesImpl(const bool throw_on_error) const
{
{
std::lock_guard lock(shared->external_dictionaries_mutex);
if (shared->external_dictionaries)
return *shared->external_dictionaries;
}
const auto & config = getConfigRef();
std::lock_guard lock(shared->external_dictionaries_mutex);
if (!shared->external_dictionaries)
{
if (!this->global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
auto config_repository = shared->runtime_components_factory->createExternalDictionariesConfigRepository();
shared->external_dictionaries.emplace(std::move(config_repository), config, *this->global_context);
shared->external_dictionaries->init(throw_on_error);
}
return *shared->external_dictionaries;
}
ExternalModels & Context::getExternalModelsImpl(bool throw_on_error) const
{
std::lock_guard lock(shared->external_models_mutex);
if (!shared->external_models)
{
if (!this->global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
auto config_repository = shared->runtime_components_factory->createExternalModelsConfigRepository();
shared->external_models.emplace(std::move(config_repository), *this->global_context);
shared->external_models->init(throw_on_error);
}
return *shared->external_models;
}
void Context::tryCreateEmbeddedDictionaries() const
{
static_cast<void>(getEmbeddedDictionariesImpl(true));
}
void Context::tryCreateExternalDictionaries() const
{
static_cast<void>(getExternalDictionariesImpl(true));
}
void Context::tryCreateExternalModels() const
{
static_cast<void>(getExternalModelsImpl(true));
}
void Context::setProgressCallback(ProgressCallback callback)
{
/// Callback is set to a session or to a query. In the session, only one query is processed at a time. Therefore, the lock is not needed.

View File

@ -285,8 +285,6 @@ public:
ExternalDictionaries & getExternalDictionaries();
ExternalModels & getExternalModels();
void tryCreateEmbeddedDictionaries() const;
void tryCreateExternalDictionaries() const;
void tryCreateExternalModels() const;
/// I/O formats.
BlockInputStreamPtr getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size) const;
@ -501,8 +499,6 @@ private:
void setProfile(const String & profile);
EmbeddedDictionaries & getEmbeddedDictionariesImpl(bool throw_on_error) const;
ExternalDictionaries & getExternalDictionariesImpl(bool throw_on_error) const;
ExternalModels & getExternalModelsImpl(bool throw_on_error) const;
StoragePtr getTableImpl(const String & database_name, const String & table_name, Exception * exception) const;

View File

@ -5,47 +5,26 @@
namespace DB
{
namespace
{
const ExternalLoaderUpdateSettings externalDictionariesUpdateSettings {};
const ExternalLoaderConfigSettings & getExternalDictionariesConfigSettings()
{
static ExternalLoaderConfigSettings settings;
static std::once_flag flag;
std::call_once(flag, []
{
settings.external_config = "dictionary";
settings.external_name = "name";
settings.path_setting_name = "dictionaries_config";
});
return settings;
}
}
/// Must not acquire Context lock in constructor to avoid possibility of deadlocks.
ExternalDictionaries::ExternalDictionaries(
std::unique_ptr<IExternalLoaderConfigRepository> config_repository,
const Poco::Util::AbstractConfiguration & config,
Context & context)
: ExternalLoader(config,
externalDictionariesUpdateSettings,
getExternalDictionariesConfigSettings(),
std::move(config_repository),
&Logger::get("ExternalDictionaries"),
"external dictionary"),
"external dictionary",
&Logger::get("ExternalDictionaries")),
context(context)
{
addConfigRepository(std::move(config_repository), {"dictionary", "name", "dictionaries_config"});
enableAsyncLoading(true);
enablePeriodicUpdates(true);
}
std::unique_ptr<IExternalLoadable> ExternalDictionaries::create(
const std::string & name, const Configuration & config, const std::string & config_prefix) const
ExternalLoader::LoadablePtr ExternalDictionaries::create(
const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config) const
{
return DictionaryFactory::instance().create(name, config, config_prefix, context);
return DictionaryFactory::instance().create(name, config, key_in_config, context);
}
}

View File

@ -15,7 +15,7 @@ class Context;
class ExternalDictionaries : public ExternalLoader
{
public:
using DictPtr = std::shared_ptr<IDictionaryBase>;
using DictPtr = std::shared_ptr<const IDictionaryBase>;
/// Dictionaries will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds.
ExternalDictionaries(
@ -23,25 +23,19 @@ public:
const Poco::Util::AbstractConfiguration & config,
Context & context);
/// Forcibly reloads specified dictionary.
void reloadDictionary(const std::string & name) { reload(name); }
DictPtr getDictionary(const std::string & name) const
{
return std::static_pointer_cast<IDictionaryBase>(getLoadable(name));
return std::static_pointer_cast<const IDictionaryBase>(getLoadable(name));
}
DictPtr tryGetDictionary(const std::string & name) const
{
return std::static_pointer_cast<IDictionaryBase>(tryGetLoadable(name));
return std::static_pointer_cast<const IDictionaryBase>(tryGetLoadable(name));
}
protected:
std::unique_ptr<IExternalLoadable> create(const std::string & name, const Configuration & config,
const std::string & config_prefix) const override;
using ExternalLoader::getObjectsMap;
LoadablePtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config,
const std::string & key_in_config) const override;
friend class StorageSystemDictionaries;
friend class DatabaseDictionary;

File diff suppressed because it is too large Load Diff

View File

@ -1,24 +1,16 @@
#pragma once
#include <common/logger_useful.h>
#include <Poco/Event.h>
#include <mutex>
#include <thread>
#include <unordered_map>
#include <unordered_set>
#include <chrono>
#include <tuple>
#include <functional>
#include <unordered_map>
#include <Core/Types.h>
#include <Interpreters/IExternalLoadable.h>
#include <Interpreters/IExternalLoaderConfigRepository.h>
#include <Core/Types.h>
#include <pcg_random.hpp>
#include <Common/randomSeed.h>
#include <Common/ThreadPool.h>
#include <common/logger_useful.h>
namespace DB
{
struct ExternalLoaderUpdateSettings
{
UInt64 check_period_sec = 5;
@ -28,9 +20,7 @@ struct ExternalLoaderUpdateSettings
ExternalLoaderUpdateSettings() = default;
ExternalLoaderUpdateSettings(UInt64 check_period_sec, UInt64 backoff_initial_sec, UInt64 backoff_max_sec)
: check_period_sec(check_period_sec),
backoff_initial_sec(backoff_initial_sec),
backoff_max_sec(backoff_max_sec) {}
: check_period_sec(check_period_sec), backoff_initial_sec(backoff_initial_sec), backoff_max_sec(backoff_max_sec) {}
};
@ -51,154 +41,171 @@ struct ExternalLoaderConfigSettings
std::string path_setting_name;
};
/** Manages user-defined objects.
* Monitors configuration file and automatically reloads objects in a separate thread.
* The monitoring thread wakes up every 'check_period_sec' seconds and checks
* modification time of objects' configuration file. If said time is greater than
* 'config_last_modified', the objects are created from scratch using configuration file,
* possibly overriding currently existing objects with the same name (previous versions of
* overridden objects will live as long as there are any users retaining them).
*
* Apart from checking configuration file for modifications, each object
* has a lifetime of its own and may be updated if it supportUpdates.
* The time of next update is calculated by choosing uniformly a random number
* distributed between lifetime.min_sec and lifetime.max_sec.
* If either of lifetime.min_sec and lifetime.max_sec is zero, such object is never updated.
*/
* Monitors configuration file and automatically reloads objects in separate threads.
* The monitoring thread wakes up every 'check_period_sec' seconds and checks
* modification time of objects' configuration file. If said time is greater than
* 'config_last_modified', the objects are created from scratch using configuration file,
* possibly overriding currently existing objects with the same name (previous versions of
* overridden objects will live as long as there are any users retaining them).
*
* Apart from checking configuration file for modifications, each object
* has a lifetime of its own and may be updated if it supportUpdates.
* The time of next update is calculated by choosing uniformly a random number
* distributed between lifetime.min_sec and lifetime.max_sec.
* If either of lifetime.min_sec and lifetime.max_sec is zero, such object is never updated.
*/
class ExternalLoader
{
public:
using LoadablePtr = std::shared_ptr<IExternalLoadable>;
using LoadablePtr = std::shared_ptr<const IExternalLoadable>;
using Loadables = std::vector<LoadablePtr>;
private:
struct LoadableInfo final
enum class Status
{
LoadablePtr loadable;
std::string origin;
NOT_LOADED, /// Object hasn't been tried to load. This is an initial state.
LOADED, /// Object has been loaded successfully.
FAILED, /// Object has been failed to load.
LOADING, /// Object is being loaded right now for the first time.
FAILED_AND_RELOADING, /// Object was failed to load before and it's being reloaded right now.
LOADED_AND_RELOADING, /// Object was loaded successfully before and it's being reloaded right now.
NOT_EXIST, /// Object with this name wasn't found in the configuration.
};
static std::vector<std::pair<String, Int8>> getStatusEnumAllPossibleValues();
using Duration = std::chrono::milliseconds;
using TimePoint = std::chrono::system_clock::time_point;
struct LoadResult
{
LoadResult(Status status_) : status(status_) {}
Status status;
LoadablePtr object;
String origin;
TimePoint loading_start_time;
Duration loading_duration;
std::exception_ptr exception;
};
struct FailedLoadableInfo final
{
std::unique_ptr<IExternalLoadable> loadable;
std::chrono::system_clock::time_point next_attempt_time;
UInt64 error_count;
};
using LoadResults = std::vector<std::pair<String, LoadResult>>;
public:
using Configuration = Poco::Util::AbstractConfiguration;
using ObjectsMap = std::unordered_map<std::string, LoadableInfo>;
/// Call init() after constructing the instance of any derived class.
ExternalLoader(const Configuration & config_main,
const ExternalLoaderUpdateSettings & update_settings,
const ExternalLoaderConfigSettings & config_settings,
std::unique_ptr<IExternalLoaderConfigRepository> config_repository,
Logger * log, const std::string & loadable_object_name);
ExternalLoader(const Poco::Util::AbstractConfiguration & main_config, const String & type_name_, Logger * log);
virtual ~ExternalLoader();
/// Should be called after creating an instance of a derived class.
/// Loads the objects immediately and starts a separate thread to update them once in each 'reload_period' seconds.
/// This function does nothing if called again.
void init(bool throw_on_error);
/// Adds a repository which will be used to read configurations from.
void addConfigRepository(
std::unique_ptr<IExternalLoaderConfigRepository> config_repository, const ExternalLoaderConfigSettings & config_settings);
/// Forcibly reloads all loadable objects.
void reload();
/// Sets whether all the objects from the configuration should be always loaded (even those which are never used).
void enableAlwaysLoadEverything(bool enable);
/// Forcibly reloads specified loadable object.
void reload(const std::string & name);
/// Sets whether the objects should be loaded asynchronously, each loading in a new thread (from the thread pool).
void enableAsyncLoading(bool enable);
LoadablePtr getLoadable(const std::string & name) const;
LoadablePtr tryGetLoadable(const std::string & name) const;
/// Sets settings for periodic updates.
void enablePeriodicUpdates(bool enable, const ExternalLoaderUpdateSettings & settings = {});
/// Returns the names of all the objects in the configuration (loaded or not).
std::vector<String> getNames() const;
size_t getNumberOfNames() const;
/// Returns the status of the object.
/// If the object has not been loaded yet then the function returns Status::NOT_LOADED.
/// If the specified name isn't found in the configuration then the function returns Status::NOT_EXIST.
Status getCurrentStatus(const String & name) const;
/// Returns the result of loading the object.
/// The function doesn't load anything, it just returns the current load result as is.
LoadResult getCurrentLoadResult(const String & name) const;
using FilterByNameFunction = std::function<bool(const String &)>;
/// Returns all the load results as a map.
/// The function doesn't load anything, it just returns the current load results as is.
LoadResults getCurrentLoadResults() const;
LoadResults getCurrentLoadResults(const FilterByNameFunction & filter_by_name) const;
/// Returns all loaded objects as a map.
/// The function doesn't load anything, it just returns the current load results as is.
Loadables getCurrentlyLoadedObjects() const;
Loadables getCurrentlyLoadedObjects(const FilterByNameFunction & filter_by_name) const;
size_t getNumberOfCurrentlyLoadedObjects() const;
static constexpr Duration NO_TIMEOUT = Duration::max();
/// Starts loading of a specified object.
void load(const String & name) const;
/// Tries to finish loading of a specified object during the timeout.
/// Returns nullptr if the loading is unsuccessful or if there is no such object.
void load(const String & name, LoadablePtr & loaded_object, Duration timeout = NO_TIMEOUT) const;
void load(const String & name, LoadResult & load_result, Duration timeout = NO_TIMEOUT) const;
LoadablePtr loadAndGet(const String & name, Duration timeout = NO_TIMEOUT) const { LoadablePtr object; load(name, object, timeout); return object; }
LoadablePtr tryGetLoadable(const String & name) const { return loadAndGet(name); }
/// Tries to finish loading of a specified object during the timeout.
/// Throws an exception if the loading is unsuccessful or if there is no such object.
void loadStrict(const String & name, LoadablePtr & loaded_object) const;
void loadStrict(const String & name, LoadResult & load_result) const;
LoadablePtr loadAndGetStrict(const String & name) const { LoadablePtr object; loadStrict(name, object); return object; }
LoadablePtr getLoadable(const String & name) const { return loadAndGetStrict(name); }
/// Tries to start loading of the objects for which the specified function returns true.
void load(const FilterByNameFunction & filter_by_name) const;
/// Tries to finish loading of the objects for which the specified function returns true.
void load(const FilterByNameFunction & filter_by_name, Loadables & loaded_objects, Duration timeout = NO_TIMEOUT) const;
void load(const FilterByNameFunction & filter_by_name, LoadResults & load_results, Duration timeout = NO_TIMEOUT) const;
Loadables loadAndGet(const FilterByNameFunction & filter_by_name, Duration timeout = NO_TIMEOUT) const { Loadables loaded_objects; load(filter_by_name, loaded_objects, timeout); return loaded_objects; }
/// Starts loading of all the objects.
void load() const;
/// Tries to finish loading of all the objects during the timeout.
void load(Loadables & loaded_objects, Duration timeout = NO_TIMEOUT) const;
void load(LoadResults & load_results, Duration timeout = NO_TIMEOUT) const;
/// Starts reloading of a specified object.
/// `load_never_loading` specifies what to do if the object has never been loading before.
/// The function can either skip it (false) or load for the first time (true).
void reload(const String & name, bool load_never_loading = false);
/// Starts reloading of the objects for which the specified function returns true.
/// `load_never_loading` specifies what to do with the objects which have never been loading before.
/// The function can either skip them (false) or load for the first time (true).
void reload(const FilterByNameFunction & filter_by_name, bool load_never_loading = false);
/// Starts reloading of all the objects.
/// `load_never_loading` specifies what to do with the objects which have never been loading before.
/// The function can either skip them (false) or load for the first time (true).
void reload(bool load_never_loading = false);
protected:
virtual std::unique_ptr<IExternalLoadable> create(const std::string & name, const Configuration & config,
const std::string & config_prefix) const = 0;
class LockedObjectsMap
{
public:
LockedObjectsMap(std::mutex & mutex, const ObjectsMap & objects_map) : lock(mutex), objects_map(objects_map) {}
const ObjectsMap & get() { return objects_map; }
private:
std::unique_lock<std::mutex> lock;
const ObjectsMap & objects_map;
};
/// Direct access to objects.
LockedObjectsMap getObjectsMap() const;
virtual LoadablePtr create(const String & name, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) const = 0;
private:
std::once_flag is_initialized_flag;
struct ObjectConfig;
using ObjectWithException = std::pair<LoadablePtr, std::exception_ptr>;
/// Protects only objects map.
/** Reading and assignment of "loadable" should be done under mutex.
* Creating new versions of "loadable" should not be done under mutex.
*/
mutable std::mutex map_mutex;
ObjectWithException
createObject(const String & name, const ObjectConfig & config, bool config_changed, const LoadablePtr & previous_version) const;
TimePoint calculateNextUpdateTime(const LoadablePtr & loaded_object, size_t error_count) const;
/// Protects all data, currently used to avoid races between updating thread and SYSTEM queries.
/// The mutex is recursive because creating of objects might be recursive, i.e.
/// creating objects might cause creating other objects.
mutable std::recursive_mutex all_mutex;
class ConfigFilesReader;
std::unique_ptr<ConfigFilesReader> config_files_reader;
/// name -> loadable.
mutable ObjectsMap loadable_objects;
class LoadingDispatcher;
std::unique_ptr<LoadingDispatcher> loading_dispatcher;
struct LoadableCreationInfo
{
std::string name;
Poco::AutoPtr<Poco::Util::AbstractConfiguration> config;
std::string config_path;
std::string config_prefix;
};
class PeriodicUpdater;
std::unique_ptr<PeriodicUpdater> periodic_updater;
/// Objects which should be reloaded soon.
mutable std::unordered_map<std::string, LoadableCreationInfo> objects_to_reload;
/// Here are loadable objects, that has been never loaded successfully.
/// They are also in 'loadable_objects', but with nullptr as 'loadable'.
mutable std::unordered_map<std::string, FailedLoadableInfo> failed_loadable_objects;
/// Both for loadable_objects and failed_loadable_objects.
mutable std::unordered_map<std::string, std::chrono::system_clock::time_point> update_times;
std::unordered_map<std::string, std::unordered_set<std::string>> loadable_objects_defined_in_config;
mutable pcg64 rnd_engine{randomSeed()};
const Configuration & config_main;
const ExternalLoaderUpdateSettings & update_settings;
const ExternalLoaderConfigSettings & config_settings;
std::unique_ptr<IExternalLoaderConfigRepository> config_repository;
ThreadFromGlobalPool reloading_thread;
Poco::Event destroy;
Logger * log;
/// Loadable object name to use in log messages.
std::string object_name;
std::unordered_map<std::string, Poco::Timestamp> last_modification_times;
void initImpl(bool throw_on_error);
/// Check objects definitions in config files and reload or/and add new ones if the definition is changed
/// If loadable_name is not empty, load only loadable object with name loadable_name
void reloadFromConfigFiles(bool throw_on_error, bool force_reload = false, const std::string & loadable_name = "");
void reloadFromConfigFile(const std::string & config_path, const bool force_reload, const std::string & loadable_name);
/// Check config files and update expired loadable objects
void reloadAndUpdate(bool throw_on_error = false);
void reloadPeriodically();
void finishReload(const std::string & loadable_name, bool throw_on_error) const;
void finishAllReloads(bool throw_on_error) const;
void finishReloadImpl(const LoadableCreationInfo & creation_info, bool throw_on_error) const;
LoadablePtr getLoadableImpl(const std::string & name, bool throw_on_error) const;
const String type_name;
};
String toString(ExternalLoader::Status status);
std::ostream & operator<<(std::ostream & out, ExternalLoader::Status status);
}

View File

@ -9,43 +9,21 @@ namespace ErrorCodes
extern const int INVALID_CONFIG_PARAMETER;
}
namespace
{
const ExternalLoaderUpdateSettings externalModelsUpdateSettings { };
const ExternalLoaderConfigSettings & getExternalModelsConfigSettings()
{
static ExternalLoaderConfigSettings settings;
static std::once_flag flag;
std::call_once(flag, []
{
settings.external_config = "model";
settings.external_name = "name";
settings.path_setting_name = "models_config";
});
return settings;
}
}
ExternalModels::ExternalModels(
std::unique_ptr<IExternalLoaderConfigRepository> config_repository,
Context & context)
: ExternalLoader(context.getConfigRef(),
externalModelsUpdateSettings,
getExternalModelsConfigSettings(),
std::move(config_repository),
&Logger::get("ExternalModels"),
"external model"),
"external model",
&Logger::get("ExternalModels")),
context(context)
{
addConfigRepository(std::move(config_repository), {"model", "name", "models_config"});
enablePeriodicUpdates(true);
}
std::unique_ptr<IExternalLoadable> ExternalModels::create(
const std::string & name, const Configuration & config, const std::string & config_prefix) const
std::shared_ptr<const IExternalLoadable> ExternalModels::create(
const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) const
{
String type = config.getString(config_prefix + ".type");
ExternalLoadableLifetime lifetime(config, config_prefix + ".lifetime");

View File

@ -15,27 +15,21 @@ class Context;
class ExternalModels : public ExternalLoader
{
public:
using ModelPtr = std::shared_ptr<IModel>;
using ModelPtr = std::shared_ptr<const IModel>;
/// Models will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds.
ExternalModels(
std::unique_ptr<IExternalLoaderConfigRepository> config_repository,
Context & context);
/// Forcibly reloads specified model.
void reloadModel(const std::string & name) { reload(name); }
ModelPtr getModel(const std::string & name) const
{
return std::static_pointer_cast<IModel>(getLoadable(name));
return std::static_pointer_cast<const IModel>(getLoadable(name));
}
protected:
std::unique_ptr<IExternalLoadable> create(const std::string & name, const Configuration & config,
const std::string & config_prefix) const override;
using ExternalLoader::getObjectsMap;
LoadablePtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config,
const std::string & key_in_config) const override;
friend class StorageSystemModels;
private:

View File

@ -0,0 +1,19 @@
#include <Interpreters/IExternalLoadable.h>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
ExternalLoadableLifetime::ExternalLoadableLifetime(const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix)
{
const auto & lifetime_min_key = config_prefix + ".min";
const auto has_min = config.has(lifetime_min_key);
min_sec = has_min ? config.getUInt64(lifetime_min_key) : config.getUInt64(config_prefix);
max_sec = has_min ? config.getUInt64(config_prefix + ".max") : min_sec;
}
}

View File

@ -17,7 +17,7 @@ namespace DB
{
/// Min and max lifetimes for a loadable object or it's entry
struct ExternalLoadableLifetime final
struct ExternalLoadableLifetime
{
UInt64 min_sec;
UInt64 max_sec;
@ -40,7 +40,7 @@ public:
/// If lifetime exceeded and isModified(), ExternalLoader replace current object with the result of clone().
virtual bool isModified() const = 0;
/// Returns new object with the same configuration. Is used to update modified object when lifetime exceeded.
virtual std::unique_ptr<IExternalLoadable> clone() const = 0;
virtual std::shared_ptr<const IExternalLoadable> clone() const = 0;
virtual std::chrono::time_point<std::chrono::system_clock> getCreationTime() const = 0;

View File

@ -156,7 +156,7 @@ BlockIO InterpreterSystemQuery::execute()
break;
#endif
case Type::RELOAD_DICTIONARY:
system_context.getExternalDictionaries().reloadDictionary(query.target_dictionary);
system_context.getExternalDictionaries().reload(query.target_dictionary, true /* load the dictionary even if it wasn't loading before */);
break;
case Type::RELOAD_DICTIONARIES:
executeCommandsAndThrowIfError(

View File

@ -105,17 +105,13 @@ bool StorageMerge::hasColumn(const String & column_name) const
template <typename F>
StoragePtr StorageMerge::getFirstTable(F && predicate) const
{
auto database = global_context.getDatabase(source_database);
auto iterator = database->getIterator(global_context);
auto iterator = getDatabaseIterator(global_context);
while (iterator->isValid())
{
if (table_name_regexp.match(iterator->name()))
{
auto & table = iterator->table();
if (table.get() != this && predicate(table))
return table;
}
iterator->next();
}
@ -156,14 +152,11 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context &
{
auto stage_in_source_tables = QueryProcessingStage::FetchColumns;
DatabasePtr database = context.getDatabase(source_database);
DatabaseIteratorPtr iterator = database->getIterator(context);
DatabaseIteratorPtr iterator = getDatabaseIterator(context);
size_t selected_table_size = 0;
while (iterator->isValid())
{
if (table_name_regexp.match(iterator->name()))
{
auto & table = iterator->table();
if (table.get() != this)
@ -172,8 +165,6 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context &
stage_in_source_tables = std::max(stage_in_source_tables, table->getQueryProcessingStage(context));
}
}
iterator->next();
}
@ -347,20 +338,17 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer
return source_streams;
}
StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const String & query_id) const
{
StorageListWithLocks selected_tables;
auto database = global_context.getDatabase(source_database);
auto iterator = database->getIterator(global_context);
auto iterator = getDatabaseIterator(global_context);
while (iterator->isValid())
{
if (table_name_regexp.match(iterator->name()))
{
auto & table = iterator->table();
if (table.get() != this)
selected_tables.emplace_back(table, table->lockStructureForShare(false, query_id));
}
iterator->next();
}
@ -372,14 +360,11 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const String
StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr & query, bool has_virtual_column, bool get_lock, const String & query_id) const
{
StorageListWithLocks selected_tables;
DatabasePtr database = global_context.getDatabase(source_database);
DatabaseIteratorPtr iterator = database->getIterator(global_context);
DatabaseIteratorPtr iterator = getDatabaseIterator(global_context);
auto virtual_column = ColumnString::create();
while (iterator->isValid())
{
if (table_name_regexp.match(iterator->name()))
{
StoragePtr storage = iterator->table();
@ -391,7 +376,6 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr
virtual_column->insert(storage->getTableName());
selected_tables.emplace_back(storage, get_lock ? storage->lockStructureForShare(false, query_id) : TableStructureReadLockHolder{});
}
}
iterator->next();
}
@ -409,6 +393,15 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr
return selected_tables;
}
DatabaseIteratorPtr StorageMerge::getDatabaseIterator(const Context & context) const
{
auto database = context.getDatabase(source_database);
auto table_name_match = [this](const String & table_name) { return table_name_regexp.match(table_name); };
return database->getIterator(global_context, table_name_match);
}
void StorageMerge::alter(
const AlterCommands & params, const String & database_name, const String & table_name,
const Context & context, TableStructureWriteLockHolder & table_lock_holder)

View File

@ -65,6 +65,8 @@ private:
template <typename F>
StoragePtr getFirstTable(F && predicate) const;
DatabaseIteratorPtr getDatabaseIterator(const Context & context) const;
protected:
StorageMerge(
const std::string & name_,

View File

@ -2,6 +2,7 @@
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeEnum.h>
#include <Dictionaries/IDictionary.h>
#include <Dictionaries/IDictionarySource.h>
#include <Dictionaries/DictionaryStructure.h>
@ -19,6 +20,7 @@ NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes()
{
return {
{ "name", std::make_shared<DataTypeString>() },
{ "status", std::make_shared<DataTypeEnum8>(ExternalLoader::getStatusEnumAllPossibleValues()) },
{ "origin", std::make_shared<DataTypeString>() },
{ "type", std::make_shared<DataTypeString>() },
{ "key", std::make_shared<DataTypeString>() },
@ -29,8 +31,10 @@ NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes()
{ "hit_rate", std::make_shared<DataTypeFloat64>() },
{ "element_count", std::make_shared<DataTypeUInt64>() },
{ "load_factor", std::make_shared<DataTypeFloat64>() },
{ "creation_time", std::make_shared<DataTypeDateTime>() },
{ "source", std::make_shared<DataTypeString>() },
{ "loading_start_time", std::make_shared<DataTypeDateTime>() },
{ "loading_duration", std::make_shared<DataTypeFloat32>() },
//{ "creation_time", std::make_shared<DataTypeDateTime>() },
{ "last_exception", std::make_shared<DataTypeString>() },
};
}
@ -38,18 +42,17 @@ NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes()
void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const
{
const auto & external_dictionaries = context.getExternalDictionaries();
auto objects_map = external_dictionaries.getObjectsMap();
const auto & dictionaries = objects_map.get();
for (const auto & dict_info : dictionaries)
for (const auto & [dict_name, load_result] : external_dictionaries.getCurrentLoadResults())
{
size_t i = 0;
res_columns[i++]->insert(dict_info.first);
res_columns[i++]->insert(dict_info.second.origin);
res_columns[i++]->insert(dict_name);
res_columns[i++]->insert(static_cast<Int8>(load_result.status));
res_columns[i++]->insert(load_result.origin);
if (dict_info.second.loadable)
if (load_result.object)
{
const auto dict_ptr = std::static_pointer_cast<IDictionaryBase>(dict_info.second.loadable);
const auto dict_ptr = std::static_pointer_cast<const IDictionaryBase>(load_result.object);
res_columns[i++]->insert(dict_ptr->getTypeName());
@ -62,26 +65,19 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Con
res_columns[i++]->insert(dict_ptr->getHitRate());
res_columns[i++]->insert(dict_ptr->getElementCount());
res_columns[i++]->insert(dict_ptr->getLoadFactor());
res_columns[i++]->insert(static_cast<UInt64>(std::chrono::system_clock::to_time_t(dict_ptr->getCreationTime())));
res_columns[i++]->insert(dict_ptr->getSource()->toString());
}
else
{
while (i < 13)
for (size_t j = 0; j != 10; ++j)
res_columns[i++]->insertDefault();
}
if (dict_info.second.exception)
{
try
{
std::rethrow_exception(dict_info.second.exception);
}
catch (...)
{
res_columns[i++]->insert(getCurrentExceptionMessage(false));
}
}
res_columns[i++]->insert(static_cast<UInt64>(std::chrono::system_clock::to_time_t(load_result.loading_start_time)));
res_columns[i++]->insert(std::chrono::duration_cast<std::chrono::duration<float>>(load_result.loading_duration).count());
if (load_result.exception)
res_columns[i++]->insert(getExceptionMessage(load_result.exception, false));
else
res_columns[i++]->insertDefault();
}

View File

@ -2,9 +2,12 @@
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeEnum.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExternalModels.h>
#include <Interpreters/CatBoostModel.h>
namespace DB
{
@ -12,9 +15,12 @@ NamesAndTypesList StorageSystemModels::getNamesAndTypes()
{
return {
{ "name", std::make_shared<DataTypeString>() },
{ "status", std::make_shared<DataTypeEnum8>(ExternalLoader::getStatusEnumAllPossibleValues()) },
{ "origin", std::make_shared<DataTypeString>() },
{ "type", std::make_shared<DataTypeString>() },
{ "creation_time", std::make_shared<DataTypeDateTime>() },
{ "loading_start_time", std::make_shared<DataTypeDateTime>() },
{ "loading_duration", std::make_shared<DataTypeFloat32>() },
//{ "creation_time", std::make_shared<DataTypeDateTime>() },
{ "last_exception", std::make_shared<DataTypeString>() },
};
}
@ -22,40 +28,31 @@ NamesAndTypesList StorageSystemModels::getNamesAndTypes()
void StorageSystemModels::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const
{
const auto & external_models = context.getExternalModels();
auto objects_map = external_models.getObjectsMap();
const auto & models = objects_map.get();
auto load_results = external_models.getCurrentLoadResults();
for (const auto & model_info : models)
for (const auto & [name, load_result] : load_results)
{
res_columns[0]->insert(model_info.first);
res_columns[1]->insert(model_info.second.origin);
res_columns[0]->insert(name);
res_columns[1]->insert(static_cast<Int8>(load_result.status));
res_columns[2]->insert(load_result.origin);
if (model_info.second.loadable)
if (load_result.object)
{
const auto model_ptr = std::static_pointer_cast<IModel>(model_info.second.loadable);
res_columns[2]->insert(model_ptr->getTypeName());
res_columns[3]->insert(static_cast<UInt64>(std::chrono::system_clock::to_time_t(model_ptr->getCreationTime())));
const auto model_ptr = std::static_pointer_cast<const IModel>(load_result.object);
res_columns[3]->insert(model_ptr->getTypeName());
}
else
{
res_columns[2]->insertDefault();
res_columns[3]->insertDefault();
}
if (model_info.second.exception)
{
try
{
std::rethrow_exception(model_info.second.exception);
}
catch (...)
{
res_columns[4]->insert(getCurrentExceptionMessage(false));
}
}
res_columns[4]->insert(static_cast<UInt64>(std::chrono::system_clock::to_time_t(load_result.loading_start_time)));
res_columns[5]->insert(std::chrono::duration_cast<std::chrono::duration<float>>(load_result.loading_duration).count());
if (load_result.exception)
res_columns[6]->insert(getExceptionMessage(load_result.exception, false));
else
res_columns[4]->insertDefault();
res_columns[6]->insertDefault();
}
}

View File

@ -27,23 +27,16 @@ namespace ErrorCodes
static NamesAndTypesList chooseColumns(const String & source_database, const String & table_name_regexp_, const Context & context)
{
OptimizedRegularExpression table_name_regexp(table_name_regexp_);
auto table_name_match = [&](const String & table_name) { return table_name_regexp.match(table_name); };
StoragePtr any_table;
{
auto database = context.getDatabase(source_database);
auto iterator = database->getIterator(context);
auto iterator = database->getIterator(context, table_name_match);
while (iterator->isValid())
{
if (table_name_regexp.match(iterator->name()))
{
if (iterator->isValid())
any_table = iterator->table();
break;
}
iterator->next();
}
}
if (!any_table)

View File

@ -53,6 +53,10 @@ def run_single_test(args, ext, server_logs_level, case_file, stdout_file, stderr
while (datetime.now() - start_time).total_seconds() < args.timeout and proc.poll() is None:
sleep(0.01)
# Normalize randomized database names in stdout, stderr files.
os.system("sed -i 's/{test_db}/default/g' {file}".format(test_db=args.database, file=stdout_file))
os.system("sed -i 's/{test_db}/default/g' {file}".format(test_db=args.database, file=stderr_file))
stdout = open(stdout_file, 'r').read() if os.path.exists(stdout_file) else ''
stdout = unicode(stdout, errors='replace', encoding='utf-8')
stderr = open(stderr_file, 'r').read() if os.path.exists(stderr_file) else ''
@ -432,7 +436,7 @@ if __name__ == '__main__':
parser.add_argument('--testname', action='store_true', default=None, dest='testname', help='Make query with test name before test run')
parser.add_argument('--hung-check', action='store_true', default=False)
parser.add_argument('--force-color', action='store_true', default=False)
parser.add_argument('--database', default='test', help='Default database for tests')
parser.add_argument('--database', help='Database for tests (random name test_XXXXXX by default)')
parser.add_argument('--parallel', default='1/1', help='One parallel test run number/total')
parser.add_argument('-j', '--jobs', default=1, help='Run all tests in parallel', type=int) # default=multiprocessing.cpu_count()
@ -483,7 +487,13 @@ if __name__ == '__main__':
args.client += ' --database=' + os.getenv("CLICKHOUSE_DATABASE")
args.client_with_database = args.client
if args.database:
if not args.database:
def random_str(length=6):
import random
import string
alphabet = string.ascii_lowercase + string.digits
return ''.join(random.choice(alphabet) for _ in range(length))
args.database = 'test_{suffix}'.format(suffix=random_str())
args.client_with_database += ' --database=' + args.database
if args.extract_from_config is None:

View File

@ -7,7 +7,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>decimals</table>
</clickhouse>
</source>
@ -45,7 +45,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>decimals</table>
</clickhouse>
</source>
@ -83,7 +83,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>decimals</table>
</clickhouse>
</source>
@ -121,7 +121,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>decimals</table>
</clickhouse>
</source>
@ -162,7 +162,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>decimals</table>
</clickhouse>
</source>

View File

@ -7,7 +7,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>ints</table>
</clickhouse>
</source>
@ -70,7 +70,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>ints</table>
</clickhouse>
</source>
@ -133,7 +133,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>ints</table>
</clickhouse>
</source>
@ -196,7 +196,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>ints</table>
</clickhouse>
</source>
@ -262,7 +262,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>ints</table>
</clickhouse>
</source>

View File

@ -7,7 +7,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>strings</table>
</clickhouse>
</source>
@ -35,7 +35,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>strings</table>
</clickhouse>
</source>
@ -63,7 +63,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>strings</table>
</clickhouse>
</source>
@ -91,7 +91,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>strings</table>
</clickhouse>
</source>
@ -122,7 +122,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>strings</table>
</clickhouse>
</source>
@ -153,7 +153,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>strings</table>
</clickhouse>
</source>
@ -184,7 +184,7 @@
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<db>test_00950</db>
<table>strings</table>
</clickhouse>
</source>

View File

@ -0,0 +1,19 @@
<?xml version="1.0"?>
<dictionaries>
<dictionary>
<name>cmd</name>
<source>
<executable>
<command>echo '7\t8';</command>
<format>TabSeparated</format>
</executable>
</source>
<lifetime>0</lifetime>
<layout><flat/></layout>
<structure><id><name>key</name> </id>
<attribute><name>a</name><type>Int32</type>
<null_value>0</null_value>
</attribute>
</structure>
</dictionary>
</dictionaries>

View File

@ -0,0 +1,36 @@
<?xml version="1.0"?>
<dictionaries>
<dictionary>
<name>file</name>
<source>
<file>
<path>/etc/clickhouse-server/config.d/dictionary_preset_file.txt</path>
<format>TabSeparated</format>
</file>
</source>
<lifetime>1</lifetime>
<layout><flat/></layout>
<structure><id><name>key</name> </id>
<attribute><name>a</name><type>Int32</type>
<null_value>0</null_value>
</attribute>
</structure>
</dictionary>
<dictionary>
<name>no_file</name>
<source>
<file>
<path>/etc/clickhouse-server/config.d/dictionary_preset_no_file.txt</path>
<format>TabSeparated</format>
</file>
</source>
<lifetime>1</lifetime>
<layout><flat/></layout>
<structure><id><name>key</name> </id>
<attribute><name>a</name><type>Int32</type>
<null_value>0</null_value>
</attribute>
</structure>
</dictionary>
</dictionaries>

View File

@ -0,0 +1,19 @@
<?xml version="1.0"?>
<dictionaries>
<dictionary>
<name>longload</name>
<source>
<executable>
<command>sleep 100 &amp;&amp; echo '5\t6';</command>
<format>TabSeparated</format>
</executable>
</source>
<lifetime>0</lifetime>
<layout><flat/></layout>
<structure><id><name>key</name> </id>
<attribute><name>a</name><type>Int32</type>
<null_value>0</null_value>
</attribute>
</structure>
</dictionary>
</dictionaries>

View File

@ -183,7 +183,7 @@ def generate_dictionaries(path, structure):
file_names = []
# Add ready dictionaries.
file_names.extend(glob.glob(os.path.join(path, "*dictionary_preset*.xml")))
file_names.extend(glob.glob(os.path.join(path, "*dictionary_preset*")))
# Generate dictionaries.
for (name, key_idx, has_parent), (source, layout) in zip(structure, sources_and_layouts):

View File

@ -1,5 +1,6 @@
import pytest
import os
import time
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
@ -12,6 +13,25 @@ instance = None
test_table = None
def get_status(dictionary_name):
return instance.query("SELECT status FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n")
def get_loading_start_time(dictionary_name):
s = instance.query("SELECT loading_start_time FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n")
if s == "0000-00-00 00:00:00":
return None
return time.strptime(s, "%Y-%m-%d %H:%M:%S")
def get_loading_duration(dictionary_name):
return float(instance.query("SELECT loading_duration FROM system.dictionaries WHERE name='" + dictionary_name + "'"))
def replace_in_file_in_container(file_name, what, replace_with):
instance.exec_in_container('sed -i "s/' + what + '/' + replace_with + '/g" ' + file_name)
def setup_module(module):
global cluster
global instance
@ -139,9 +159,138 @@ def test_null_value(started_cluster):
def test_dictionary_dependency(started_cluster):
query = instance.query
# dictionaries_lazy_load == false, so these dictionary are not loaded.
assert get_status('dep_x') == 'NOT_LOADED'
assert get_status('dep_y') == 'NOT_LOADED'
assert get_status('dep_z') == 'NOT_LOADED'
# Dictionary 'dep_x' depends on 'dep_z', which depends on 'dep_y'.
# So they all should be loaded at once.
assert query("SELECT dictGetString('dep_x', 'String_', toUInt64(1))") == "10577349846663553072\n"
assert get_status('dep_x') == 'LOADED'
assert get_status('dep_y') == 'LOADED'
assert get_status('dep_z') == 'LOADED'
# Other dictionaries should work too.
assert query("SELECT dictGetString('dep_y', 'String_', toUInt64(1))") == "10577349846663553072\n"
assert query("SELECT dictGetString('dep_z', 'String_', toUInt64(1))") == "10577349846663553072\n"
assert query("SELECT dictGetString('dep_x', 'String_', toUInt64(12121212))") == "XX\n"
assert query("SELECT dictGetString('dep_y', 'String_', toUInt64(12121212))") == "YY\n"
assert query("SELECT dictGetString('dep_z', 'String_', toUInt64(12121212))") == "ZZ\n"
def test_reload_while_loading(started_cluster):
query = instance.query
# dictionaries_lazy_load == false, so this dictionary is not loaded.
assert get_status('longload') == "NOT_LOADED"
assert get_loading_duration('longload') == 0
# It's not possible to get a value from the dictionary within 1.0 second, so the following query fails by timeout.
assert query("SELECT dictGetInt32('longload', 'a', toUInt64(5))", timeout = 1, ignore_error = True) == ""
# The dictionary is now loading.
assert get_status('longload') == "LOADING"
start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload')
assert duration > 0
time.sleep(0.5) # Still loading.
assert get_status('longload') == "LOADING"
prev_start_time, prev_duration = start_time, duration
start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload')
assert start_time == prev_start_time
assert duration >= prev_duration
# SYSTEM RELOAD DICTIONARY should restart loading.
query("SYSTEM RELOAD DICTIONARY 'longload'")
assert get_status('longload') == "LOADING"
prev_start_time, prev_duration = start_time, duration
start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload')
assert start_time > prev_start_time
assert duration < prev_duration
time.sleep(0.5) # Still loading.
assert get_status('longload') == "LOADING"
prev_start_time, prev_duration = start_time, duration
start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload')
assert start_time == prev_start_time
assert duration >= prev_duration
# SYSTEM RELOAD DICTIONARIES should restart loading again.
query("SYSTEM RELOAD DICTIONARIES")
assert get_status('longload') == "LOADING"
prev_start_time, prev_duration = start_time, duration
start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload')
assert start_time > prev_start_time
assert duration < prev_duration
# Changing the configuration file should restart loading one more time.
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_longload.xml', 'sleep 100', 'sleep 0')
time.sleep(5) # Configuration files are reloaded once in 5 seconds.
# This time loading should finish quickly.
assert get_status('longload') == "LOADED"
assert query("SELECT dictGetInt32('longload', 'a', toUInt64(5))") == "6\n"
def test_reload_after_loading(started_cluster):
query = instance.query
assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "8\n"
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "10\n"
# Change the dictionaries' data.
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_cmd.xml', '8', '81')
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_file.txt', '10', '101')
# SYSTEM RELOAD 'name' reloads only the specified dictionary.
query("SYSTEM RELOAD DICTIONARY 'cmd'")
assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "81\n"
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "10\n"
query("SYSTEM RELOAD DICTIONARY 'file'")
assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "81\n"
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "101\n"
# SYSTEM RELOAD DICTIONARIES reloads all loaded dictionaries.
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_cmd.xml', '81', '82')
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_file.txt', '101', '102')
query("SYSTEM RELOAD DICTIONARIES")
assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "82\n"
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "102\n"
# Configuration files are reloaded and lifetimes are checked automatically once in 5 seconds.
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_cmd.xml', '82', '83')
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_file.txt', '102', '103')
time.sleep(5)
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "103\n"
assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "83\n"
def test_reload_after_fail(started_cluster):
query = instance.query
# dictionaries_lazy_load == false, so this dictionary is not loaded.
assert get_status("no_file") == "NOT_LOADED"
# We expect an error because the file source doesn't exist.
expected_error = "No such file"
assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file', 'a', toUInt64(9))")
assert get_status("no_file") == "FAILED"
# SYSTEM RELOAD should not change anything now, the status is still FAILED.
query("SYSTEM RELOAD DICTIONARY 'no_file'")
assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file', 'a', toUInt64(9))")
assert get_status("no_file") == "FAILED"
# Creating the file source makes the dictionary able to load.
instance.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/dictionaries/dictionary_preset_file.txt"), "/etc/clickhouse-server/config.d/dictionary_preset_no_file.txt")
query("SYSTEM RELOAD DICTIONARY 'no_file'")
query("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") == "10\n"
assert get_status("no_file") == "LOADED"
# Removing the file source should not spoil the loaded dictionary.
instance.exec_in_container("rm /etc/clickhouse-server/config.d/dictionary_preset_no_file.txt")
query("SYSTEM RELOAD DICTIONARY 'no_file'")
query("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") == "10\n"
assert get_status("no_file") == "LOADED"

View File

@ -9,3 +9,4 @@ SELECT s, n.x, n.y FROM nested_test ARRAY JOIN nest AS n;
SELECT s, n.x, n.y, nest.x FROM nested_test ARRAY JOIN nest AS n;
SELECT s, n.x, n.y, nest.x, nest.y FROM nested_test ARRAY JOIN nest AS n;
SELECT s, n.x, n.y, nest.x, nest.y, num FROM nested_test ARRAY JOIN nest AS n, arrayEnumerate(nest.x) AS num;
DROP TABLE nested_test;

View File

@ -1,6 +1,7 @@
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.big_array;
CREATE TABLE test.big_array (x Array(UInt8)) ENGINE=TinyLog;
DROP TABLE IF EXISTS big_array;
CREATE TABLE big_array (x Array(UInt8)) ENGINE=TinyLog;
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
INSERT INTO test.big_array SELECT groupArray(number % 255) AS x FROM (SELECT * FROM system.numbers LIMIT 1000000);
SELECT sum(y) AS s FROM remote('127.0.0.{2,3}', test, big_array) ARRAY JOIN x AS y;
INSERT INTO big_array SELECT groupArray(number % 255) AS x FROM (SELECT * FROM system.numbers LIMIT 1000000);
SELECT sum(y) AS s FROM remote('127.0.0.{2,3}', currentDatabase(), big_array) ARRAY JOIN x AS y;
SELECT sum(s) FROM (SELECT y AS s FROM remote('127.0.0.{2,3}', currentDatabase(), big_array) ARRAY JOIN x AS y);
DROP TABLE big_array;

View File

@ -1,2 +0,0 @@
SELECT sum(s) FROM (SELECT y AS s FROM remote('127.0.0.{2,3}', test, big_array) ARRAY JOIN x AS y);
DROP TABLE test.big_array;

View File

@ -1,14 +1,14 @@
d Date
k UInt64
i32 Int32
CREATE TABLE test.alter_00061 (`d` Date, `k` UInt64, `i32` Int32) ENGINE = MergeTree(d, k, 8192)
CREATE TABLE default.alter_00061 (`d` Date, `k` UInt64, `i32` Int32) ENGINE = MergeTree(d, k, 8192)
2015-01-01 10 42
d Date
k UInt64
i32 Int32
n.ui8 Array(UInt8)
n.s Array(String)
CREATE TABLE test.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = MergeTree(d, k, 8192)
CREATE TABLE default.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = MergeTree(d, k, 8192)
2015-01-01 8 40 [1,2,3] ['12','13','14']
2015-01-01 10 42 [] []
d Date
@ -17,7 +17,7 @@ i32 Int32
n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
CREATE TABLE test.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192)
CREATE TABLE default.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192)
2015-01-01 7 39 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03']
2015-01-01 8 40 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00']
2015-01-01 10 42 [] [] []
@ -28,7 +28,7 @@ n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
s String DEFAULT \'0\'
CREATE TABLE test.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), `s` String DEFAULT \'0\') ENGINE = MergeTree(d, k, 8192)
CREATE TABLE default.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), `s` String DEFAULT \'0\') ENGINE = MergeTree(d, k, 8192)
2015-01-01 6 38 [10,20,30] ['asd','qwe','qwe'] ['2000-01-01','2000-01-01','2000-01-03'] 100500
2015-01-01 7 39 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 0
2015-01-01 8 40 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00'] 0
@ -39,7 +39,7 @@ i32 Int32
n.ui8 Array(UInt8)
n.s Array(String)
s Int64
CREATE TABLE test.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `s` Int64) ENGINE = MergeTree(d, k, 8192)
CREATE TABLE default.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `s` Int64) ENGINE = MergeTree(d, k, 8192)
2015-01-01 6 38 [10,20,30] ['asd','qwe','qwe'] 100500
2015-01-01 7 39 [10,20,30] ['120','130','140'] 0
2015-01-01 8 40 [1,2,3] ['12','13','14'] 0
@ -51,7 +51,7 @@ n.ui8 Array(UInt8)
n.s Array(String)
s UInt32
n.d Array(Date)
CREATE TABLE test.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `s` UInt32, `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192)
CREATE TABLE default.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `s` UInt32, `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192)
2015-01-01 6 38 [10,20,30] ['asd','qwe','qwe'] 100500 ['0000-00-00','0000-00-00','0000-00-00']
2015-01-01 7 39 [10,20,30] ['120','130','140'] 0 ['0000-00-00','0000-00-00','0000-00-00']
2015-01-01 8 40 [1,2,3] ['12','13','14'] 0 ['0000-00-00','0000-00-00','0000-00-00']
@ -65,7 +65,7 @@ k UInt64
i32 Int32
n.s Array(String)
s UInt32
CREATE TABLE test.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `n.s` Array(String), `s` UInt32) ENGINE = MergeTree(d, k, 8192)
CREATE TABLE default.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `n.s` Array(String), `s` UInt32) ENGINE = MergeTree(d, k, 8192)
2015-01-01 6 38 ['asd','qwe','qwe'] 100500
2015-01-01 7 39 ['120','130','140'] 0
2015-01-01 8 40 ['12','13','14'] 0
@ -74,7 +74,7 @@ d Date
k UInt64
i32 Int32
s UInt32
CREATE TABLE test.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `s` UInt32) ENGINE = MergeTree(d, k, 8192)
CREATE TABLE default.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `s` UInt32) ENGINE = MergeTree(d, k, 8192)
2015-01-01 6 38 100500
2015-01-01 7 39 0
2015-01-01 8 40 0
@ -85,7 +85,7 @@ i32 Int32
s UInt32
n.s Array(String)
n.d Array(Date)
CREATE TABLE test.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `s` UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192)
CREATE TABLE default.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `s` UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192)
2015-01-01 6 38 100500 [] []
2015-01-01 7 39 0 [] []
2015-01-01 8 40 0 [] []
@ -94,7 +94,7 @@ d Date
k UInt64
i32 Int32
s UInt32
CREATE TABLE test.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `s` UInt32) ENGINE = MergeTree(d, k, 8192)
CREATE TABLE default.alter_00061 (`d` Date, `k` UInt64, `i32` Int32, `s` UInt32) ENGINE = MergeTree(d, k, 8192)
2015-01-01 6 38 100500
2015-01-01 7 39 0
2015-01-01 8 40 0

View File

@ -1,71 +1,71 @@
DROP TABLE IF EXISTS test.alter_00061;
CREATE TABLE test.alter_00061 (d Date, k UInt64, i32 Int32) ENGINE=MergeTree(d, k, 8192);
DROP TABLE IF EXISTS alter_00061;
CREATE TABLE alter_00061 (d Date, k UInt64, i32 Int32) ENGINE=MergeTree(d, k, 8192);
INSERT INTO test.alter_00061 VALUES ('2015-01-01', 10, 42);
INSERT INTO alter_00061 VALUES ('2015-01-01', 10, 42);
DESC TABLE test.alter_00061;
SHOW CREATE TABLE test.alter_00061;
SELECT * FROM test.alter_00061 ORDER BY k;
DESC TABLE alter_00061;
SHOW CREATE TABLE alter_00061;
SELECT * FROM alter_00061 ORDER BY k;
ALTER TABLE test.alter_00061 ADD COLUMN n Nested(ui8 UInt8, s String);
INSERT INTO test.alter_00061 VALUES ('2015-01-01', 8, 40, [1,2,3], ['12','13','14']);
ALTER TABLE alter_00061 ADD COLUMN n Nested(ui8 UInt8, s String);
INSERT INTO alter_00061 VALUES ('2015-01-01', 8, 40, [1,2,3], ['12','13','14']);
DESC TABLE test.alter_00061;
SHOW CREATE TABLE test.alter_00061;
SELECT * FROM test.alter_00061 ORDER BY k;
DESC TABLE alter_00061;
SHOW CREATE TABLE alter_00061;
SELECT * FROM alter_00061 ORDER BY k;
ALTER TABLE test.alter_00061 ADD COLUMN `n.d` Array(Date);
INSERT INTO test.alter_00061 VALUES ('2015-01-01', 7, 39, [10,20,30], ['120','130','140'],['2000-01-01','2000-01-01','2000-01-03']);
ALTER TABLE alter_00061 ADD COLUMN `n.d` Array(Date);
INSERT INTO alter_00061 VALUES ('2015-01-01', 7, 39, [10,20,30], ['120','130','140'],['2000-01-01','2000-01-01','2000-01-03']);
DESC TABLE test.alter_00061;
SHOW CREATE TABLE test.alter_00061;
SELECT * FROM test.alter_00061 ORDER BY k;
DESC TABLE alter_00061;
SHOW CREATE TABLE alter_00061;
SELECT * FROM alter_00061 ORDER BY k;
ALTER TABLE test.alter_00061 ADD COLUMN s String DEFAULT '0';
INSERT INTO test.alter_00061 VALUES ('2015-01-01', 6,38,[10,20,30],['asd','qwe','qwe'],['2000-01-01','2000-01-01','2000-01-03'],'100500');
ALTER TABLE alter_00061 ADD COLUMN s String DEFAULT '0';
INSERT INTO alter_00061 VALUES ('2015-01-01', 6,38,[10,20,30],['asd','qwe','qwe'],['2000-01-01','2000-01-01','2000-01-03'],'100500');
DESC TABLE test.alter_00061;
SHOW CREATE TABLE test.alter_00061;
SELECT * FROM test.alter_00061 ORDER BY k;
DESC TABLE alter_00061;
SHOW CREATE TABLE alter_00061;
SELECT * FROM alter_00061 ORDER BY k;
ALTER TABLE test.alter_00061 DROP COLUMN `n.d`, MODIFY COLUMN s Int64;
ALTER TABLE alter_00061 DROP COLUMN `n.d`, MODIFY COLUMN s Int64;
DESC TABLE test.alter_00061;
SHOW CREATE TABLE test.alter_00061;
SELECT * FROM test.alter_00061 ORDER BY k;
DESC TABLE alter_00061;
SHOW CREATE TABLE alter_00061;
SELECT * FROM alter_00061 ORDER BY k;
ALTER TABLE test.alter_00061 ADD COLUMN `n.d` Array(Date), MODIFY COLUMN s UInt32;
ALTER TABLE alter_00061 ADD COLUMN `n.d` Array(Date), MODIFY COLUMN s UInt32;
DESC TABLE test.alter_00061;
SHOW CREATE TABLE test.alter_00061;
SELECT * FROM test.alter_00061 ORDER BY k;
DESC TABLE alter_00061;
SHOW CREATE TABLE alter_00061;
SELECT * FROM alter_00061 ORDER BY k;
OPTIMIZE TABLE test.alter_00061;
OPTIMIZE TABLE alter_00061;
SELECT * FROM test.alter_00061 ORDER BY k;
SELECT * FROM alter_00061 ORDER BY k;
ALTER TABLE test.alter_00061 DROP COLUMN n.ui8, DROP COLUMN n.d;
ALTER TABLE alter_00061 DROP COLUMN n.ui8, DROP COLUMN n.d;
DESC TABLE test.alter_00061;
SHOW CREATE TABLE test.alter_00061;
SELECT * FROM test.alter_00061 ORDER BY k;
DESC TABLE alter_00061;
SHOW CREATE TABLE alter_00061;
SELECT * FROM alter_00061 ORDER BY k;
ALTER TABLE test.alter_00061 DROP COLUMN n.s;
ALTER TABLE alter_00061 DROP COLUMN n.s;
DESC TABLE test.alter_00061;
SHOW CREATE TABLE test.alter_00061;
SELECT * FROM test.alter_00061 ORDER BY k;
DESC TABLE alter_00061;
SHOW CREATE TABLE alter_00061;
SELECT * FROM alter_00061 ORDER BY k;
ALTER TABLE test.alter_00061 ADD COLUMN n.s Array(String), ADD COLUMN n.d Array(Date);
ALTER TABLE alter_00061 ADD COLUMN n.s Array(String), ADD COLUMN n.d Array(Date);
DESC TABLE test.alter_00061;
SHOW CREATE TABLE test.alter_00061;
SELECT * FROM test.alter_00061 ORDER BY k;
DESC TABLE alter_00061;
SHOW CREATE TABLE alter_00061;
SELECT * FROM alter_00061 ORDER BY k;
ALTER TABLE test.alter_00061 DROP COLUMN n;
ALTER TABLE alter_00061 DROP COLUMN n;
DESC TABLE test.alter_00061;
SHOW CREATE TABLE test.alter_00061;
SELECT * FROM test.alter_00061 ORDER BY k;
DESC TABLE alter_00061;
SHOW CREATE TABLE alter_00061;
SELECT * FROM alter_00061 ORDER BY k;
DROP TABLE test.alter_00061;
DROP TABLE alter_00061;

View File

@ -14,3 +14,6 @@ CREATE TABLE check_query_log (N UInt32,S String) Engine = Log;
INSERT INTO check_query_log VALUES (1, 'A'), (2, 'B'), (3, 'C')
CHECK TABLE check_query_log;
DROP TABLE check_query_log;
DROP TABLE check_query_tiny_log;

View File

@ -125,3 +125,4 @@ CREATE TABLE addresses(addr String) ENGINE = Memory;
INSERT INTO addresses(addr) VALUES ('00000000000000000000FFFFC1FC110A'), ('00000000000000000000FFFF4D583737'), ('00000000000000000000FFFF7F000001');
SELECT cutIPv6(toFixedString(unhex(addr), 16), 0, 3) FROM addresses ORDER BY addr ASC;
DROP TABLE addresses;

View File

@ -39,3 +39,5 @@ INSERT INTO summing (k, s) VALUES (0, 1), (666, 1), (666, 0);
OPTIMIZE TABLE summing PARTITION 197001;
SELECT k, s FROM summing ORDER BY k;
DROP TABLE summing;

View File

@ -1,13 +1,13 @@
DROP TABLE IF EXISTS test.report1;
DROP TABLE IF EXISTS test.report2;
DROP TABLE IF EXISTS report1;
DROP TABLE IF EXISTS report2;
CREATE TABLE test.report1(id UInt32, event_date Date, priority UInt32, description String) ENGINE = MergeTree(event_date, intHash32(id), (id, event_date, intHash32(id)), 8192);
CREATE TABLE test.report2(id UInt32, event_date Date, priority UInt32, description String) ENGINE = MergeTree(event_date, intHash32(id), (id, event_date, intHash32(id)), 8192);
CREATE TABLE report1(id UInt32, event_date Date, priority UInt32, description String) ENGINE = MergeTree(event_date, intHash32(id), (id, event_date, intHash32(id)), 8192);
CREATE TABLE report2(id UInt32, event_date Date, priority UInt32, description String) ENGINE = MergeTree(event_date, intHash32(id), (id, event_date, intHash32(id)), 8192);
INSERT INTO test.report1(id,event_date,priority,description) VALUES (1, '2015-01-01', 1, 'foo')(2, '2015-02-01', 2, 'bar')(3, '2015-03-01', 3, 'foo')(4, '2015-04-01', 4, 'bar')(5, '2015-05-01', 5, 'foo');
INSERT INTO test.report2(id,event_date,priority,description) VALUES (1, '2016-01-01', 6, 'bar')(2, '2016-02-01', 7, 'foo')(3, '2016-03-01', 8, 'bar')(4, '2016-04-01', 9, 'foo')(5, '2016-05-01', 10, 'bar');
INSERT INTO report1(id,event_date,priority,description) VALUES (1, '2015-01-01', 1, 'foo')(2, '2015-02-01', 2, 'bar')(3, '2015-03-01', 3, 'foo')(4, '2015-04-01', 4, 'bar')(5, '2015-05-01', 5, 'foo');
INSERT INTO report2(id,event_date,priority,description) VALUES (1, '2016-01-01', 6, 'bar')(2, '2016-02-01', 7, 'foo')(3, '2016-03-01', 8, 'bar')(4, '2016-04-01', 9, 'foo')(5, '2016-05-01', 10, 'bar');
SELECT * FROM (SELECT id, event_date, priority, description FROM remote('127.0.0.{2,3}', test, report1) UNION ALL SELECT id, event_date, priority, description FROM remote('127.0.0.{2,3}', test, report2)) ORDER BY id, event_date ASC;
SELECT * FROM (SELECT id, event_date, priority, description FROM remote('127.0.0.{2,3}', currentDatabase(), report1) UNION ALL SELECT id, event_date, priority, description FROM remote('127.0.0.{2,3}', currentDatabase(), report2)) ORDER BY id, event_date ASC;
DROP TABLE test.report1;
DROP TABLE test.report2;
DROP TABLE report1;
DROP TABLE report2;

View File

@ -2,19 +2,19 @@ SET max_rows_to_group_by = 100000;
SET max_block_size = 100001;
SET group_by_overflow_mode = 'any';
DROP TABLE IF EXISTS test.numbers500k;
CREATE VIEW test.numbers500k AS SELECT number FROM system.numbers LIMIT 500000;
DROP TABLE IF EXISTS numbers500k;
CREATE VIEW numbers500k AS SELECT number FROM system.numbers LIMIT 500000;
SET totals_mode = 'after_having_auto';
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', test, numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', currentDatabase(), numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
SET totals_mode = 'after_having_inclusive';
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', test, numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', currentDatabase(), numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
SET totals_mode = 'after_having_exclusive';
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', test, numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', currentDatabase(), numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
SET totals_mode = 'before_having';
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', test, numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', currentDatabase(), numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
DROP TABLE test.numbers500k;
DROP TABLE numbers500k;

View File

@ -1,9 +1,9 @@
SET max_memory_usage = 100000000;
SET max_bytes_before_external_sort = 20000000;
DROP TABLE IF EXISTS test.numbers10m;
CREATE VIEW test.numbers10m AS SELECT number FROM system.numbers LIMIT 10000000;
DROP TABLE IF EXISTS numbers10m;
CREATE VIEW numbers10m AS SELECT number FROM system.numbers LIMIT 10000000;
SELECT number FROM remote('127.0.0.{2,3}', test, numbers10m) ORDER BY number * 1234567890123456789 LIMIT 19999980, 20;
SELECT number FROM remote('127.0.0.{2,3}', currentDatabase(), numbers10m) ORDER BY number * 1234567890123456789 LIMIT 19999980, 20;
DROP TABLE test.numbers10m;
DROP TABLE numbers10m;

View File

@ -4,34 +4,34 @@ SELECT '';
SELECT length(toString(groupArrayState(toDate(number)))) FROM (SELECT * FROM system.numbers LIMIT 10);
SELECT length(toString(groupArrayState(toDateTime(number)))) FROM (SELECT * FROM system.numbers LIMIT 10);
DROP TABLE IF EXISTS test.numbers_mt;
CREATE TABLE test.numbers_mt (number UInt64) ENGINE = Log;
INSERT INTO test.numbers_mt SELECT * FROM system.numbers LIMIT 1, 1000000;
DROP TABLE IF EXISTS numbers_mt;
CREATE TABLE numbers_mt (number UInt64) ENGINE = Log;
INSERT INTO numbers_mt SELECT * FROM system.numbers LIMIT 1, 1000000;
SELECT count(), sum(ns), max(ns) FROM (SELECT intDiv(number, 100) AS k, groupArray(number) AS ns FROM test.numbers_mt GROUP BY k) ARRAY JOIN ns;
SELECT count(), sum(toUInt64(ns)), max(toUInt64(ns)) FROM (SELECT intDiv(number, 100) AS k, groupArray(toString(number)) AS ns FROM test.numbers_mt GROUP BY k) ARRAY JOIN ns;
SELECT count(), sum(toUInt64(ns[1])), max(toUInt64(ns[1])), sum(toUInt64(ns[2]))/10 FROM (SELECT intDiv(number, 100) AS k, groupArray([toString(number), toString(number*10)]) AS ns FROM test.numbers_mt GROUP BY k) ARRAY JOIN ns;
SELECT count(), sum(ns[1]), max(ns[1]), sum(ns[2])/10 FROM (SELECT intDiv(number, 100) AS k, groupArray([number, number*10]) AS ns FROM test.numbers_mt GROUP BY k) ARRAY JOIN ns;
SELECT count(), sum(ns), max(ns) FROM (SELECT intDiv(number, 100) AS k, groupArray(number) AS ns FROM numbers_mt GROUP BY k) ARRAY JOIN ns;
SELECT count(), sum(toUInt64(ns)), max(toUInt64(ns)) FROM (SELECT intDiv(number, 100) AS k, groupArray(toString(number)) AS ns FROM numbers_mt GROUP BY k) ARRAY JOIN ns;
SELECT count(), sum(toUInt64(ns[1])), max(toUInt64(ns[1])), sum(toUInt64(ns[2]))/10 FROM (SELECT intDiv(number, 100) AS k, groupArray([toString(number), toString(number*10)]) AS ns FROM numbers_mt GROUP BY k) ARRAY JOIN ns;
SELECT count(), sum(ns[1]), max(ns[1]), sum(ns[2])/10 FROM (SELECT intDiv(number, 100) AS k, groupArray([number, number*10]) AS ns FROM numbers_mt GROUP BY k) ARRAY JOIN ns;
SELECT count(), sum(ns), max(ns) FROM (SELECT intDiv(number, 100) AS k, groupArray(number) AS ns FROM remote('127.0.0.{2,3}', 'test', 'numbers_mt') GROUP BY k) ARRAY JOIN ns;
SELECT count(), sum(toUInt64(ns)), max(toUInt64(ns)) FROM (SELECT intDiv(number, 100) AS k, groupArray(toString(number)) AS ns FROM remote('127.0.0.{2,3}', 'test', 'numbers_mt') GROUP BY k) ARRAY JOIN ns;
SELECT count(), sum(toUInt64(ns[1])), max(toUInt64(ns[1])), sum(toUInt64(ns[2]))/10 FROM (SELECT intDiv(number, 100) AS k, groupArray([toString(number), toString(number*10)]) AS ns FROM remote('127.0.0.{2,3}', 'test', 'numbers_mt') GROUP BY k) ARRAY JOIN ns;
SELECT count(), sum(ns), max(ns) FROM (SELECT intDiv(number, 100) AS k, groupArray(number) AS ns FROM remote('127.0.0.{2,3}', currentDatabase(), 'numbers_mt') GROUP BY k) ARRAY JOIN ns;
SELECT count(), sum(toUInt64(ns)), max(toUInt64(ns)) FROM (SELECT intDiv(number, 100) AS k, groupArray(toString(number)) AS ns FROM remote('127.0.0.{2,3}', currentDatabase(), 'numbers_mt') GROUP BY k) ARRAY JOIN ns;
SELECT count(), sum(toUInt64(ns[1])), max(toUInt64(ns[1])), sum(toUInt64(ns[2]))/10 FROM (SELECT intDiv(number, 100) AS k, groupArray([toString(number), toString(number*10)]) AS ns FROM remote('127.0.0.{2,3}', currentDatabase(), 'numbers_mt') GROUP BY k) ARRAY JOIN ns;
DROP TABLE test.numbers_mt;
CREATE TABLE test.numbers_mt (number UInt64) ENGINE = Log;
INSERT INTO test.numbers_mt SELECT * FROM system.numbers LIMIT 1, 1048575;
DROP TABLE numbers_mt;
CREATE TABLE numbers_mt (number UInt64) ENGINE = Log;
INSERT INTO numbers_mt SELECT * FROM system.numbers LIMIT 1, 1048575;
SELECT '';
SELECT roundToExp2(number) AS k, length(groupArray(1)(number AS i)), length(groupArray(1024)(i)), length(groupArray(65536)(i)) AS s FROM test.numbers_mt GROUP BY k ORDER BY k LIMIT 9, 11;
SELECT roundToExp2(number) AS k, length(groupArray(1)(hex(number) AS i)), length(groupArray(1024)(i)), length(groupArray(65536)(i)) AS s FROM test.numbers_mt GROUP BY k ORDER BY k LIMIT 9, 11;
SELECT roundToExp2(number) AS k, length(groupArray(1)([hex(number)] AS i)), length(groupArray(1024)(i)), length(groupArray(65536)(i)) AS s FROM test.numbers_mt GROUP BY k ORDER BY k LIMIT 9, 11;
SELECT roundToExp2(number) AS k, length(groupArray(1)(number AS i)), length(groupArray(1024)(i)), length(groupArray(65536)(i)) AS s FROM numbers_mt GROUP BY k ORDER BY k LIMIT 9, 11;
SELECT roundToExp2(number) AS k, length(groupArray(1)(hex(number) AS i)), length(groupArray(1024)(i)), length(groupArray(65536)(i)) AS s FROM numbers_mt GROUP BY k ORDER BY k LIMIT 9, 11;
SELECT roundToExp2(number) AS k, length(groupArray(1)([hex(number)] AS i)), length(groupArray(1024)(i)), length(groupArray(65536)(i)) AS s FROM numbers_mt GROUP BY k ORDER BY k LIMIT 9, 11;
SELECT '';
SELECT roundToExp2(number) AS k, length(groupArray(1)(number AS i)), length(groupArray(1500)(i)), length(groupArray(70000)(i)) AS s FROM remote('127.0.0.{2,3}', 'test', 'numbers_mt') GROUP BY k ORDER BY k LIMIT 9, 11;
SELECT roundToExp2(number) AS k, length(groupArray(1)(hex(number) AS i)), length(groupArray(1500)(i)), length(groupArray(70000)(i)) AS s FROM remote('127.0.0.{2,3}', 'test', 'numbers_mt') GROUP BY k ORDER BY k LIMIT 9, 11;
SELECT roundToExp2(number) AS k, length(groupArray(1)([hex(number)] AS i)), length(groupArray(1500)(i)), length(groupArray(70000)(i)) AS s FROM remote('127.0.0.{2,3}', 'test', 'numbers_mt') GROUP BY k ORDER BY k LIMIT 9, 11;
SELECT roundToExp2(number) AS k, length(groupArray(1)(number AS i)), length(groupArray(1500)(i)), length(groupArray(70000)(i)) AS s FROM remote('127.0.0.{2,3}', currentDatabase(), 'numbers_mt') GROUP BY k ORDER BY k LIMIT 9, 11;
SELECT roundToExp2(number) AS k, length(groupArray(1)(hex(number) AS i)), length(groupArray(1500)(i)), length(groupArray(70000)(i)) AS s FROM remote('127.0.0.{2,3}', currentDatabase(), 'numbers_mt') GROUP BY k ORDER BY k LIMIT 9, 11;
SELECT roundToExp2(number) AS k, length(groupArray(1)([hex(number)] AS i)), length(groupArray(1500)(i)), length(groupArray(70000)(i)) AS s FROM remote('127.0.0.{2,3}', currentDatabase(), 'numbers_mt') GROUP BY k ORDER BY k LIMIT 9, 11;
DROP TABLE test.numbers_mt;
DROP TABLE numbers_mt;
-- Check binary compatibility:
-- clickhouse-client -h old -q "SELECT arrayReduce('groupArrayState', [['1'], ['22'], ['333']]) FORMAT RowBinary" | clickhouse-local -s --input-format RowBinary --structure "d AggregateFunction(groupArray2, Array(String))" -q "SELECT groupArray2Merge(d) FROM table"

View File

@ -1,9 +1,7 @@
DROP TABLE IF EXISTS test.set;
DROP TABLE IF EXISTS test.set2;
DROP TABLE IF EXISTS set;
DROP TABLE IF EXISTS set2;
CREATE TABLE test.set (x String) ENGINE = Set;
USE test;
CREATE TABLE set (x String) ENGINE = Set;
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set;
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s NOT IN set;
@ -14,10 +12,10 @@ SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s
RENAME TABLE set TO set2;
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2;
INSERT INTO test.set2 VALUES ('Hello'), ('World');
INSERT INTO set2 VALUES ('Hello'), ('World');
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2;
INSERT INTO test.set2 VALUES ('abc'), ('World');
INSERT INTO set2 VALUES ('abc'), ('World');
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2;
DETACH TABLE set2;
@ -28,6 +26,4 @@ SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s
RENAME TABLE set2 TO set;
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set;
USE default;
DROP TABLE test.set;
DROP TABLE set;

View File

@ -1,15 +1,11 @@
DROP TABLE IF EXISTS test.join;
DROP TABLE IF EXISTS join;
CREATE TABLE test.join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k);
CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k);
USE test;
INSERT INTO test.join VALUES (1, 'abc'), (2, 'def');
INSERT INTO join VALUES (1, 'abc'), (2, 'def');
SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) ANY LEFT JOIN join USING k;
INSERT INTO test.join VALUES (6, 'ghi');
INSERT INTO join VALUES (6, 'ghi');
SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) ANY LEFT JOIN join USING k;
USE default;
DROP TABLE test.join;
DROP TABLE join;

View File

@ -1,11 +1,10 @@
SET max_parallel_replicas = 2;
DROP TABLE IF EXISTS test.report;
DROP TABLE IF EXISTS report;
CREATE TABLE test.report(id UInt32, event_date Date, priority UInt32, description String) ENGINE = MergeTree(event_date, intHash32(id), (id, event_date, intHash32(id)), 8192);
CREATE TABLE report(id UInt32, event_date Date, priority UInt32, description String) ENGINE = MergeTree(event_date, intHash32(id), (id, event_date, intHash32(id)), 8192);
INSERT INTO test.report(id,event_date,priority,description) VALUES (1, '2015-01-01', 1, 'foo')(2, '2015-02-01', 2, 'bar')(3, '2015-03-01', 3, 'foo')(4, '2015-04-01', 4, 'bar')(5, '2015-05-01', 5, 'foo');
SELECT * FROM (SELECT id, event_date, priority, description FROM remote('127.0.0.{2|3}', test, report)) ORDER BY id ASC;
DROP TABLE test.report;
INSERT INTO report(id,event_date,priority,description) VALUES (1, '2015-01-01', 1, 'foo')(2, '2015-02-01', 2, 'bar')(3, '2015-03-01', 3, 'foo')(4, '2015-04-01', 4, 'bar')(5, '2015-05-01', 5, 'foo');
SELECT * FROM (SELECT id, event_date, priority, description FROM remote('127.0.0.{2|3}', currentDatabase(), report)) ORDER BY id ASC;
DROP TABLE report;

View File

@ -1,62 +1,62 @@
DROP TABLE IF EXISTS test.buffer_00126;
DROP TABLE IF EXISTS test.null_sink_00126;
DROP TABLE IF EXISTS buffer_00126;
DROP TABLE IF EXISTS null_sink_00126;
CREATE TABLE test.null_sink_00126 (a UInt8, b String, c Array(UInt32)) ENGINE = Null;
CREATE TABLE test.buffer_00126 (a UInt8, b String, c Array(UInt32)) ENGINE = Buffer(test, null_sink_00126, 1, 1000, 1000, 1000, 1000, 1000000, 1000000);
CREATE TABLE null_sink_00126 (a UInt8, b String, c Array(UInt32)) ENGINE = Null;
CREATE TABLE buffer_00126 (a UInt8, b String, c Array(UInt32)) ENGINE = Buffer(currentDatabase(), null_sink_00126, 1, 1000, 1000, 1000, 1000, 1000000, 1000000);
INSERT INTO test.buffer_00126 VALUES (1, '2', [3]);
INSERT INTO buffer_00126 VALUES (1, '2', [3]);
SELECT a, b, c FROM test.buffer_00126 ORDER BY a, b, c;
SELECT b, c, a FROM test.buffer_00126 ORDER BY a, b, c;
SELECT c, a, b FROM test.buffer_00126 ORDER BY a, b, c;
SELECT a, c, b FROM test.buffer_00126 ORDER BY a, b, c;
SELECT b, a, c FROM test.buffer_00126 ORDER BY a, b, c;
SELECT c, b, a FROM test.buffer_00126 ORDER BY a, b, c;
SELECT a, b FROM test.buffer_00126 ORDER BY a, b, c;
SELECT b, c FROM test.buffer_00126 ORDER BY a, b, c;
SELECT c, a FROM test.buffer_00126 ORDER BY a, b, c;
SELECT a, c FROM test.buffer_00126 ORDER BY a, b, c;
SELECT b, a FROM test.buffer_00126 ORDER BY a, b, c;
SELECT c, b FROM test.buffer_00126 ORDER BY a, b, c;
SELECT a FROM test.buffer_00126 ORDER BY a, b, c;
SELECT b FROM test.buffer_00126 ORDER BY a, b, c;
SELECT c FROM test.buffer_00126 ORDER BY a, b, c;
SELECT a, b, c FROM buffer_00126 ORDER BY a, b, c;
SELECT b, c, a FROM buffer_00126 ORDER BY a, b, c;
SELECT c, a, b FROM buffer_00126 ORDER BY a, b, c;
SELECT a, c, b FROM buffer_00126 ORDER BY a, b, c;
SELECT b, a, c FROM buffer_00126 ORDER BY a, b, c;
SELECT c, b, a FROM buffer_00126 ORDER BY a, b, c;
SELECT a, b FROM buffer_00126 ORDER BY a, b, c;
SELECT b, c FROM buffer_00126 ORDER BY a, b, c;
SELECT c, a FROM buffer_00126 ORDER BY a, b, c;
SELECT a, c FROM buffer_00126 ORDER BY a, b, c;
SELECT b, a FROM buffer_00126 ORDER BY a, b, c;
SELECT c, b FROM buffer_00126 ORDER BY a, b, c;
SELECT a FROM buffer_00126 ORDER BY a, b, c;
SELECT b FROM buffer_00126 ORDER BY a, b, c;
SELECT c FROM buffer_00126 ORDER BY a, b, c;
INSERT INTO test.buffer_00126 (c, b, a) VALUES ([7], '8', 9);
INSERT INTO buffer_00126 (c, b, a) VALUES ([7], '8', 9);
SELECT a, b, c FROM test.buffer_00126 ORDER BY a, b, c;
SELECT b, c, a FROM test.buffer_00126 ORDER BY a, b, c;
SELECT c, a, b FROM test.buffer_00126 ORDER BY a, b, c;
SELECT a, c, b FROM test.buffer_00126 ORDER BY a, b, c;
SELECT b, a, c FROM test.buffer_00126 ORDER BY a, b, c;
SELECT c, b, a FROM test.buffer_00126 ORDER BY a, b, c;
SELECT a, b FROM test.buffer_00126 ORDER BY a, b, c;
SELECT b, c FROM test.buffer_00126 ORDER BY a, b, c;
SELECT c, a FROM test.buffer_00126 ORDER BY a, b, c;
SELECT a, c FROM test.buffer_00126 ORDER BY a, b, c;
SELECT b, a FROM test.buffer_00126 ORDER BY a, b, c;
SELECT c, b FROM test.buffer_00126 ORDER BY a, b, c;
SELECT a FROM test.buffer_00126 ORDER BY a, b, c;
SELECT b FROM test.buffer_00126 ORDER BY a, b, c;
SELECT c FROM test.buffer_00126 ORDER BY a, b, c;
SELECT a, b, c FROM buffer_00126 ORDER BY a, b, c;
SELECT b, c, a FROM buffer_00126 ORDER BY a, b, c;
SELECT c, a, b FROM buffer_00126 ORDER BY a, b, c;
SELECT a, c, b FROM buffer_00126 ORDER BY a, b, c;
SELECT b, a, c FROM buffer_00126 ORDER BY a, b, c;
SELECT c, b, a FROM buffer_00126 ORDER BY a, b, c;
SELECT a, b FROM buffer_00126 ORDER BY a, b, c;
SELECT b, c FROM buffer_00126 ORDER BY a, b, c;
SELECT c, a FROM buffer_00126 ORDER BY a, b, c;
SELECT a, c FROM buffer_00126 ORDER BY a, b, c;
SELECT b, a FROM buffer_00126 ORDER BY a, b, c;
SELECT c, b FROM buffer_00126 ORDER BY a, b, c;
SELECT a FROM buffer_00126 ORDER BY a, b, c;
SELECT b FROM buffer_00126 ORDER BY a, b, c;
SELECT c FROM buffer_00126 ORDER BY a, b, c;
INSERT INTO test.buffer_00126 (a, c) VALUES (11, [33]);
INSERT INTO buffer_00126 (a, c) VALUES (11, [33]);
SELECT a, b, c FROM test.buffer_00126 ORDER BY a, b, c;
SELECT b, c, a FROM test.buffer_00126 ORDER BY a, b, c;
SELECT c, a, b FROM test.buffer_00126 ORDER BY a, b, c;
SELECT a, c, b FROM test.buffer_00126 ORDER BY a, b, c;
SELECT b, a, c FROM test.buffer_00126 ORDER BY a, b, c;
SELECT c, b, a FROM test.buffer_00126 ORDER BY a, b, c;
SELECT a, b FROM test.buffer_00126 ORDER BY a, b, c;
SELECT b, c FROM test.buffer_00126 ORDER BY a, b, c;
SELECT c, a FROM test.buffer_00126 ORDER BY a, b, c;
SELECT a, c FROM test.buffer_00126 ORDER BY a, b, c;
SELECT b, a FROM test.buffer_00126 ORDER BY a, b, c;
SELECT c, b FROM test.buffer_00126 ORDER BY a, b, c;
SELECT a FROM test.buffer_00126 ORDER BY a, b, c;
SELECT b FROM test.buffer_00126 ORDER BY a, b, c;
SELECT c FROM test.buffer_00126 ORDER BY a, b, c;
SELECT a, b, c FROM buffer_00126 ORDER BY a, b, c;
SELECT b, c, a FROM buffer_00126 ORDER BY a, b, c;
SELECT c, a, b FROM buffer_00126 ORDER BY a, b, c;
SELECT a, c, b FROM buffer_00126 ORDER BY a, b, c;
SELECT b, a, c FROM buffer_00126 ORDER BY a, b, c;
SELECT c, b, a FROM buffer_00126 ORDER BY a, b, c;
SELECT a, b FROM buffer_00126 ORDER BY a, b, c;
SELECT b, c FROM buffer_00126 ORDER BY a, b, c;
SELECT c, a FROM buffer_00126 ORDER BY a, b, c;
SELECT a, c FROM buffer_00126 ORDER BY a, b, c;
SELECT b, a FROM buffer_00126 ORDER BY a, b, c;
SELECT c, b FROM buffer_00126 ORDER BY a, b, c;
SELECT a FROM buffer_00126 ORDER BY a, b, c;
SELECT b FROM buffer_00126 ORDER BY a, b, c;
SELECT c FROM buffer_00126 ORDER BY a, b, c;
DROP TABLE test.buffer_00126;
DROP TABLE test.null_sink_00126;
DROP TABLE buffer_00126;
DROP TABLE null_sink_00126;

View File

@ -1,23 +1,23 @@
DROP TABLE IF EXISTS test.mt_00160;
DROP TABLE IF EXISTS test.merge_00160;
DROP TABLE IF EXISTS mt_00160;
DROP TABLE IF EXISTS merge_00160;
CREATE TABLE test.mt_00160 (d Date DEFAULT toDate('2015-05-01'), x UInt64) ENGINE = MergeTree(d, x, 1);
CREATE TABLE test.merge_00160 (d Date, x UInt64) ENGINE = Merge(test, '^mt_00160$');
CREATE TABLE mt_00160 (d Date DEFAULT toDate('2015-05-01'), x UInt64) ENGINE = MergeTree(d, x, 1);
CREATE TABLE merge_00160 (d Date, x UInt64) ENGINE = Merge(currentDatabase(), '^mt_00160$');
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
SET max_block_size = 1000000;
INSERT INTO test.mt_00160 (x) SELECT number AS x FROM system.numbers LIMIT 100000;
INSERT INTO mt_00160 (x) SELECT number AS x FROM system.numbers LIMIT 100000;
SELECT *, b FROM test.mt_00160 WHERE x IN (12345, 67890) AND NOT ignore(blockSize() < 10 AS b) ORDER BY x;
SELECT *, b FROM test.merge_00160 WHERE x IN (12345, 67890) AND NOT ignore(blockSize() < 10 AS b) ORDER BY x;
SELECT *, b FROM mt_00160 WHERE x IN (12345, 67890) AND NOT ignore(blockSize() < 10 AS b) ORDER BY x;
SELECT *, b FROM merge_00160 WHERE x IN (12345, 67890) AND NOT ignore(blockSize() < 10 AS b) ORDER BY x;
DROP TABLE test.merge_00160;
DROP TABLE test.mt_00160;
DROP TABLE merge_00160;
DROP TABLE mt_00160;
CREATE TABLE test.mt_00160 (d Date DEFAULT toDate('2015-05-01'), x UInt64, y UInt64, z UInt64) ENGINE = MergeTree(d, (x, z), 1);
CREATE TABLE mt_00160 (d Date DEFAULT toDate('2015-05-01'), x UInt64, y UInt64, z UInt64) ENGINE = MergeTree(d, (x, z), 1);
INSERT INTO test.mt_00160 (x, y, z) SELECT number AS x, number + 10 AS y, number / 2 AS z FROM system.numbers LIMIT 100000;
INSERT INTO mt_00160 (x, y, z) SELECT number AS x, number + 10 AS y, number / 2 AS z FROM system.numbers LIMIT 100000;
SELECT *, b FROM test.mt_00160 WHERE (z, y, x) IN ((617, 1244, 1234), (2839, 5688, 5678), (1,1,1)) AND NOT ignore(blockSize() < 10 AS b) ORDER BY (x, y, z);
SELECT *, b FROM mt_00160 WHERE (z, y, x) IN ((617, 1244, 1234), (2839, 5688, 5678), (1,1,1)) AND NOT ignore(blockSize() < 10 AS b) ORDER BY (x, y, z);
DROP TABLE test.mt_00160;
DROP TABLE mt_00160;

View File

@ -1,12 +1,12 @@
DROP TABLE IF EXISTS test.mt_00168;
DROP TABLE IF EXISTS test.mt_00168_buffer;
CREATE TABLE test.mt_00168 (EventDate Date, UTCEventTime DateTime, MoscowEventDate Date DEFAULT toDate(UTCEventTime)) ENGINE = MergeTree(EventDate, UTCEventTime, 8192);
CREATE TABLE test.mt_00168_buffer AS test.mt_00168 ENGINE = Buffer(test, mt_00168, 16, 10, 100, 10000, 1000000, 10000000, 100000000);
DESC TABLE test.mt_00168;
DESC TABLE test.mt_00168_buffer;
INSERT INTO test.mt_00168 (EventDate, UTCEventTime) VALUES ('2015-06-09', '2015-06-09 01:02:03');
SELECT * FROM test.mt_00168_buffer;
INSERT INTO test.mt_00168_buffer (EventDate, UTCEventTime) VALUES ('2015-06-09', '2015-06-09 01:02:03');
SELECT * FROM test.mt_00168_buffer;
DROP TABLE test.mt_00168_buffer;
DROP TABLE test.mt_00168;
DROP TABLE IF EXISTS mt_00168;
DROP TABLE IF EXISTS mt_00168_buffer;
CREATE TABLE mt_00168 (EventDate Date, UTCEventTime DateTime, MoscowEventDate Date DEFAULT toDate(UTCEventTime)) ENGINE = MergeTree(EventDate, UTCEventTime, 8192);
CREATE TABLE mt_00168_buffer AS mt_00168 ENGINE = Buffer(currentDatabase(), mt_00168, 16, 10, 100, 10000, 1000000, 10000000, 100000000);
DESC TABLE mt_00168;
DESC TABLE mt_00168_buffer;
INSERT INTO mt_00168 (EventDate, UTCEventTime) VALUES ('2015-06-09', '2015-06-09 01:02:03');
SELECT * FROM mt_00168_buffer;
INSERT INTO mt_00168_buffer (EventDate, UTCEventTime) VALUES ('2015-06-09', '2015-06-09 01:02:03');
SELECT * FROM mt_00168_buffer;
DROP TABLE mt_00168_buffer;
DROP TABLE mt_00168;

View File

@ -1,5 +1,5 @@
DROP TABLE IF EXISTS test.numbers_memory;
CREATE TABLE test.numbers_memory AS system.numbers ENGINE = Memory;
INSERT INTO test.numbers_memory SELECT number FROM system.numbers LIMIT 100;
SELECT DISTINCT number FROM remote('127.0.0.{2,3}', test.numbers_memory) ORDER BY number LIMIT 10;
DROP TABLE test.numbers_memory;
DROP TABLE IF EXISTS numbers_memory;
CREATE TABLE numbers_memory AS system.numbers ENGINE = Memory;
INSERT INTO numbers_memory SELECT number FROM system.numbers LIMIT 100;
SELECT DISTINCT number FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_memory) ORDER BY number LIMIT 10;
DROP TABLE numbers_memory;

View File

@ -22,3 +22,6 @@ INSERT INTO t2_00203 VALUES ('ccc', 4, 3, 2), ('ddd', 7, 6, 5);
SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY FULL JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3;
SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY RIGHT JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3;
DROP TABLE t1_00203;
DROP TABLE t2_00203;

View File

@ -1,131 +1,131 @@
SET max_block_size = 1000;
DROP TABLE IF EXISTS test.numbers_10_00223;
CREATE TABLE test.numbers_10_00223 ENGINE = Log AS SELECT * FROM system.numbers LIMIT 10000;
DROP TABLE IF EXISTS numbers_10_00223;
CREATE TABLE numbers_10_00223 ENGINE = Log AS SELECT * FROM system.numbers LIMIT 10000;
SET distributed_aggregation_memory_efficient = 0;
SET group_by_two_level_threshold = 1000;
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SET distributed_aggregation_memory_efficient = 0;
SET group_by_two_level_threshold = 7;
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SET distributed_aggregation_memory_efficient = 1;
SET group_by_two_level_threshold = 1000;
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SET distributed_aggregation_memory_efficient = 1;
SET group_by_two_level_threshold = 7;
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SET distributed_aggregation_memory_efficient = 1;
SET group_by_two_level_threshold = 1;
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10) FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY number);
SET distributed_aggregation_memory_efficient = 1;
SET group_by_two_level_threshold = 1000;
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SET distributed_aggregation_memory_efficient = 1;
SET group_by_two_level_threshold = 1;
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SELECT sum(c = 1) IN (0, 10), sum(c = 2) IN (0, 5), sum(c) = 10 FROM (SELECT number, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) AND number >= (randConstant() % 2 ? 0 : 5) GROUP BY number);
SET distributed_aggregation_memory_efficient = 1;
SET group_by_two_level_threshold = 7;
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 1) IN (0, 5), sum(c = 2) IN (5, 10), sum(c) IN (10, 15, 20) FROM (SELECT number AS k1, number + 1 AS k2, count() AS c FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 5 : 10) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', test.numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
SELECT sum(c = 20) IN (5, 10), sum(c = 10) IN (0, 5), sum(u != 10) = 0 FROM (SELECT intDiv(number, 10) AS k1, k1 + 1 AS k2, count() AS c, uniq(number) AS u FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) WHERE number < (randConstant() % 2 ? 50 : 100) GROUP BY k1, k2 HAVING count() > 0 ORDER BY k1, k2);
DROP TABLE test.numbers_10_00223;
DROP TABLE numbers_10_00223;
SELECT count() FROM remote('127.0.0.{2,3}', system.one);

View File

@ -1,7 +1,7 @@
DROP TABLE IF EXISTS test.numbers_100k_log;
CREATE TABLE test.numbers_100k_log ENGINE = Log AS SELECT * FROM system.numbers LIMIT 100000;
DROP TABLE IF EXISTS numbers_100k_log;
CREATE TABLE numbers_100k_log ENGINE = Log AS SELECT * FROM system.numbers LIMIT 100000;
SELECT count() = 200000 FROM remote('127.0.0.{2,3}', test.numbers_100k_log) GROUP BY number WITH TOTALS ORDER BY number LIMIT 10;
SELECT count() = 200000 FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_100k_log) GROUP BY number WITH TOTALS ORDER BY number LIMIT 10;
SET distributed_aggregation_memory_efficient = 1,
group_by_two_level_threshold = 1000,
@ -9,6 +9,6 @@ SET distributed_aggregation_memory_efficient = 1,
max_rows_to_group_by = 1000,
totals_mode = 'after_having_auto';
SELECT count() = 200000 FROM remote('127.0.0.{2,3}', test.numbers_100k_log) GROUP BY number WITH TOTALS ORDER BY number LIMIT 10;
SELECT count() = 200000 FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_100k_log) GROUP BY number WITH TOTALS ORDER BY number LIMIT 10;
DROP TABLE test.numbers_100k_log;
DROP TABLE numbers_100k_log;

View File

@ -1,6 +1,8 @@
DROP TABLE IF EXISTS test.storage;
CREATE TABLE test.storage(UserID UInt64) ENGINE=Memory;
INSERT INTO test.storage(UserID) values (6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(402895971392036118)(402895971392036118)(402895971392036118);
DROP TABLE IF EXISTS storage;
CREATE TABLE storage(UserID UInt64) ENGINE=Memory;
INSERT INTO storage(UserID) values (6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(6460432721393873721)(402895971392036118)(402895971392036118)(402895971392036118);
SELECT sum(UserID GLOBAL IN (SELECT UserID FROM remote('127.0.0.{2,3}', test.storage))) FROM remote('127.0.0.{2,3}', test.storage);
SELECT sum(UserID GLOBAL IN (SELECT UserID FROM test.storage)) FROM remote('127.0.0.{2,3}', test.storage);
SELECT sum(UserID GLOBAL IN (SELECT UserID FROM remote('127.0.0.{2,3}', currentDatabase(), storage))) FROM remote('127.0.0.{2,3}', currentDatabase(), storage);
SELECT sum(UserID GLOBAL IN (SELECT UserID FROM storage)) FROM remote('127.0.0.{2,3}', currentDatabase(), storage);
DROP TABLE storage;

View File

@ -6,18 +6,18 @@ SELECT quantileTimingWeighted(0.5)(number, 1) FROM (SELECT number FROM system.nu
SELECT quantilesTimingWeighted(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(number, 1) FROM (SELECT number FROM system.numbers LIMIT 1001);
SELECT quantilesTimingWeighted(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(number, number) FROM (SELECT number FROM system.numbers LIMIT 1001);
DROP TABLE IF EXISTS test.numbers_1001;
CREATE TABLE test.numbers_1001 (number UInt64) ENGINE = Memory;
DROP TABLE IF EXISTS numbers_1001;
CREATE TABLE numbers_1001 (number UInt64) ENGINE = Memory;
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
SET max_block_size = 10;
INSERT INTO test.numbers_1001 SELECT number FROM system.numbers LIMIT 1001;
INSERT INTO numbers_1001 SELECT number FROM system.numbers LIMIT 1001;
SELECT quantileExactWeighted(0.5)(number, 1) FROM remote('127.0.0.{2,3}', test.numbers_1001);
SELECT quantilesExactWeighted(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(number, 1) FROM remote('127.0.0.{2,3}', test.numbers_1001);
SELECT quantilesExactWeighted(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(number, number) FROM remote('127.0.0.{2,3}', test.numbers_1001);
SELECT quantileExactWeighted(0.5)(number, 1) FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_1001);
SELECT quantilesExactWeighted(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(number, 1) FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_1001);
SELECT quantilesExactWeighted(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(number, number) FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_1001);
SELECT quantileTimingWeighted(0.5)(number, 1) FROM remote('127.0.0.{2,3}', test.numbers_1001);
SELECT quantilesTimingWeighted(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(number, 1) FROM remote('127.0.0.{2,3}', test.numbers_1001);
SELECT quantilesTimingWeighted(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(number, number) FROM remote('127.0.0.{2,3}', test.numbers_1001);
SELECT quantileTimingWeighted(0.5)(number, 1) FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_1001);
SELECT quantilesTimingWeighted(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(number, 1) FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_1001);
SELECT quantilesTimingWeighted(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(number, number) FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_1001);
DROP TABLE test.numbers_1001;
DROP TABLE numbers_1001;

View File

@ -1,17 +1,17 @@
DROP TABLE IF EXISTS test.numbers_10_00290;
DROP TABLE IF EXISTS numbers_10_00290;
SET max_block_size = 1000;
CREATE TABLE test.numbers_10_00290 ENGINE = Log AS SELECT * FROM system.numbers LIMIT 10000;
CREATE TABLE numbers_10_00290 ENGINE = Log AS SELECT * FROM system.numbers LIMIT 10000;
SET distributed_aggregation_memory_efficient = 1, group_by_two_level_threshold = 5000;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', test.numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', test.numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', test.numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', test.numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', test.numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', test.numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', test.numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', test.numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', test.numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', test.numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
SELECT concat(toString(number), arrayStringConcat(arrayMap(x -> '.', range(number % 10)))) AS k FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00290) WHERE number < (randConstant() % 2 ? 4999 : 10000) GROUP BY k ORDER BY k LIMIT 10;
DROP TABLE test.numbers_10_00290;
DROP TABLE numbers_10_00290;

View File

@ -1,80 +1,80 @@
set max_threads = 1;
drop table if exists test.enums;
drop table if exists enums;
create table test.enums (
create table enums (
d Date default '2015-12-29', k default 0,
e Enum8('world' = 2, 'hello' = 1), sign Enum8('minus' = -1, 'plus' = 1),
letter Enum16('a' = 0, 'b' = 1, 'c' = 2, '*' = -256)
) engine = MergeTree(d, k, 1);
desc table test.enums;
desc table enums;
-- insert default values
insert into test.enums (k) values (0);
select * from test.enums;
insert into enums (k) values (0);
select * from enums;
alter table test.enums modify column e Enum8('world' = 2, 'hello' = 1, '!' = 3);
desc table test.enums;
alter table enums modify column e Enum8('world' = 2, 'hello' = 1, '!' = 3);
desc table enums;
insert into test.enums (e, sign, letter) values ('!', 'plus', 'b');
select * from test.enums ORDER BY _part;
insert into enums (e, sign, letter) values ('!', 'plus', 'b');
select * from enums ORDER BY _part;
-- expand `e` and `sign` from Enum8 to Enum16 without changing values, change values of `letter` without changing type
alter table test.enums
alter table enums
modify column e Enum16('world' = 2, 'hello' = 1, '!' = 3),
modify column sign Enum16('minus' = -1, 'plus' = 1),
modify column letter Enum16('a' = 0, 'b' = 1, 'c' = 2, 'no letter' = -256);
desc table test.enums;
desc table enums;
select * from test.enums ORDER BY _part;
select * from enums ORDER BY _part;
alter table test.enums
alter table enums
modify column e Enum8('world' = 2, 'hello' = 1, '!' = 3),
modify column sign Enum8('minus' = -1, 'plus' = 1);
desc table test.enums;
desc table enums;
insert into test.enums (letter, e) values ('c', 'world');
select * from test.enums ORDER BY _part;
insert into enums (letter, e) values ('c', 'world');
select * from enums ORDER BY _part;
drop table test.enums;
drop table enums;
create table test.enums (e Enum8('a' = 0, 'b' = 1, 'c' = 2, 'd' = 3)) engine = TinyLog;
insert into test.enums values ('d'), ('b'), ('a'), ('c'), ('a'), ('d');
select * from test.enums;
create table enums (e Enum8('a' = 0, 'b' = 1, 'c' = 2, 'd' = 3)) engine = TinyLog;
insert into enums values ('d'), ('b'), ('a'), ('c'), ('a'), ('d');
select * from enums;
-- ORDER BY
select * from test.enums order by e;
select * from test.enums order by e desc;
select * from enums order by e;
select * from enums order by e desc;
-- GROUP BY
select count(), e from test.enums group by e;
select any(e) from test.enums;
select count(), e from enums group by e;
select any(e) from enums;
-- IN
select * from test.enums where e in ('a', 'd');
select * from test.enums where e in (select e from test.enums);
select * from enums where e in ('a', 'd');
select * from enums where e in (select e from enums);
-- DISTINCT
select distinct e from test.enums;
select distinct e from enums;
-- Comparison
select * from test.enums where e = e;
select * from test.enums where e = 'a' or e = 'd';
select * from test.enums where e != 'a';
select *, e < 'b' from test.enums;
select *, e > 'b' from test.enums;
select * from enums where e = e;
select * from enums where e = 'a' or e = 'd';
select * from enums where e != 'a';
select *, e < 'b' from enums;
select *, e > 'b' from enums;
-- Conversion
select toInt8(e), toInt16(e), toUInt64(e), toString(e), e from test.enums;
select toInt8(e), toInt16(e), toUInt64(e), toString(e), e from enums;
drop table if exists test.enums_copy;
create table test.enums_copy engine = TinyLog as select * from test.enums;
select * from test.enums_copy;
drop table if exists enums_copy;
create table enums_copy engine = TinyLog as select * from enums;
select * from enums_copy;
drop table test.enums_copy;
create table test.enums_copy engine = TinyLog as select * from remote('127.0.0.2', test, enums);
select * from remote('127.0.0.2', test, enums_copy);
drop table enums_copy;
create table enums_copy engine = TinyLog as select * from remote('127.0.0.2', currentDatabase(), enums);
select * from remote('127.0.0.2', currentDatabase(), enums_copy);
drop table test.enums_copy;
drop table test.enums;
drop table enums_copy;
drop table enums;

View File

@ -1,20 +1,20 @@
DROP TABLE IF EXISTS test.sample_00314_1;
DROP TABLE IF EXISTS test.sample_00314_2;
DROP TABLE IF EXISTS test.sample_merge_00314;
DROP TABLE IF EXISTS sample_00314_1;
DROP TABLE IF EXISTS sample_00314_2;
DROP TABLE IF EXISTS sample_merge_00314;
CREATE TABLE test.sample_00314_1 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10);
CREATE TABLE test.sample_00314_2 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10);
CREATE TABLE sample_00314_1 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10);
CREATE TABLE sample_00314_2 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10);
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
INSERT INTO test.sample_00314_1 (x) SELECT number AS x FROM system.numbers LIMIT 1000000;
INSERT INTO test.sample_00314_2 (x) SELECT number AS x FROM system.numbers LIMIT 2000000;
INSERT INTO sample_00314_1 (x) SELECT number AS x FROM system.numbers LIMIT 1000000;
INSERT INTO sample_00314_2 (x) SELECT number AS x FROM system.numbers LIMIT 2000000;
CREATE TABLE test.sample_merge_00314 AS test.sample_00314_1 ENGINE = Merge(test, '^sample_00314_\\d$');
CREATE TABLE sample_merge_00314 AS sample_00314_1 ENGINE = Merge(currentDatabase(), '^sample_00314_\\d$');
SELECT abs(sum(_sample_factor) - 3000000) / 3000000 < 0.001 FROM test.sample_merge_00314 SAMPLE 100000;
SELECT abs(sum(_sample_factor) - 3000000) / 3000000 < 0.001 FROM merge(test, '^sample_00314_\\d$') SAMPLE 100000;
SELECT abs(sum(_sample_factor) - 3000000) / 3000000 < 0.001 FROM sample_merge_00314 SAMPLE 100000;
SELECT abs(sum(_sample_factor) - 3000000) / 3000000 < 0.001 FROM merge(currentDatabase(), '^sample_00314_\\d$') SAMPLE 100000;
DROP TABLE test.sample_00314_1;
DROP TABLE test.sample_00314_2;
DROP TABLE test.sample_merge_00314;
DROP TABLE sample_00314_1;
DROP TABLE sample_00314_2;
DROP TABLE sample_merge_00314;

View File

@ -1,12 +1,12 @@
DROP TABLE IF EXISTS test.group_uniq_arr_int;
CREATE TABLE test.group_uniq_arr_int ENGINE = Memory AS
DROP TABLE IF EXISTS group_uniq_arr_int;
CREATE TABLE group_uniq_arr_int ENGINE = Memory AS
SELECT g as id, if(c == 0, [v], if(c == 1, emptyArrayInt64(), [v, v])) as v FROM
(SELECT intDiv(number%1000000, 100) as v, intDiv(number%100, 10) as g, number%10 as c FROM system.numbers WHERE c < 3 LIMIT 10000000);
SELECT length(groupUniqArray(v)) FROM test.group_uniq_arr_int GROUP BY id ORDER BY id;
SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', 'test', 'group_uniq_arr_int') GROUP BY id ORDER BY id;
SELECT length(groupUniqArray(10)(v)) FROM test.group_uniq_arr_int GROUP BY id ORDER BY id;
SELECT length(groupUniqArray(100000)(v)) FROM test.group_uniq_arr_int GROUP BY id ORDER BY id;
SELECT length(groupUniqArray(v)) FROM group_uniq_arr_int GROUP BY id ORDER BY id;
SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_arr_int') GROUP BY id ORDER BY id;
SELECT length(groupUniqArray(10)(v)) FROM group_uniq_arr_int GROUP BY id ORDER BY id;
SELECT length(groupUniqArray(100000)(v)) FROM group_uniq_arr_int GROUP BY id ORDER BY id;
DROP TABLE IF EXISTS test.group_uniq_arr_int;
DROP TABLE IF EXISTS group_uniq_arr_int;

View File

@ -1,9 +1,9 @@
DROP TABLE IF EXISTS test.group_uniq_arr_str;
CREATE TABLE test.group_uniq_arr_str ENGINE = Memory AS
DROP TABLE IF EXISTS group_uniq_arr_str;
CREATE TABLE group_uniq_arr_str ENGINE = Memory AS
SELECT hex(intHash32(g)) as id, if(c == 0, [hex(v)], if(c == 1, emptyArrayString(), [hex(v), hex(v)])) as v FROM
(SELECT intDiv(number%1000000, 100) as v, intDiv(number%100, 10) as g, number%10 as c FROM system.numbers WHERE c < 3 LIMIT 10000000);
SELECT length(groupUniqArray(v)) FROM test.group_uniq_arr_str GROUP BY id ORDER BY id;
SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', 'test', 'group_uniq_arr_str') GROUP BY id ORDER BY id;
SELECT length(groupUniqArray(v)) FROM group_uniq_arr_str GROUP BY id ORDER BY id;
SELECT length(groupUniqArray(v)) FROM remote('127.0.0.{2,3,4,5}', currentDatabase(), 'group_uniq_arr_str') GROUP BY id ORDER BY id;
DROP TABLE IF EXISTS test.group_uniq_arr_str;
DROP TABLE IF EXISTS group_uniq_arr_str;

View File

@ -40,3 +40,5 @@ SELECT cityHash64(groupArray(d)) FROM enum_pk WHERE d IN ('0', '1');
SELECT cityHash64(groupArray(x)) FROM enum_pk WHERE (x != '0' AND x != '1');
SELECT cityHash64(groupArray(d)) FROM enum_pk WHERE (d != '0' AND d != '1');
DROP TABLE enum_pk;

View File

@ -1,23 +1,22 @@
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.has_column_in_table;
CREATE TABLE test.has_column_in_table (i Int64, s String, nest Nested(x UInt8, y UInt32)) ENGINE = Memory;
DROP TABLE IF EXISTS has_column_in_table;
CREATE TABLE has_column_in_table (i Int64, s String, nest Nested(x UInt8, y UInt32)) ENGINE = Memory;
/* existing column */
SELECT hasColumnInTable('test', 'has_column_in_table', 'i');
SELECT hasColumnInTable('localhost', 'test', 'has_column_in_table', 'i');
SELECT hasColumnInTable('test', 'has_column_in_table', 's');
SELECT hasColumnInTable('localhost', 'test', 'has_column_in_table', 's');
SELECT hasColumnInTable('test', 'has_column_in_table', 'nest.x');
SELECT hasColumnInTable('localhost', 'test', 'has_column_in_table', 'nest.x');
SELECT hasColumnInTable('test', 'has_column_in_table', 'nest.y');
SELECT hasColumnInTable('localhost', 'test', 'has_column_in_table', 'nest.y');
SELECT hasColumnInTable(currentDatabase(), 'has_column_in_table', 'i');
SELECT hasColumnInTable('localhost', currentDatabase(), 'has_column_in_table', 'i');
SELECT hasColumnInTable(currentDatabase(), 'has_column_in_table', 's');
SELECT hasColumnInTable('localhost', currentDatabase(), 'has_column_in_table', 's');
SELECT hasColumnInTable(currentDatabase(), 'has_column_in_table', 'nest.x');
SELECT hasColumnInTable('localhost', currentDatabase(), 'has_column_in_table', 'nest.x');
SELECT hasColumnInTable(currentDatabase(), 'has_column_in_table', 'nest.y');
SELECT hasColumnInTable('localhost', currentDatabase(), 'has_column_in_table', 'nest.y');
/* not existing column */
SELECT hasColumnInTable('test', 'has_column_in_table', 'nest');
SELECT hasColumnInTable('localhost', 'test', 'has_column_in_table', 'nest');
SELECT hasColumnInTable('test', 'has_column_in_table', 'nest.not_existing');
SELECT hasColumnInTable('localhost', 'test', 'has_column_in_table', 'nest.not_existing');
SELECT hasColumnInTable('test', 'has_column_in_table', 'not_existing');
SELECT hasColumnInTable('localhost', 'test', 'has_column_in_table', 'not_existing');
SELECT hasColumnInTable(currentDatabase(), 'has_column_in_table', 'nest');
SELECT hasColumnInTable('localhost', currentDatabase(), 'has_column_in_table', 'nest');
SELECT hasColumnInTable(currentDatabase(), 'has_column_in_table', 'nest.not_existing');
SELECT hasColumnInTable('localhost', currentDatabase(), 'has_column_in_table', 'nest.not_existing');
SELECT hasColumnInTable(currentDatabase(), 'has_column_in_table', 'not_existing');
SELECT hasColumnInTable('localhost', currentDatabase(), 'has_column_in_table', 'not_existing');
DROP TABLE test.has_column_in_table;
DROP TABLE has_column_in_table;

View File

@ -1,4 +1,3 @@
SELECT '----- NULL value -----';
SELECT NULL;
@ -148,18 +147,18 @@ SELECT col1 FROM test1_00395 WHERE col1 IS NULL;
SELECT '----- if -----';
DROP TABLE IF EXISTS test.test1_00395;
CREATE TABLE test.test1_00395 (col1 Nullable(String)) ENGINE=TinyLog;
INSERT INTO test.test1_00395 VALUES ('a'), ('b'), ('c'), (NULL);
DROP TABLE IF EXISTS test1_00395;
CREATE TABLE test1_00395 (col1 Nullable(String)) ENGINE=TinyLog;
INSERT INTO test1_00395 VALUES ('a'), ('b'), ('c'), (NULL);
SELECT col1, if(col1 IN ('a' ,'b'), 1, 0) AS t, toTypeName(t) FROM test.test1_00395;
SELECT col1, if(col1 IN ('a' ,'b'), NULL, 0) AS t, toTypeName(t) FROM test.test1_00395;
SELECT col1, if(col1 IN ('a' ,'b'), 1, 0) AS t, toTypeName(t) FROM test1_00395;
SELECT col1, if(col1 IN ('a' ,'b'), NULL, 0) AS t, toTypeName(t) FROM test1_00395;
SELECT '----- case when -----';
SELECT col1, CASE WHEN col1 IN ('a' ,'b') THEN 1 ELSE 0 END AS t, toTypeName(t) FROM test.test1_00395;
SELECT col1, CASE WHEN col1 IN ('a' ,'b') THEN NULL ELSE 0 END AS t, toTypeName(t) FROM test.test1_00395;
SELECT col1, CASE WHEN col1 IN ('a' ,'b') THEN 1 END AS t, toTypeName(t) FROM test.test1_00395;
SELECT col1, CASE WHEN col1 IN ('a' ,'b') THEN 1 ELSE 0 END AS t, toTypeName(t) FROM test1_00395;
SELECT col1, CASE WHEN col1 IN ('a' ,'b') THEN NULL ELSE 0 END AS t, toTypeName(t) FROM test1_00395;
SELECT col1, CASE WHEN col1 IN ('a' ,'b') THEN 1 END AS t, toTypeName(t) FROM test1_00395;
SELECT '----- multiIf -----';
@ -497,3 +496,4 @@ INSERT INTO test1_00395(col1,col2) VALUES([NULL], 'ACDEFBGH');
SELECT col1, count() FROM test1_00395 GROUP BY col1 ORDER BY col1;
DROP TABLE IF EXISTS test1_00395;
DROP TABLE test2;

View File

@ -5,3 +5,4 @@ SET preferred_max_column_in_block_size_bytes = 32;
SELECT blockSize(), * from tab_00481 format Null;
SELECT 0;
DROP TABLE tab_00481;

View File

@ -32,3 +32,4 @@ insert into tab_00484 select today(), number, 'abc' from system.numbers limit 81
set preferred_block_size_bytes = 0;
select count(*) from tab_00484 prewhere s != 'abc' format Null;
select count(*) from tab_00484 prewhere s = 'abc' format Null;
drop table tab_00484;

View File

@ -1,8 +1,8 @@
DROP TABLE IF EXISTS test.nested;
CREATE TABLE test.nested (n Nested(x UInt8)) ENGINE = Memory;
INSERT INTO test.nested VALUES ([1, 2]);
SELECT 1 AS x FROM remote('127.0.0.2', test.nested) ARRAY JOIN n.x;
DROP TABLE test.nested;
DROP TABLE IF EXISTS nested;
CREATE TABLE nested (n Nested(x UInt8)) ENGINE = Memory;
INSERT INTO nested VALUES ([1, 2]);
SELECT 1 AS x FROM remote('127.0.0.2', currentDatabase(), nested) ARRAY JOIN n.x;
DROP TABLE nested;
SELECT dummy AS dummy, dummy AS b FROM system.one;
SELECT dummy AS dummy, dummy AS b, b AS c FROM system.one;

View File

@ -9,7 +9,7 @@ Sum before DETACH PARTITION:
Sum after DETACH PARTITION:
0
system.detached_parts after DETACH PARTITION:
test not_partitioned all all_1_2_1 1 2 1
default not_partitioned all all_1_2_1 1 2 1
*** Partitioned by week ***
Parts before OPTIMIZE:
1999-12-27 19991227_1_1_0

View File

@ -1,109 +1,109 @@
SELECT '*** Not partitioned ***';
DROP TABLE IF EXISTS test.not_partitioned;
CREATE TABLE test.not_partitioned(x UInt8) ENGINE MergeTree ORDER BY x;
DROP TABLE IF EXISTS not_partitioned;
CREATE TABLE not_partitioned(x UInt8) ENGINE MergeTree ORDER BY x;
INSERT INTO test.not_partitioned VALUES (1), (2), (3);
INSERT INTO test.not_partitioned VALUES (4), (5);
INSERT INTO not_partitioned VALUES (1), (2), (3);
INSERT INTO not_partitioned VALUES (4), (5);
SELECT 'Parts before OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'not_partitioned' AND active ORDER BY name;
OPTIMIZE TABLE test.not_partitioned PARTITION tuple() FINAL;
SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'not_partitioned' AND active ORDER BY name;
OPTIMIZE TABLE not_partitioned PARTITION tuple() FINAL;
SELECT 'Parts after OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'not_partitioned' AND active ORDER BY name;
SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'not_partitioned' AND active ORDER BY name;
SELECT 'Sum before DETACH PARTITION:';
SELECT sum(x) FROM test.not_partitioned;
ALTER TABLE test.not_partitioned DETACH PARTITION ID 'all';
SELECT sum(x) FROM not_partitioned;
ALTER TABLE not_partitioned DETACH PARTITION ID 'all';
SELECT 'Sum after DETACH PARTITION:';
SELECT sum(x) FROM test.not_partitioned;
SELECT sum(x) FROM not_partitioned;
SELECT 'system.detached_parts after DETACH PARTITION:';
SELECT * FROM system.detached_parts WHERE table = 'not_partitioned';
DROP TABLE test.not_partitioned;
DROP TABLE not_partitioned;
SELECT '*** Partitioned by week ***';
DROP TABLE IF EXISTS test.partitioned_by_week;
CREATE TABLE test.partitioned_by_week(d Date, x UInt8) ENGINE = MergeTree PARTITION BY toMonday(d) ORDER BY x;
DROP TABLE IF EXISTS partitioned_by_week;
CREATE TABLE partitioned_by_week(d Date, x UInt8) ENGINE = MergeTree PARTITION BY toMonday(d) ORDER BY x;
-- 2000-01-03 belongs to a different week than 2000-01-01 and 2000-01-02
INSERT INTO test.partitioned_by_week VALUES ('2000-01-01', 1), ('2000-01-02', 2), ('2000-01-03', 3);
INSERT INTO test.partitioned_by_week VALUES ('2000-01-03', 4), ('2000-01-03', 5);
INSERT INTO partitioned_by_week VALUES ('2000-01-01', 1), ('2000-01-02', 2), ('2000-01-03', 3);
INSERT INTO partitioned_by_week VALUES ('2000-01-03', 4), ('2000-01-03', 5);
SELECT 'Parts before OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_week' AND active ORDER BY name;
OPTIMIZE TABLE test.partitioned_by_week PARTITION '2000-01-03' FINAL;
SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_week' AND active ORDER BY name;
OPTIMIZE TABLE partitioned_by_week PARTITION '2000-01-03' FINAL;
SELECT 'Parts after OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_week' AND active ORDER BY name;
SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_week' AND active ORDER BY name;
SELECT 'Sum before DROP PARTITION:';
SELECT sum(x) FROM test.partitioned_by_week;
ALTER TABLE test.partitioned_by_week DROP PARTITION '1999-12-27';
SELECT sum(x) FROM partitioned_by_week;
ALTER TABLE partitioned_by_week DROP PARTITION '1999-12-27';
SELECT 'Sum after DROP PARTITION:';
SELECT sum(x) FROM test.partitioned_by_week;
SELECT sum(x) FROM partitioned_by_week;
DROP TABLE test.partitioned_by_week;
DROP TABLE partitioned_by_week;
SELECT '*** Partitioned by a (Date, UInt8) tuple ***';
DROP TABLE IF EXISTS test.partitioned_by_tuple;
CREATE TABLE test.partitioned_by_tuple(d Date, x UInt8, y UInt8) ENGINE MergeTree ORDER BY x PARTITION BY (d, x);
DROP TABLE IF EXISTS partitioned_by_tuple;
CREATE TABLE partitioned_by_tuple(d Date, x UInt8, y UInt8) ENGINE MergeTree ORDER BY x PARTITION BY (d, x);
INSERT INTO test.partitioned_by_tuple VALUES ('2000-01-01', 1, 1), ('2000-01-01', 2, 2), ('2000-01-02', 1, 3);
INSERT INTO test.partitioned_by_tuple VALUES ('2000-01-02', 1, 4), ('2000-01-01', 1, 5);
INSERT INTO partitioned_by_tuple VALUES ('2000-01-01', 1, 1), ('2000-01-01', 2, 2), ('2000-01-02', 1, 3);
INSERT INTO partitioned_by_tuple VALUES ('2000-01-02', 1, 4), ('2000-01-01', 1, 5);
SELECT 'Parts before OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_tuple' AND active ORDER BY name;
OPTIMIZE TABLE test.partitioned_by_tuple PARTITION ('2000-01-01', 1) FINAL;
OPTIMIZE TABLE test.partitioned_by_tuple PARTITION ('2000-01-02', 1) FINAL;
SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_tuple' AND active ORDER BY name;
OPTIMIZE TABLE partitioned_by_tuple PARTITION ('2000-01-01', 1) FINAL;
OPTIMIZE TABLE partitioned_by_tuple PARTITION ('2000-01-02', 1) FINAL;
SELECT 'Parts after OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_tuple' AND active ORDER BY name;
SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_tuple' AND active ORDER BY name;
SELECT 'Sum before DETACH PARTITION:';
SELECT sum(y) FROM test.partitioned_by_tuple;
ALTER TABLE test.partitioned_by_tuple DETACH PARTITION ID '20000101-1';
SELECT sum(y) FROM partitioned_by_tuple;
ALTER TABLE partitioned_by_tuple DETACH PARTITION ID '20000101-1';
SELECT 'Sum after DETACH PARTITION:';
SELECT sum(y) FROM test.partitioned_by_tuple;
SELECT sum(y) FROM partitioned_by_tuple;
DROP TABLE test.partitioned_by_tuple;
DROP TABLE partitioned_by_tuple;
SELECT '*** Partitioned by String ***';
DROP TABLE IF EXISTS test.partitioned_by_string;
CREATE TABLE test.partitioned_by_string(s String, x UInt8) ENGINE = MergeTree PARTITION BY s ORDER BY x;
DROP TABLE IF EXISTS partitioned_by_string;
CREATE TABLE partitioned_by_string(s String, x UInt8) ENGINE = MergeTree PARTITION BY s ORDER BY x;
INSERT INTO test.partitioned_by_string VALUES ('aaa', 1), ('aaa', 2), ('bbb', 3);
INSERT INTO test.partitioned_by_string VALUES ('bbb', 4), ('aaa', 5);
INSERT INTO partitioned_by_string VALUES ('aaa', 1), ('aaa', 2), ('bbb', 3);
INSERT INTO partitioned_by_string VALUES ('bbb', 4), ('aaa', 5);
SELECT 'Parts before OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_string' AND active ORDER BY name;
OPTIMIZE TABLE test.partitioned_by_string PARTITION 'aaa' FINAL;
SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_string' AND active ORDER BY name;
OPTIMIZE TABLE partitioned_by_string PARTITION 'aaa' FINAL;
SELECT 'Parts after OPTIMIZE:';
SELECT partition, name FROM system.parts WHERE database = 'test' AND table = 'partitioned_by_string' AND active ORDER BY name;
SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_string' AND active ORDER BY name;
SELECT 'Sum before DROP PARTITION:';
SELECT sum(x) FROM test.partitioned_by_string;
ALTER TABLE test.partitioned_by_string DROP PARTITION 'bbb';
SELECT sum(x) FROM partitioned_by_string;
ALTER TABLE partitioned_by_string DROP PARTITION 'bbb';
SELECT 'Sum after DROP PARTITION:';
SELECT sum(x) FROM test.partitioned_by_string;
SELECT sum(x) FROM partitioned_by_string;
DROP TABLE test.partitioned_by_string;
DROP TABLE partitioned_by_string;
SELECT '*** Table without columns with fixed size ***';
DROP TABLE IF EXISTS test.without_fixed_size_columns;
CREATE TABLE test.without_fixed_size_columns(s String) ENGINE MergeTree PARTITION BY length(s) ORDER BY s;
DROP TABLE IF EXISTS without_fixed_size_columns;
CREATE TABLE without_fixed_size_columns(s String) ENGINE MergeTree PARTITION BY length(s) ORDER BY s;
INSERT INTO test.without_fixed_size_columns VALUES ('a'), ('aa'), ('b'), ('cc');
INSERT INTO without_fixed_size_columns VALUES ('a'), ('aa'), ('b'), ('cc');
SELECT 'Parts:';
SELECT partition, name, rows FROM system.parts WHERE database = 'test' AND table = 'without_fixed_size_columns' AND active ORDER BY name;
SELECT partition, name, rows FROM system.parts WHERE database = currentDatabase() AND table = 'without_fixed_size_columns' AND active ORDER BY name;
SELECT 'Before DROP PARTITION:';
SELECT * FROM test.without_fixed_size_columns ORDER BY s;
ALTER TABLE test.without_fixed_size_columns DROP PARTITION 1;
SELECT * FROM without_fixed_size_columns ORDER BY s;
ALTER TABLE without_fixed_size_columns DROP PARTITION 1;
SELECT 'After DROP PARTITION:';
SELECT * FROM test.without_fixed_size_columns ORDER BY s;
SELECT * FROM without_fixed_size_columns ORDER BY s;
DROP TABLE test.without_fixed_size_columns;
DROP TABLE without_fixed_size_columns;

View File

@ -1,58 +1,57 @@
SELECT X FROM (SELECT * FROM (SELECT 1 AS X, 2 AS Y) UNION ALL SELECT 3, 4) ORDER BY X;
DROP TABLE IF EXISTS test.globalin;
DROP TABLE IF EXISTS globalin;
CREATE TABLE test.globalin (CounterID UInt32, StartDate Date ) ENGINE = Memory;
CREATE TABLE globalin (CounterID UInt32, StartDate Date ) ENGINE = Memory;
INSERT INTO test.globalin VALUES (34, toDate('2017-10-02')), (42, toDate('2017-10-02')), (55, toDate('2017-10-01'));
INSERT INTO globalin VALUES (34, toDate('2017-10-02')), (42, toDate('2017-10-02')), (55, toDate('2017-10-01'));
SELECT * FROM ( SELECT CounterID FROM remote('127.0.0.2', 'test', 'globalin') WHERE (CounterID GLOBAL IN ( SELECT toUInt32(34))) GROUP BY CounterID);
SELECT * FROM ( SELECT CounterID FROM remote('127.0.0.2', currentDatabase(), 'globalin') WHERE (CounterID GLOBAL IN ( SELECT toUInt32(34))) GROUP BY CounterID);
SELECT 'NOW okay =========================:';
SELECT CounterID FROM remote('127.0.0.2', 'test', 'globalin') WHERE (CounterID GLOBAL IN ( SELECT toUInt32(34) )) GROUP BY CounterID UNION ALL SELECT CounterID FROM remote('127.0.0.2', 'test', 'globalin') WHERE (CounterID GLOBAL IN ( SELECT toUInt32(34))) GROUP BY CounterID;
SELECT CounterID FROM remote('127.0.0.2', currentDatabase(), 'globalin') WHERE (CounterID GLOBAL IN ( SELECT toUInt32(34) )) GROUP BY CounterID UNION ALL SELECT CounterID FROM remote('127.0.0.2', currentDatabase(), 'globalin') WHERE (CounterID GLOBAL IN ( SELECT toUInt32(34))) GROUP BY CounterID;
SELECT 'NOW BAD ==========================:';
SELECT * FROM ( SELECT CounterID FROM remote('127.0.0.2', 'test', 'globalin') WHERE (CounterID GLOBAL IN ( SELECT toUInt32(34) )) GROUP BY CounterID UNION ALL SELECT CounterID FROM remote('127.0.0.2', 'test', 'globalin') WHERE (CounterID GLOBAL IN ( SELECT toUInt32(34))) GROUP BY CounterID);
SELECT * FROM ( SELECT CounterID FROM remote('127.0.0.2', currentDatabase(), 'globalin') WHERE (CounterID GLOBAL IN ( SELECT toUInt32(34) )) GROUP BY CounterID UNION ALL SELECT CounterID FROM remote('127.0.0.2', currentDatabase(), 'globalin') WHERE (CounterID GLOBAL IN ( SELECT toUInt32(34))) GROUP BY CounterID);
SELECT 'finish ===========================;';
DROP TABLE test.globalin;
DROP TABLE globalin;
DROP TABLE IF EXISTS test.union_bug;
DROP TABLE IF EXISTS union_bug;
CREATE TABLE test.union_bug (
CREATE TABLE union_bug (
Event String,
Datetime DateTime('Europe/Moscow')
) Engine = Memory;
INSERT INTO test.union_bug VALUES ('A', 1), ('B', 2);
INSERT INTO union_bug VALUES ('A', 1), ('B', 2);
SELECT ' * A UNION * B:';
SELECT * FROM (
SELECT * FROM test.union_bug WHERE Event = 'A'
SELECT * FROM union_bug WHERE Event = 'A'
UNION ALL
SELECT * FROM test.union_bug WHERE Event = 'B'
SELECT * FROM union_bug WHERE Event = 'B'
) ORDER BY Datetime;
SELECT ' Event, Datetime A UNION * B:';
SELECT * FROM (
SELECT Event, Datetime FROM test.union_bug WHERE Event = 'A'
SELECT Event, Datetime FROM union_bug WHERE Event = 'A'
UNION ALL
SELECT * FROM test.union_bug WHERE Event = 'B'
SELECT * FROM union_bug WHERE Event = 'B'
) ORDER BY Datetime;
SELECT ' * A UNION Event, Datetime B:';
SELECT * FROM (
SELECT * FROM test.union_bug WHERE Event = 'A'
SELECT * FROM union_bug WHERE Event = 'A'
UNION ALL
SELECT Event, Datetime FROM test.union_bug WHERE Event = 'B'
SELECT Event, Datetime FROM union_bug WHERE Event = 'B'
) ORDER BY Datetime;
SELECT ' Event, Datetime A UNION Event, Datetime B:';
SELECT * FROM (
SELECT Event, Datetime FROM test.union_bug WHERE Event = 'A'
SELECT Event, Datetime FROM union_bug WHERE Event = 'A'
UNION ALL
SELECT Event, Datetime FROM test.union_bug WHERE Event = 'B'
SELECT Event, Datetime FROM union_bug WHERE Event = 'B'
) ORDER BY Datetime;
DROP TABLE test.union_bug;
DROP TABLE union_bug;

View File

@ -1,24 +1,24 @@
-- https://github.com/yandex/ClickHouse/issues/1059
DROP TABLE IF EXISTS test.union1;
DROP TABLE IF EXISTS test.union2;
DROP TABLE IF EXISTS test.union3;
DROP TABLE IF EXISTS union1;
DROP TABLE IF EXISTS union2;
DROP TABLE IF EXISTS union3;
CREATE TABLE test.union1 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = MergeTree(date, (a, date), 8192);
CREATE TABLE test.union2 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = Distributed(test_shard_localhost, 'test', 'union1');
CREATE TABLE test.union3 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = Distributed(test_shard_localhost, 'test', 'union2');
CREATE TABLE union1 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = MergeTree(date, (a, date), 8192);
CREATE TABLE union2 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = Distributed(test_shard_localhost, currentDatabase(), 'union1');
CREATE TABLE union3 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = Distributed(test_shard_localhost, currentDatabase(), 'union2');
INSERT INTO test.union1 VALUES (1, 2, 3, 4, 5);
INSERT INTO test.union1 VALUES (11,12,13,14,15);
INSERT INTO test.union2 VALUES (21,22,23,24,25);
INSERT INTO test.union3 VALUES (31,32,33,34,35);
INSERT INTO union1 VALUES (1, 2, 3, 4, 5);
INSERT INTO union1 VALUES (11,12,13,14,15);
INSERT INTO union2 VALUES (21,22,23,24,25);
INSERT INTO union3 VALUES (31,32,33,34,35);
select b, sum(c) from ( select a, b, sum(c) as c from test.union2 where a>1 group by a,b UNION ALL select a, b, sum(c) as c from test.union2 where b>1 group by a, b ) as a group by b;
select b, sum(c) from ( select a, b, sum(c) as c from test.union1 where a>1 group by a,b UNION ALL select a, b, sum(c) as c from test.union2 where b>1 group by a, b ) as a group by b;
select b, sum(c) from ( select a, b, sum(c) as c from test.union1 where a>1 group by a,b UNION ALL select a, b, sum(c) as c from test.union1 where b>1 group by a, b ) as a group by b;
select b, sum(c) from ( select a, b, sum(c) as c from test.union2 where a>1 group by a,b UNION ALL select a, b, sum(c) as c from test.union3 where b>1 group by a, b ) as a group by b;
select b, sum(c) from ( select a, b, sum(c) as c from union2 where a>1 group by a,b UNION ALL select a, b, sum(c) as c from union2 where b>1 group by a, b ) as a group by b;
select b, sum(c) from ( select a, b, sum(c) as c from union1 where a>1 group by a,b UNION ALL select a, b, sum(c) as c from union2 where b>1 group by a, b ) as a group by b;
select b, sum(c) from ( select a, b, sum(c) as c from union1 where a>1 group by a,b UNION ALL select a, b, sum(c) as c from union1 where b>1 group by a, b ) as a group by b;
select b, sum(c) from ( select a, b, sum(c) as c from union2 where a>1 group by a,b UNION ALL select a, b, sum(c) as c from union3 where b>1 group by a, b ) as a group by b;
DROP TABLE test.union1;
DROP TABLE test.union2;
DROP TABLE test.union3;
DROP TABLE union1;
DROP TABLE union2;
DROP TABLE union3;

View File

@ -1,12 +1,12 @@
drop table if exists test.tab;
create table test.tab (date Date, val UInt64, val2 UInt8 default 42, val3 UInt8 default val2 + 1, val4 UInt64 alias val) engine = MergeTree(date, (date, val), 8192);
desc test.tab;
drop table if exists tab;
create table tab (date Date, val UInt64, val2 UInt8 default 42, val3 UInt8 default val2 + 1, val4 UInt64 alias val) engine = MergeTree(date, (date, val), 8192);
desc tab;
select '-';
desc table test.tab;
desc table tab;
select '-';
desc remote('127.0.0.2', test.tab);
desc remote('127.0.0.2', currentDatabase(), tab);
select '-';
desc table remote('127.0.0.2', test.tab);
desc table remote('127.0.0.2', currentDatabase(), tab);
select '-';
desc (select 1);
select '-';
@ -14,4 +14,4 @@ desc table (select 1);
select '-';
desc (select * from system.numbers);
select '-';
drop table if exists test.tab;
drop table if exists tab;

View File

@ -1,19 +1,19 @@
DROP TABLE IF EXISTS test.nums;
DROP TABLE IF EXISTS test.nums_buf;
DROP TABLE IF EXISTS nums;
DROP TABLE IF EXISTS nums_buf;
SET insert_allow_materialized_columns = 1;
CREATE TABLE test.nums ( n UInt64, m UInt64 MATERIALIZED n+1 ) ENGINE = Log;
CREATE TABLE test.nums_buf AS test.nums ENGINE = Buffer(test, nums, 1, 10, 100, 1, 3, 10000000, 100000000);
CREATE TABLE nums ( n UInt64, m UInt64 MATERIALIZED n+1 ) ENGINE = Log;
CREATE TABLE nums_buf AS nums ENGINE = Buffer(currentDatabase(), nums, 1, 10, 100, 1, 3, 10000000, 100000000);
INSERT INTO test.nums_buf (n) VALUES (1);
INSERT INTO test.nums_buf (n) VALUES (2);
INSERT INTO test.nums_buf (n) VALUES (3);
INSERT INTO test.nums_buf (n) VALUES (4);
INSERT INTO test.nums_buf (n) VALUES (5);
INSERT INTO nums_buf (n) VALUES (1);
INSERT INTO nums_buf (n) VALUES (2);
INSERT INTO nums_buf (n) VALUES (3);
INSERT INTO nums_buf (n) VALUES (4);
INSERT INTO nums_buf (n) VALUES (5);
SELECT n,m FROM test.nums ORDER BY n;
SELECT n,m FROM test.nums_buf ORDER BY n;
SELECT n,m FROM nums ORDER BY n;
SELECT n,m FROM nums_buf ORDER BY n;
DROP TABLE IF EXISTS test.nums_buf;
DROP TABLE IF EXISTS test.nums;
DROP TABLE IF EXISTS nums_buf;
DROP TABLE IF EXISTS nums;

View File

@ -1,8 +1,7 @@
create database if not exists test;
drop table if exists test.tab;
create table test.tab (val UInt8) engine = MergeTree order by val;
insert into function remote('127.0.0.2', test.tab) values (1);
insert into function remote('127.0.0.{2|3}', test.tab) values (2);
insert into function remote('127.0.0.{2|3|4}', test.tab) values (3);
select * from test.tab order by val;
drop table if exists tab;
create table tab (val UInt8) engine = MergeTree order by val;
insert into function remote('127.0.0.2', currentDatabase(), tab) values (1);
insert into function remote('127.0.0.{2|3}', currentDatabase(), tab) values (2);
insert into function remote('127.0.0.{2|3|4}', currentDatabase(), tab) values (3);
select * from tab order by val;
drop table tab;

View File

@ -5,4 +5,4 @@ insert into tab_00577 values ('2018-01-01', 0, 0);
select * from tab_00577 order by version;
OPTIMIZE TABLE tab_00577;
select * from tab_00577;
drop table tab_00577;

View File

@ -1,42 +1,42 @@
DROP TABLE IF EXISTS test.numbers1;
DROP TABLE IF EXISTS test.numbers2;
DROP TABLE IF EXISTS test.numbers3;
DROP TABLE IF EXISTS test.numbers4;
DROP TABLE IF EXISTS test.numbers5;
DROP TABLE IF EXISTS numbers1;
DROP TABLE IF EXISTS numbers2;
DROP TABLE IF EXISTS numbers3;
DROP TABLE IF EXISTS numbers4;
DROP TABLE IF EXISTS numbers5;
CREATE TABLE test.numbers1 ENGINE = StripeLog AS SELECT number FROM numbers(1000);
CREATE TABLE test.numbers2 ENGINE = TinyLog AS SELECT number FROM numbers(1000);
CREATE TABLE test.numbers3 ENGINE = Log AS SELECT number FROM numbers(1000);
CREATE TABLE test.numbers4 ENGINE = Memory AS SELECT number FROM numbers(1000);
CREATE TABLE test.numbers5 ENGINE = MergeTree ORDER BY number AS SELECT number FROM numbers(1000);
CREATE TABLE numbers1 ENGINE = StripeLog AS SELECT number FROM numbers(1000);
CREATE TABLE numbers2 ENGINE = TinyLog AS SELECT number FROM numbers(1000);
CREATE TABLE numbers3 ENGINE = Log AS SELECT number FROM numbers(1000);
CREATE TABLE numbers4 ENGINE = Memory AS SELECT number FROM numbers(1000);
CREATE TABLE numbers5 ENGINE = MergeTree ORDER BY number AS SELECT number FROM numbers(1000);
SELECT count() FROM merge(test, '^numbers\\d+$');
SELECT DISTINCT count() FROM merge(test, '^numbers\\d+$') GROUP BY number;
SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$');
SELECT DISTINCT count() FROM merge(currentDatabase(), '^numbers\\d+$') GROUP BY number;
SET max_rows_to_read = 1000;
SET max_threads = 'auto';
SELECT count() FROM merge(test, '^numbers\\d+$') WHERE _table = 'numbers1';
SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table = 'numbers1';
SET max_threads = 1;
SELECT count() FROM merge(test, '^numbers\\d+$') WHERE _table = 'numbers2';
SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table = 'numbers2';
SET max_threads = 10;
SELECT count() FROM merge(test, '^numbers\\d+$') WHERE _table = 'numbers3';
SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table = 'numbers3';
SET max_rows_to_read = 1;
SET max_threads = 'auto';
SELECT count() FROM merge(test, '^numbers\\d+$') WHERE _table = 'non_existing';
SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table = 'non_existing';
SET max_threads = 1;
SELECT count() FROM merge(test, '^numbers\\d+$') WHERE _table = 'non_existing';
SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table = 'non_existing';
SET max_threads = 10;
SELECT count() FROM merge(test, '^numbers\\d+$') WHERE _table = 'non_existing';
SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table = 'non_existing';
DROP TABLE test.numbers1;
DROP TABLE test.numbers2;
DROP TABLE test.numbers3;
DROP TABLE test.numbers4;
DROP TABLE test.numbers5;
DROP TABLE numbers1;
DROP TABLE numbers2;
DROP TABLE numbers3;
DROP TABLE numbers4;
DROP TABLE numbers5;

View File

@ -1,18 +1,18 @@
DROP TABLE IF EXISTS test.numbers1;
DROP TABLE IF EXISTS test.numbers2;
DROP TABLE IF EXISTS numbers1;
DROP TABLE IF EXISTS numbers2;
CREATE TABLE test.numbers1 ENGINE = Memory AS SELECT number FROM numbers(1000);
CREATE TABLE test.numbers2 ENGINE = Memory AS SELECT number FROM numbers(1000);
CREATE TABLE numbers1 ENGINE = Memory AS SELECT number FROM numbers(1000);
CREATE TABLE numbers2 ENGINE = Memory AS SELECT number FROM numbers(1000);
SELECT * FROM merge(test, '^numbers\\d+$') SAMPLE 0.1; -- { serverError 141 }
SELECT * FROM merge(currentDatabase(), '^numbers\\d+$') SAMPLE 0.1; -- { serverError 141 }
DROP TABLE test.numbers1;
DROP TABLE test.numbers2;
DROP TABLE numbers1;
DROP TABLE numbers2;
CREATE TABLE test.numbers1 ENGINE = MergeTree ORDER BY intHash32(number) SAMPLE BY intHash32(number) AS SELECT number FROM numbers(1000);
CREATE TABLE test.numbers2 ENGINE = MergeTree ORDER BY intHash32(number) SAMPLE BY intHash32(number) AS SELECT number FROM numbers(1000);
CREATE TABLE numbers1 ENGINE = MergeTree ORDER BY intHash32(number) SAMPLE BY intHash32(number) AS SELECT number FROM numbers(1000);
CREATE TABLE numbers2 ENGINE = MergeTree ORDER BY intHash32(number) SAMPLE BY intHash32(number) AS SELECT number FROM numbers(1000);
SELECT * FROM merge(test, '^numbers\\d+$') SAMPLE 0.01;
SELECT * FROM merge(currentDatabase(), '^numbers\\d+$') SAMPLE 0.01;
DROP TABLE test.numbers1;
DROP TABLE test.numbers2;
DROP TABLE numbers1;
DROP TABLE numbers2;

View File

@ -1,18 +1,18 @@
DROP TABLE IF EXISTS test.sample_00579_1;
DROP TABLE IF EXISTS test.sample_00579_2;
DROP TABLE IF EXISTS test.sample_merge_00579;
DROP TABLE IF EXISTS sample_00579_1;
DROP TABLE IF EXISTS sample_00579_2;
DROP TABLE IF EXISTS sample_merge_00579;
CREATE TABLE test.sample_00579_1 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10);
CREATE TABLE test.sample_00579_2 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10);
CREATE TABLE sample_00579_1 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10);
CREATE TABLE sample_00579_2 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10);
INSERT INTO test.sample_00579_1 (x) SELECT number AS x FROM system.numbers LIMIT 1000;
INSERT INTO test.sample_00579_2 (x) SELECT number AS x FROM system.numbers LIMIT 2000;
INSERT INTO sample_00579_1 (x) SELECT number AS x FROM system.numbers LIMIT 1000;
INSERT INTO sample_00579_2 (x) SELECT number AS x FROM system.numbers LIMIT 2000;
CREATE TABLE test.sample_merge_00579 AS test.sample_00579_1 ENGINE = Merge(test, '^sample_00579_\\d$');
CREATE TABLE sample_merge_00579 AS sample_00579_1 ENGINE = Merge(currentDatabase(), '^sample_00579_\\d$');
SET max_threads = 1;
SELECT _sample_factor FROM merge(test, '^sample_00579_\\d$');
SELECT _sample_factor FROM merge(currentDatabase(), '^sample_00579_\\d$');
DROP TABLE test.sample_00579_1;
DROP TABLE test.sample_00579_2;
DROP TABLE test.sample_merge_00579;
DROP TABLE sample_00579_1;
DROP TABLE sample_00579_2;
DROP TABLE sample_merge_00579;

View File

@ -1,11 +1,11 @@
DROP TABLE IF EXISTS test.local_statements;
DROP TABLE IF EXISTS test.statements;
DROP TABLE IF EXISTS local_statements;
DROP TABLE IF EXISTS statements;
CREATE TABLE test.local_statements ( statementId String, eventDate Date, eventHour DateTime, eventTime DateTime, verb String, objectId String, onCourse UInt8, courseId UInt16, contextRegistration String, resultScoreRaw Float64, resultScoreMin Float64, resultScoreMax Float64, resultSuccess UInt8, resultCompletition UInt8, resultDuration UInt32, resultResponse String, learnerId String, learnerHash String, contextId UInt16) ENGINE = MergeTree ORDER BY tuple();
CREATE TABLE local_statements ( statementId String, eventDate Date, eventHour DateTime, eventTime DateTime, verb String, objectId String, onCourse UInt8, courseId UInt16, contextRegistration String, resultScoreRaw Float64, resultScoreMin Float64, resultScoreMax Float64, resultSuccess UInt8, resultCompletition UInt8, resultDuration UInt32, resultResponse String, learnerId String, learnerHash String, contextId UInt16) ENGINE = MergeTree ORDER BY tuple();
CREATE TABLE test.statements ( statementId String, eventDate Date, eventHour DateTime, eventTime DateTime, verb String, objectId String, onCourse UInt8, courseId UInt16, contextRegistration String, resultScoreRaw Float64, resultScoreMin Float64, resultScoreMax Float64, resultSuccess UInt8, resultCompletition UInt8, resultDuration UInt32, resultResponse String, learnerId String, learnerHash String, contextId UInt16) ENGINE = Distributed(test_shard_localhost, 'test', 'local_statements', sipHash64(learnerHash));
CREATE TABLE statements ( statementId String, eventDate Date, eventHour DateTime, eventTime DateTime, verb String, objectId String, onCourse UInt8, courseId UInt16, contextRegistration String, resultScoreRaw Float64, resultScoreMin Float64, resultScoreMax Float64, resultSuccess UInt8, resultCompletition UInt8, resultDuration UInt32, resultResponse String, learnerId String, learnerHash String, contextId UInt16) ENGINE = Distributed(test_shard_localhost, currentDatabase(), 'local_statements', sipHash64(learnerHash));
INSERT INTO test.local_statements FORMAT CSV "2b3b04ee-0bb8-4200-906f-d47c48e56bd0","2016-08-25","2016-08-25 14:00:00","2016-08-25 14:43:34","http://adlnet.gov/expapi/verbs/passed","https://crmm.ru/xapi/courses/spp/2/0/3/2/8",0,1,"c13d788c-26e0-40e3-bacb-a1ff78ee1518",100,0,0,0,0,0,"","https://sberbank-school.ru/xapi/accounts/userid/94312","6f696f938a69b5e173093718e1c2bbf2",0
INSERT INTO local_statements FORMAT CSV "2b3b04ee-0bb8-4200-906f-d47c48e56bd0","2016-08-25","2016-08-25 14:00:00","2016-08-25 14:43:34","http://adlnet.gov/expapi/verbs/passed","https://crmm.ru/xapi/courses/spp/2/0/3/2/8",0,1,"c13d788c-26e0-40e3-bacb-a1ff78ee1518",100,0,0,0,0,0,"","https://sberbank-school.ru/xapi/accounts/userid/94312","6f696f938a69b5e173093718e1c2bbf2",0
SELECT avg(diff)
FROM
@ -16,7 +16,7 @@ FROM
SELECT
learnerHash,
passed - eventTime AS diff
FROM test.statements
FROM statements
GLOBAL ANY INNER JOIN
(
SELECT
@ -28,7 +28,7 @@ FROM
learnerHash,
eventTime,
resultScoreRaw
FROM test.statements
FROM statements
WHERE (courseId = 1) AND (onCourse = 0)
AND (verb = 'http://adlnet.gov/expapi/verbs/passed') AND (objectId = 'https://crmm.ru/xapi/courses/spp/1/1/0-1')
ORDER BY eventTime ASC
@ -44,5 +44,5 @@ FROM
LIMIT 7, 126
);
DROP TABLE test.local_statements;
DROP TABLE test.statements;
DROP TABLE local_statements;
DROP TABLE statements;

View File

@ -1,7 +1,7 @@
DROP TABLE IF EXISTS test.alias_local10;
DROP TABLE IF EXISTS test.alias10;
DROP TABLE IF EXISTS alias_local10;
DROP TABLE IF EXISTS alias10;
CREATE TABLE test.alias_local10 (
CREATE TABLE alias_local10 (
Id Int8,
EventDate Date DEFAULT '2000-01-01',
field1 Int8,
@ -9,52 +9,52 @@ CREATE TABLE test.alias_local10 (
field3 ALIAS CASE WHEN field1 = 1 THEN field2 ELSE '0' END
) ENGINE = MergeTree(EventDate, (Id, EventDate), 8192);
CREATE TABLE test.alias10 AS test.alias_local10 ENGINE = Distributed(test_shard_localhost, test, alias_local10, cityHash64(Id));
CREATE TABLE alias10 AS alias_local10 ENGINE = Distributed(test_shard_localhost, currentDatabase(), alias_local10, cityHash64(Id));
INSERT INTO test.alias_local10 (Id, EventDate, field1, field2) VALUES (1, '2000-01-01', 1, '12345'), (2, '2000-01-01', 2, '54321'), (3, '2000-01-01', 0, '');
INSERT INTO alias_local10 (Id, EventDate, field1, field2) VALUES (1, '2000-01-01', 1, '12345'), (2, '2000-01-01', 2, '54321'), (3, '2000-01-01', 0, '');
SELECT field1, field2, field3 FROM test.alias_local10;
SELECT field1, field2, field3 FROM test.alias_local10 WHERE EventDate='2000-01-01';
SELECT field1, field2 FROM test.alias_local10 WHERE EventDate='2000-01-01';
SELECT field1, field2, field3 FROM alias_local10;
SELECT field1, field2, field3 FROM alias_local10 WHERE EventDate='2000-01-01';
SELECT field1, field2 FROM alias_local10 WHERE EventDate='2000-01-01';
SELECT field1, field2, field3 FROM test.alias10;
SELECT field1, field2, field3 FROM test.alias10 WHERE EventDate='2000-01-01';
SELECT field1, field2 FROM test.alias10 WHERE EventDate='2000-01-01';
SELECT field1, field2, field3 FROM alias10;
SELECT field1, field2, field3 FROM alias10 WHERE EventDate='2000-01-01';
SELECT field1, field2 FROM alias10 WHERE EventDate='2000-01-01';
SELECT field2, field3 FROM test.alias10 WHERE EventDate='2000-01-01';
SELECT field3 FROM test.alias10 WHERE EventDate='2000-01-01';
SELECT field2, field3 FROM test.alias10;
SELECT field3 FROM test.alias10;
SELECT field2, field3 FROM alias10 WHERE EventDate='2000-01-01';
SELECT field3 FROM alias10 WHERE EventDate='2000-01-01';
SELECT field2, field3 FROM alias10;
SELECT field3 FROM alias10;
SELECT field1 FROM test.alias10 WHERE field3 = '12345';
SELECT field2 FROM test.alias10 WHERE field3 = '12345';
SELECT field3 FROM test.alias10 WHERE field3 = '12345';
SELECT field1 FROM alias10 WHERE field3 = '12345';
SELECT field2 FROM alias10 WHERE field3 = '12345';
SELECT field3 FROM alias10 WHERE field3 = '12345';
DROP TABLE test.alias10;
CREATE TABLE test.alias10 (
DROP TABLE alias10;
CREATE TABLE alias10 (
Id Int8,
EventDate Date,
field1 Int8,
field2 String,
field3 String
) ENGINE = Distributed(test_shard_localhost, test, alias_local10);
) ENGINE = Distributed(test_shard_localhost, currentDatabase(), alias_local10);
SELECT field1, field2, field3 FROM test.alias_local10;
SELECT field1, field2, field3 FROM test.alias_local10 WHERE EventDate='2000-01-01';
SELECT field1, field2 FROM test.alias_local10 WHERE EventDate='2000-01-01';
SELECT field1, field2, field3 FROM alias_local10;
SELECT field1, field2, field3 FROM alias_local10 WHERE EventDate='2000-01-01';
SELECT field1, field2 FROM alias_local10 WHERE EventDate='2000-01-01';
SELECT field1, field2, field3 FROM test.alias10;
SELECT field1, field2, field3 FROM test.alias10 WHERE EventDate='2000-01-01';
SELECT field1, field2 FROM test.alias10 WHERE EventDate='2000-01-01';
SELECT field1, field2, field3 FROM alias10;
SELECT field1, field2, field3 FROM alias10 WHERE EventDate='2000-01-01';
SELECT field1, field2 FROM alias10 WHERE EventDate='2000-01-01';
SELECT field2, field3 FROM test.alias10 WHERE EventDate='2000-01-01';
SELECT field3 FROM test.alias10 WHERE EventDate='2000-01-01';
SELECT field2, field3 FROM test.alias10;
SELECT field3 FROM test.alias10;
SELECT field2, field3 FROM alias10 WHERE EventDate='2000-01-01';
SELECT field3 FROM alias10 WHERE EventDate='2000-01-01';
SELECT field2, field3 FROM alias10;
SELECT field3 FROM alias10;
SELECT field1 FROM test.alias10 WHERE field3 = '12345';
SELECT field2 FROM test.alias10 WHERE field3 = '12345';
SELECT field3 FROM test.alias10 WHERE field3 = '12345';
SELECT field1 FROM alias10 WHERE field3 = '12345';
SELECT field2 FROM alias10 WHERE field3 = '12345';
SELECT field3 FROM alias10 WHERE field3 = '12345';
DROP TABLE test.alias_local10;
DROP TABLE test.alias10;
DROP TABLE alias_local10;
DROP TABLE alias10;

View File

@ -21,42 +21,42 @@ SELECT id\nFROM \n(\n SELECT arrayJoin([1, 2, 3]) AS id\n WHERE id = 1\n)
1
SELECT \n id, \n subquery\nFROM \n(\n SELECT \n 1 AS id, \n CAST(1, \'UInt8\') AS subquery\n WHERE subquery = 1\n) \nWHERE subquery = 1
1 1
SELECT \n a, \n b\nFROM \n(\n SELECT \n toUInt64(sum(id) AS b) AS a, \n b\n FROM test.test_00597 \n HAVING a = 3\n) \nWHERE a = 3
SELECT \n a, \n b\nFROM \n(\n SELECT \n toUInt64(sum(id) AS b) AS a, \n b\n FROM test_00597 \n HAVING a = 3\n) \nWHERE a = 3
3 3
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n name, \n value, \n min(id) AS id\n FROM test.test_00597 \n GROUP BY \n date, \n name, \n value\n HAVING id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n name, \n value, \n min(id) AS id\n FROM test_00597 \n GROUP BY \n date, \n name, \n value\n HAVING id = 1\n) \nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n a, \n b\nFROM \n(\n SELECT \n toUInt64(sum(id) AS b) AS a, \n b\n FROM test.test_00597 AS table_alias \n HAVING b = 3\n) AS outer_table_alias \nWHERE b = 3
SELECT \n a, \n b\nFROM \n(\n SELECT \n toUInt64(sum(id) AS b) AS a, \n b\n FROM test_00597 AS table_alias \n HAVING b = 3\n) AS outer_table_alias \nWHERE b = 3
3 3
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test.test_00597 \n WHERE id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n) \nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test.test_00597 \n WHERE id = 1\n ) \n WHERE id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n ) \n WHERE id = 1\n) \nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test.test_00597 \n WHERE id = 1\n ) AS b \n WHERE id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n ) AS b \n WHERE id = 1\n) \nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test.test_00597 \n WHERE id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n) \nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test.test_00597 \n WHERE id = 1\n ) \n WHERE id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n ) \n WHERE id = 1\n) \nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test.test_00597 \n WHERE id = 1\n) AS b \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n) AS b \nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test.test_00597 \n WHERE id = 1\n ) AS a \n WHERE id = 1\n) AS b \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n ) AS a \n WHERE id = 1\n) AS b \nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n id, \n date, \n value\nFROM \n(\n SELECT \n id, \n date, \n min(value) AS value\n FROM test.test_00597 \n WHERE id = 1\n GROUP BY \n id, \n date\n) \nWHERE id = 1
SELECT \n id, \n date, \n value\nFROM \n(\n SELECT \n id, \n date, \n min(value) AS value\n FROM test_00597 \n WHERE id = 1\n GROUP BY \n id, \n date\n) \nWHERE id = 1
1 2000-01-01 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test.test_00597 \n WHERE id = 1\n UNION ALL\n SELECT \n date, \n id, \n name, \n value\n FROM test.test_00597 \n WHERE id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n UNION ALL\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n) \nWHERE id = 1
2000-01-01 1 test string 1 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value, \n date, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test.test_00597 \n WHERE id = 1\n) \nANY LEFT JOIN \n(\n SELECT *\n FROM test.test_00597 \n) USING (id)\nWHERE id = 1
SELECT \n date, \n id, \n name, \n value, \n date, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n) \nANY LEFT JOIN \n(\n SELECT *\n FROM test_00597 \n) USING (id)\nWHERE id = 1
2000-01-01 1 test string 1 1 2000-01-01 test string 1 1
SELECT \n id, \n date, \n name, \n value\nFROM \n(\n SELECT toInt8(1) AS id\n) \nANY LEFT JOIN test.test_00597 USING (id)\nWHERE value = 1
SELECT \n id, \n date, \n name, \n value\nFROM \n(\n SELECT toInt8(1) AS id\n) \nANY LEFT JOIN \n(\n SELECT *\n FROM test_00597 \n) AS test_00597 USING (id)\nWHERE value = 1
1 2000-01-01 test string 1 1
SELECT value\nFROM \n(\n SELECT toInt8(1) AS id\n) \nANY LEFT JOIN test.test_00597 AS b USING (id)\nWHERE value = 1
SELECT value\nFROM \n(\n SELECT toInt8(1) AS id\n) \nANY LEFT JOIN test_00597 AS b USING (id)\nWHERE value = 1
1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value, \n date, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test.test_00597 \n WHERE id = 1\n ) \n ANY LEFT JOIN \n (\n SELECT *\n FROM test.test_00597 \n ) USING (id)\n WHERE id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value, \n date, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n ) \n ANY LEFT JOIN \n (\n SELECT *\n FROM test_00597 \n ) USING (id)\n WHERE id = 1\n) \nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value, \n b.date, \n b.name, \n b.value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test.test_00597 \n) \nANY LEFT JOIN \n(\n SELECT *\n FROM test.test_00597 \n) AS b USING (id)\nWHERE b.id = 1
SELECT \n date, \n id, \n name, \n value, \n b.date, \n b.name, \n b.value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n) \nANY LEFT JOIN \n(\n SELECT *\n FROM test_00597 \n) AS b USING (id)\nWHERE b.id = 1
2000-01-01 1 test string 1 1 2000-01-01 test string 1 1
SELECT \n id, \n date, \n name, \n value\nFROM \n(\n SELECT \n toInt8(1) AS id, \n toDate(\'2000-01-01\') AS date\n FROM system.numbers \n LIMIT 1\n) \nANY LEFT JOIN \n(\n SELECT *\n FROM test.test_00597 \n) AS b USING (date, id)\nWHERE b.date = toDate(\'2000-01-01\')
SELECT \n id, \n date, \n name, \n value\nFROM \n(\n SELECT \n toInt8(1) AS id, \n toDate(\'2000-01-01\') AS date\n FROM system.numbers \n LIMIT 1\n) \nANY LEFT JOIN \n(\n SELECT *\n FROM test_00597 \n) AS b USING (date, id)\nWHERE b.date = toDate(\'2000-01-01\')
1 2000-01-01 test string 1 1
SELECT \n date, \n id, \n name, \n value, \n `b.date`, \n `b.id`, \n `b.name`, \n `b.value`\nFROM \n(\n SELECT \n date, \n id, \n name, \n value, \n b.date, \n b.id, \n b.name, \n b.value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test.test_00597 \n WHERE id = 1\n ) AS a \n ANY LEFT JOIN \n (\n SELECT *\n FROM test.test_00597 \n ) AS b ON id = b.id\n WHERE id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value, \n `b.date`, \n `b.id`, \n `b.name`, \n `b.value`\nFROM \n(\n SELECT \n date, \n id, \n name, \n value, \n b.date, \n b.id, \n b.name, \n b.value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n ) AS a \n ANY LEFT JOIN \n (\n SELECT *\n FROM test_00597 \n ) AS b ON id = b.id\n WHERE id = 1\n) \nWHERE id = 1
2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1

View File

@ -1,13 +1,13 @@
SET send_logs_level = 'none';
DROP TABLE IF EXISTS test.test_00597;
DROP TABLE IF EXISTS test.test_view_00597;
DROP TABLE IF EXISTS test_00597;
DROP TABLE IF EXISTS test_view_00597;
CREATE TABLE test.test_00597(date Date, id Int8, name String, value Int64) ENGINE = MergeTree(date, (id, date), 8192);
CREATE VIEW test.test_view_00597 AS SELECT * FROM test.test_00597;
CREATE TABLE test_00597(date Date, id Int8, name String, value Int64) ENGINE = MergeTree(date, (id, date), 8192);
CREATE VIEW test_view_00597 AS SELECT * FROM test_00597;
INSERT INTO test.test_00597 VALUES('2000-01-01', 1, 'test string 1', 1);
INSERT INTO test.test_00597 VALUES('2000-01-01', 2, 'test string 2', 2);
INSERT INTO test_00597 VALUES('2000-01-01', 1, 'test string 1', 1);
INSERT INTO test_00597 VALUES('2000-01-01', 2, 'test string 2', 2);
SET enable_optimize_predicate_expression = 1;
SET enable_debug_queries = 1;
@ -16,7 +16,7 @@ SELECT '-------No need for predicate optimization, but still works-------';
SELECT 1;
SELECT 1 AS id WHERE id = 1;
SELECT arrayJoin([1,2,3]) AS id WHERE id = 1;
SELECT * FROM test.test_00597 WHERE id = 1;
SELECT * FROM test_00597 WHERE id = 1;
SELECT '-------Forbid push down-------';
@ -57,76 +57,76 @@ ANALYZE SELECT * FROM (SELECT 1 AS id, (SELECT 1) as subquery) WHERE subquery =
SELECT * FROM (SELECT 1 AS id, (SELECT 1) as subquery) WHERE subquery = 1;
-- Optimize predicate expressions using tables
ANALYZE SELECT * FROM (SELECT toUInt64(b) AS a, sum(id) AS b FROM test.test_00597) WHERE a = 3;
SELECT * FROM (SELECT toUInt64(b) AS a, sum(id) AS b FROM test.test_00597) WHERE a = 3;
ANALYZE SELECT * FROM (SELECT toUInt64(b) AS a, sum(id) AS b FROM test_00597) WHERE a = 3;
SELECT * FROM (SELECT toUInt64(b) AS a, sum(id) AS b FROM test_00597) WHERE a = 3;
ANALYZE SELECT date, id, name, value FROM (SELECT date, name, value, min(id) AS id FROM test.test_00597 GROUP BY date, name, value) WHERE id = 1;
SELECT date, id, name, value FROM (SELECT date, name, value, min(id) AS id FROM test.test_00597 GROUP BY date, name, value) WHERE id = 1;
ANALYZE SELECT date, id, name, value FROM (SELECT date, name, value, min(id) AS id FROM test_00597 GROUP BY date, name, value) WHERE id = 1;
SELECT date, id, name, value FROM (SELECT date, name, value, min(id) AS id FROM test_00597 GROUP BY date, name, value) WHERE id = 1;
ANALYZE SELECT * FROM (SELECT toUInt64(b) AS a, sum(id) AS b FROM test.test_00597 AS table_alias) AS outer_table_alias WHERE outer_table_alias.b = 3;
SELECT * FROM (SELECT toUInt64(b) AS a, sum(id) AS b FROM test.test_00597 AS table_alias) AS outer_table_alias WHERE outer_table_alias.b = 3;
ANALYZE SELECT * FROM (SELECT toUInt64(b) AS a, sum(id) AS b FROM test_00597 AS table_alias) AS outer_table_alias WHERE outer_table_alias.b = 3;
SELECT * FROM (SELECT toUInt64(b) AS a, sum(id) AS b FROM test_00597 AS table_alias) AS outer_table_alias WHERE outer_table_alias.b = 3;
-- Optimize predicate expression with asterisk
ANALYZE SELECT * FROM (SELECT * FROM test.test_00597) WHERE id = 1;
SELECT * FROM (SELECT * FROM test.test_00597) WHERE id = 1;
ANALYZE SELECT * FROM (SELECT * FROM test_00597) WHERE id = 1;
SELECT * FROM (SELECT * FROM test_00597) WHERE id = 1;
-- Optimize predicate expression with asterisk and nested subquery
ANALYZE SELECT * FROM (SELECT * FROM (SELECT * FROM test.test_00597)) WHERE id = 1;
SELECT * FROM (SELECT * FROM (SELECT * FROM test.test_00597)) WHERE id = 1;
ANALYZE SELECT * FROM (SELECT * FROM (SELECT * FROM test_00597)) WHERE id = 1;
SELECT * FROM (SELECT * FROM (SELECT * FROM test_00597)) WHERE id = 1;
-- Optimize predicate expression with qualified asterisk
ANALYZE SELECT * FROM (SELECT b.* FROM (SELECT * FROM test.test_00597) AS b) WHERE id = 1;
SELECT * FROM (SELECT b.* FROM (SELECT * FROM test.test_00597) AS b) WHERE id = 1;
ANALYZE SELECT * FROM (SELECT b.* FROM (SELECT * FROM test_00597) AS b) WHERE id = 1;
SELECT * FROM (SELECT b.* FROM (SELECT * FROM test_00597) AS b) WHERE id = 1;
-- Optimize predicate expression without asterisk
ANALYZE SELECT * FROM (SELECT date, id, name, value FROM test.test_00597) WHERE id = 1;
SELECT * FROM (SELECT date, id, name, value FROM test.test_00597) WHERE id = 1;
ANALYZE SELECT * FROM (SELECT date, id, name, value FROM test_00597) WHERE id = 1;
SELECT * FROM (SELECT date, id, name, value FROM test_00597) WHERE id = 1;
-- Optimize predicate expression without asterisk and contains nested subquery
ANALYZE SELECT * FROM (SELECT date, id, name, value FROM (SELECT date, id, name, value FROM test.test_00597)) WHERE id = 1;
SELECT * FROM (SELECT date, id, name, value FROM (SELECT date, id, name, value FROM test.test_00597)) WHERE id = 1;
ANALYZE SELECT * FROM (SELECT date, id, name, value FROM (SELECT date, id, name, value FROM test_00597)) WHERE id = 1;
SELECT * FROM (SELECT date, id, name, value FROM (SELECT date, id, name, value FROM test_00597)) WHERE id = 1;
-- Optimize predicate expression with qualified
ANALYZE SELECT * FROM (SELECT * FROM test.test_00597) AS b WHERE b.id = 1;
SELECT * FROM (SELECT * FROM test.test_00597) AS b WHERE b.id = 1;
ANALYZE SELECT * FROM (SELECT * FROM test_00597) AS b WHERE b.id = 1;
SELECT * FROM (SELECT * FROM test_00597) AS b WHERE b.id = 1;
-- Optimize predicate expression with qualified and nested subquery
ANALYZE SELECT * FROM (SELECT * FROM (SELECT * FROM test.test_00597) AS a) AS b WHERE b.id = 1;
SELECT * FROM (SELECT * FROM (SELECT * FROM test.test_00597) AS a) AS b WHERE b.id = 1;
ANALYZE SELECT * FROM (SELECT * FROM (SELECT * FROM test_00597) AS a) AS b WHERE b.id = 1;
SELECT * FROM (SELECT * FROM (SELECT * FROM test_00597) AS a) AS b WHERE b.id = 1;
-- Optimize predicate expression with aggregate function
ANALYZE SELECT * FROM (SELECT id, date, min(value) AS value FROM test.test_00597 GROUP BY id, date) WHERE id = 1;
SELECT * FROM (SELECT id, date, min(value) AS value FROM test.test_00597 GROUP BY id, date) WHERE id = 1;
ANALYZE SELECT * FROM (SELECT id, date, min(value) AS value FROM test_00597 GROUP BY id, date) WHERE id = 1;
SELECT * FROM (SELECT id, date, min(value) AS value FROM test_00597 GROUP BY id, date) WHERE id = 1;
-- Optimize predicate expression with union all query
ANALYZE SELECT * FROM (SELECT * FROM test.test_00597 UNION ALL SELECT * FROM test.test_00597) WHERE id = 1;
SELECT * FROM (SELECT * FROM test.test_00597 UNION ALL SELECT * FROM test.test_00597) WHERE id = 1;
ANALYZE SELECT * FROM (SELECT * FROM test_00597 UNION ALL SELECT * FROM test_00597) WHERE id = 1;
SELECT * FROM (SELECT * FROM test_00597 UNION ALL SELECT * FROM test_00597) WHERE id = 1;
-- Optimize predicate expression with join query
ANALYZE SELECT * FROM (SELECT * FROM test.test_00597) ANY LEFT JOIN (SELECT * FROM test.test_00597) USING id WHERE id = 1;
SELECT * FROM (SELECT * FROM test.test_00597) ANY LEFT JOIN (SELECT * FROM test.test_00597) USING id WHERE id = 1;
ANALYZE SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) USING id WHERE id = 1;
SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) USING id WHERE id = 1;
ANALYZE SELECT * FROM (SELECT toInt8(1) AS id) ANY LEFT JOIN test.test_00597 USING id WHERE value = 1;
SELECT * FROM (SELECT toInt8(1) AS id) ANY LEFT JOIN test.test_00597 USING id WHERE value = 1;
ANALYZE SELECT * FROM (SELECT toInt8(1) AS id) ANY LEFT JOIN test_00597 USING id WHERE value = 1;
SELECT * FROM (SELECT toInt8(1) AS id) ANY LEFT JOIN test_00597 USING id WHERE value = 1;
-- FIXME: no support for aliased tables for now.
ANALYZE SELECT b.value FROM (SELECT toInt8(1) AS id) ANY LEFT JOIN test.test_00597 AS b USING id WHERE value = 1;
SELECT b.value FROM (SELECT toInt8(1) AS id) ANY LEFT JOIN test.test_00597 AS b USING id WHERE value = 1;
ANALYZE SELECT b.value FROM (SELECT toInt8(1) AS id) ANY LEFT JOIN test_00597 AS b USING id WHERE value = 1;
SELECT b.value FROM (SELECT toInt8(1) AS id) ANY LEFT JOIN test_00597 AS b USING id WHERE value = 1;
-- Optimize predicate expression with join and nested subquery
ANALYZE SELECT * FROM (SELECT * FROM (SELECT * FROM test.test_00597) ANY LEFT JOIN (SELECT * FROM test.test_00597) USING id) WHERE id = 1;
SELECT * FROM (SELECT * FROM (SELECT * FROM test.test_00597) ANY LEFT JOIN (SELECT * FROM test.test_00597) USING id) WHERE id = 1;
ANALYZE SELECT * FROM (SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) USING id) WHERE id = 1;
SELECT * FROM (SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) USING id) WHERE id = 1;
-- Optimize predicate expression with join query and qualified
ANALYZE SELECT * FROM (SELECT * FROM test.test_00597) ANY LEFT JOIN (SELECT * FROM test.test_00597) AS b USING id WHERE b.id = 1;
SELECT * FROM (SELECT * FROM test.test_00597) ANY LEFT JOIN (SELECT * FROM test.test_00597) AS b USING id WHERE b.id = 1;
ANALYZE SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) AS b USING id WHERE b.id = 1;
SELECT * FROM (SELECT * FROM test_00597) ANY LEFT JOIN (SELECT * FROM test_00597) AS b USING id WHERE b.id = 1;
-- Compatibility test
ANALYZE SELECT * FROM (SELECT toInt8(1) AS id, toDate('2000-01-01') AS date FROM system.numbers LIMIT 1) ANY LEFT JOIN (SELECT * FROM test.test_00597) AS b USING date, id WHERE b.date = toDate('2000-01-01');
SELECT * FROM (SELECT toInt8(1) AS id, toDate('2000-01-01') AS date FROM system.numbers LIMIT 1) ANY LEFT JOIN (SELECT * FROM test.test_00597) AS b USING date, id WHERE b.date = toDate('2000-01-01');
ANALYZE SELECT * FROM (SELECT toInt8(1) AS id, toDate('2000-01-01') AS date FROM system.numbers LIMIT 1) ANY LEFT JOIN (SELECT * FROM test_00597) AS b USING date, id WHERE b.date = toDate('2000-01-01');
SELECT * FROM (SELECT toInt8(1) AS id, toDate('2000-01-01') AS date FROM system.numbers LIMIT 1) ANY LEFT JOIN (SELECT * FROM test_00597) AS b USING date, id WHERE b.date = toDate('2000-01-01');
ANALYZE SELECT * FROM (SELECT * FROM (SELECT * FROM test.test_00597) AS a ANY LEFT JOIN (SELECT * FROM test.test_00597) AS b ON a.id = b.id) WHERE id = 1;
SELECT * FROM (SELECT * FROM (SELECT * FROM test.test_00597) AS a ANY LEFT JOIN (SELECT * FROM test.test_00597) AS b ON a.id = b.id) WHERE id = 1;
ANALYZE SELECT * FROM (SELECT * FROM (SELECT * FROM test_00597) AS a ANY LEFT JOIN (SELECT * FROM test_00597) AS b ON a.id = b.id) WHERE id = 1;
SELECT * FROM (SELECT * FROM (SELECT * FROM test_00597) AS a ANY LEFT JOIN (SELECT * FROM test_00597) AS b ON a.id = b.id) WHERE id = 1;
DROP TABLE IF EXISTS test.test_00597;
DROP TABLE IF EXISTS test.test_view_00597;
DROP TABLE IF EXISTS test_00597;
DROP TABLE IF EXISTS test_view_00597;

View File

@ -1 +1 @@
CREATE VIEW test.test_view_00599 (`id` UInt64) AS SELECT * FROM test.test_00599 WHERE id = (SELECT 1)
CREATE VIEW default.test_view_00599 (`id` UInt64) AS SELECT * FROM default.test_00599 WHERE id = (SELECT 1)

View File

@ -1,13 +1,13 @@
DROP TABLE IF EXISTS test.test_00599;
DROP TABLE IF EXISTS test.test_view_00599;
DROP TABLE IF EXISTS test_00599;
DROP TABLE IF EXISTS test_view_00599;
CREATE TABLE test.test_00599(id UInt64) ENGINE = Log;
CREATE VIEW test.test_view_00599 AS SELECT * FROM test.test_00599 WHERE id = (SELECT 1);
CREATE TABLE test_00599(id UInt64) ENGINE = Log;
CREATE VIEW test_view_00599 AS SELECT * FROM test_00599 WHERE id = (SELECT 1);
DETACH TABLE test.test_view_00599;
ATTACH TABLE test.test_view_00599;
DETACH TABLE test_view_00599;
ATTACH TABLE test_view_00599;
SHOW CREATE TABLE test.test_view_00599;
SHOW CREATE TABLE test_view_00599;
DROP TABLE IF EXISTS test.test_00599;
DROP TABLE IF EXISTS test.test_view_00599;
DROP TABLE IF EXISTS test_00599;
DROP TABLE IF EXISTS test_view_00599;

View File

@ -1,38 +1,38 @@
DROP TABLE IF EXISTS test.t1;
DROP TABLE IF EXISTS test.t2;
DROP TABLE IF EXISTS test.t3;
DROP TABLE IF EXISTS test.t4;
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
DROP TABLE IF EXISTS t3;
DROP TABLE IF EXISTS t4;
CREATE TABLE test.t1(x UInt32, y UInt32) ENGINE TinyLog;
CREATE TABLE test.t2(x UInt32, y UInt32 DEFAULT x + 1) ENGINE TinyLog;
CREATE TABLE test.t3(x UInt32, y UInt32 MATERIALIZED x + 1) ENGINE TinyLog;
CREATE TABLE test.t4(x UInt32, y UInt32 ALIAS x + 1) ENGINE TinyLog;
CREATE TABLE t1(x UInt32, y UInt32) ENGINE TinyLog;
CREATE TABLE t2(x UInt32, y UInt32 DEFAULT x + 1) ENGINE TinyLog;
CREATE TABLE t3(x UInt32, y UInt32 MATERIALIZED x + 1) ENGINE TinyLog;
CREATE TABLE t4(x UInt32, y UInt32 ALIAS x + 1) ENGINE TinyLog;
INSERT INTO test.t1 VALUES (1, 1);
INSERT INTO test.t2 VALUES (1, 1);
INSERT INTO test.t3 VALUES (1);
INSERT INTO test.t4 VALUES (1);
INSERT INTO t1 VALUES (1, 1);
INSERT INTO t2 VALUES (1, 1);
INSERT INTO t3 VALUES (1);
INSERT INTO t4 VALUES (1);
INSERT INTO FUNCTION remote('127.0.0.2', test.t1) VALUES (2, 2);
INSERT INTO FUNCTION remote('127.0.0.2', test.t2) VALUES (2, 2);
INSERT INTO FUNCTION remote('127.0.0.2', currentDatabase(), t1) VALUES (2, 2);
INSERT INTO FUNCTION remote('127.0.0.2', currentDatabase(), t2) VALUES (2, 2);
--TODO: INSERT into remote tables with MATERIALIZED columns.
--INSERT INTO FUNCTION remote('127.0.0.2', test.t3) VALUES (2);
INSERT INTO FUNCTION remote('127.0.0.2', test.t4) VALUES (2);
--INSERT INTO FUNCTION remote('127.0.0.2', currentDatabase(), t3) VALUES (2);
INSERT INTO FUNCTION remote('127.0.0.2', currentDatabase(), t4) VALUES (2);
SELECT * FROM remote('127.0.0.2', test.t1) ORDER BY x;
SELECT * FROM remote('127.0.0.2', currentDatabase(), t1) ORDER BY x;
SELECT '*** With a DEFAULT column ***';
SELECT * FROM remote('127.0.0.2', test.t2) ORDER BY x;
SELECT * FROM remote('127.0.0.2', currentDatabase(), t2) ORDER BY x;
SELECT '*** With a MATERIALIZED column ***';
SELECT * FROM remote('127.0.0.2', test.t3) ORDER BY x;
SELECT x, y FROM remote('127.0.0.2', test.t3) ORDER BY x;
SELECT * FROM remote('127.0.0.2', currentDatabase(), t3) ORDER BY x;
SELECT x, y FROM remote('127.0.0.2', currentDatabase(), t3) ORDER BY x;
SELECT '*** With an ALIAS column ***';
SELECT * FROM remote('127.0.0.2', test.t4) ORDER BY x;
SELECT x, y FROM remote('127.0.0.2', test.t4) ORDER BY x;
SELECT * FROM remote('127.0.0.2', currentDatabase(), t4) ORDER BY x;
SELECT x, y FROM remote('127.0.0.2', currentDatabase(), t4) ORDER BY x;
DROP TABLE test.t1;
DROP TABLE test.t2;
DROP TABLE test.t3;
DROP TABLE test.t4;
DROP TABLE t1;
DROP TABLE t2;
DROP TABLE t3;
DROP TABLE t4;

Some files were not shown because too many files have changed in this diff Show More