Merge branch 'master' of github.com:yandex/ClickHouse into session-cleaner-better-build-time

This commit is contained in:
Alexey Milovidov 2020-03-06 19:31:52 +03:00
commit 49d6fec359
85 changed files with 2491 additions and 213 deletions

View File

@ -12,6 +12,3 @@ ClickHouse is an open-source column-oriented database management system that all
* [Contacts](https://clickhouse.tech/#contacts) can help to get your questions answered if there are any.
* You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person.
## Upcoming Events
* [ClickHouse Meetup in Athens](https://www.meetup.com/Athens-Big-Data/events/268379195/) on March 5.

View File

@ -8,6 +8,7 @@ add_library (mysqlxx
src/Row.cpp
src/Value.cpp
src/Pool.cpp
src/PoolFactory.cpp
src/PoolWithFailover.cpp
include/mysqlxx/Connection.h
@ -15,6 +16,7 @@ add_library (mysqlxx
include/mysqlxx/mysqlxx.h
include/mysqlxx/Null.h
include/mysqlxx/Pool.h
include/mysqlxx/PoolFactory.h
include/mysqlxx/PoolWithFailover.h
include/mysqlxx/Query.h
include/mysqlxx/ResultBase.h

View File

@ -198,6 +198,8 @@ public:
return description;
}
void removeConnection(Connection * data);
protected:
/// Number of MySQL connections which are created at launch.
unsigned default_connections;

View File

@ -0,0 +1,55 @@
#pragma once
#include <mutex>
#include <memory>
#include <boost/noncopyable.hpp>
#include <mysqlxx/PoolWithFailover.h>
#define MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS 1
#define MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS 16
#define MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES 3
namespace mysqlxx
{
/*
* PoolFactory.h
* This class is a helper singleton to mutualize connections to MySQL.
*/
class PoolFactory final : private boost::noncopyable
{
public:
static PoolFactory & instance();
PoolFactory(const PoolFactory &) = delete;
/** Allocates a PoolWithFailover to connect to MySQL. */
PoolWithFailover Get(const std::string & config_name,
unsigned default_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS,
unsigned max_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS,
size_t max_tries = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES);
/** Allocates a PoolWithFailover to connect to MySQL. */
PoolWithFailover Get(const Poco::Util::AbstractConfiguration & config,
const std::string & config_name,
unsigned default_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS,
unsigned max_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS,
size_t max_tries = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES);
void reset();
~PoolFactory() = default;
PoolFactory& operator=(const PoolFactory &) = delete;
private:
PoolFactory();
struct Impl;
std::unique_ptr<Impl> impl;
};
}

View File

@ -77,6 +77,10 @@ namespace mysqlxx
size_t max_tries;
/// Mutex for set of replicas.
std::mutex mutex;
std::string config_name;
/// Can the Pool be shared
bool shareable;
public:
using Entry = Pool::Entry;
@ -100,8 +104,6 @@ namespace mysqlxx
PoolWithFailover(const PoolWithFailover & other);
PoolWithFailover & operator=(const PoolWithFailover &) = delete;
/** Allocates a connection to use. */
Entry Get();
};

View File

@ -23,26 +23,26 @@ namespace mysqlxx
class ResultBase;
/** Представляет одно значение, считанное из MySQL.
* Объект сам не хранит данные, а является всего лишь обёрткой над парой (const char *, size_t).
* Если уничтожить UseQueryResult/StoreQueryResult или Connection,
* или считать следующий Row при использовании UseQueryResult, то объект станет некорректным.
* Позволяет преобразовать значение (распарсить) в различные типы данных:
* - с помощью функций вида getUInt(), getString(), ... (рекомендуется);
* - с помощью шаблонной функции get<Type>(), которая специализирована для многих типов (для шаблонного кода);
* - шаблонная функция get<Type> работает также для всех типов, у которых есть конструктор из Value
* (это сделано для возможности расширения);
* - с помощью operator Type() - но этот метод реализован лишь для совместимости и не рекомендуется
* к использованию, так как неудобен (часто возникают неоднозначности).
/** Represents a single value read from MySQL.
* It doesn't owns the value. It's just a wrapper of a pair (const char *, size_t).
* If the UseQueryResult/StoreQueryResult or Connection is destroyed,
* or you have read the next Row while using UseQueryResult, then the object is invalidated.
* Allows to transform (parse) the value to various data types:
* - with getUInt(), getString(), ... (recommended);
* - with template function get<Type>() that is specialized for multiple data types;
* - the template function get<Type> also works for all types that can be constructed from Value
* (it is an extension point);
* - with operator Type() - this is done for compatibility and not recommended because ambiguities possible.
*
* При ошибке парсинга, выкидывается исключение.
* При попытке достать значение, которое равно nullptr, выкидывается исключение
* - используйте метод isNull() для проверки.
* On parsing error, exception is thrown.
* When trying to extract a value that is nullptr, exception is thrown
* - use isNull() method to check.
*
* Во всех распространённых системах, time_t - это всего лишь typedef от Int64 или Int32.
* Для того, чтобы можно было писать row[0].get<time_t>(), ожидая, что значение вида '2011-01-01 00:00:00'
* корректно распарсится согласно текущей тайм-зоне, сделано так, что метод getUInt и соответствующие методы get<>()
* также умеют парсить дату и дату-время.
* As time_t is just an alias for integer data type
* to allow to write row[0].get<time_t>(), and expect that the values like '2011-01-01 00:00:00'
* will be successfully parsed according to the current time zone,
* the getUInt method and the corresponding get<>() methods
* are capable of parsing Date and DateTime.
*/
class Value
{
@ -166,7 +166,7 @@ private:
else
throwException("Cannot parse DateTime");
return 0; /// чтобы не было warning-а.
return 0; /// avoid warning.
}
@ -184,7 +184,7 @@ private:
else
throwException("Cannot parse Date");
return 0; /// чтобы не было warning-а.
return 0; /// avoid warning.
}
@ -231,7 +231,7 @@ private:
double readFloatText(const char * buf, size_t length) const;
/// Выкинуть исключение с подробной информацией
void throwException(const char * text) const;
[[noreturn]] void throwException(const char * text) const;
};

View File

@ -22,15 +22,20 @@ void Pool::Entry::incrementRefCount()
if (!data)
return;
++data->ref_count;
mysql_thread_init();
if (data->ref_count == 1)
mysql_thread_init();
}
void Pool::Entry::decrementRefCount()
{
if (!data)
return;
--data->ref_count;
mysql_thread_end();
if (data->ref_count > 0)
{
--data->ref_count;
if (data->ref_count == 0)
mysql_thread_end();
}
}
@ -169,14 +174,24 @@ Pool::Entry Pool::tryGet()
return Entry();
}
void Pool::removeConnection(Connection* connection)
{
std::lock_guard<std::mutex> lock(mutex);
if (connection)
{
if (connection->ref_count > 0)
{
connection->conn.disconnect();
connection->ref_count = 0;
}
connections.remove(connection);
}
}
void Pool::Entry::disconnect()
{
if (data)
{
decrementRefCount();
data->conn.disconnect();
}
pool->removeConnection(data);
}

View File

@ -0,0 +1,122 @@
#include <mysqlxx/PoolFactory.h>
#include <Poco/Util/Application.h>
#include <Poco/Util/LayeredConfiguration.h>
namespace mysqlxx
{
struct PoolFactory::Impl
{
// Cache of already affected pools identified by their config name
std::map<std::string, std::shared_ptr<PoolWithFailover>> pools;
// Cache of Pool ID (host + port + user +...) cibling already established shareable pool
std::map<std::string, std::string> pools_by_ids;
/// Protect pools and pools_by_ids caches
std::mutex mutex;
};
PoolWithFailover PoolFactory::Get(const std::string & config_name, unsigned default_connections,
unsigned max_connections, size_t max_tries)
{
return Get(Poco::Util::Application::instance().config(), config_name, default_connections, max_connections, max_tries);
}
/// Duplicate of code from StringUtils.h. Copied here for less dependencies.
static bool startsWith(const std::string & s, const char * prefix)
{
return s.size() >= strlen(prefix) && 0 == memcmp(s.data(), prefix, strlen(prefix));
}
static std::string getPoolEntryName(const Poco::Util::AbstractConfiguration & config,
const std::string & config_name)
{
bool shared = config.getBool(config_name + ".share_connection", false);
// Not shared no need to generate a name the pool won't be stored
if (!shared)
return "";
std::string entry_name = "";
std::string host = config.getString(config_name + ".host", "");
std::string port = config.getString(config_name + ".port", "");
std::string user = config.getString(config_name + ".user", "");
std::string db = config.getString(config_name + ".db", "");
std::string table = config.getString(config_name + ".table", "");
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_name, keys);
if (config.has(config_name + ".replica"))
{
Poco::Util::AbstractConfiguration::Keys replica_keys;
config.keys(config_name, replica_keys);
for (const auto & replica_config_key : replica_keys)
{
/// There could be another elements in the same level in configuration file, like "user", "port"...
if (startsWith(replica_config_key, "replica"))
{
std::string replica_name = config_name + "." + replica_config_key;
std::string tmp_host = config.getString(replica_name + ".host", host);
std::string tmp_port = config.getString(replica_name + ".port", port);
std::string tmp_user = config.getString(replica_name + ".user", user);
entry_name += (entry_name.empty() ? "" : "|") + tmp_user + "@" + tmp_host + ":" + tmp_port + "/" + db;
}
}
}
else
{
entry_name = user + "@" + host + ":" + port + "/" + db;
}
return entry_name;
}
PoolWithFailover PoolFactory::Get(const Poco::Util::AbstractConfiguration & config,
const std::string & config_name, unsigned default_connections, unsigned max_connections, size_t max_tries)
{
std::lock_guard<std::mutex> lock(impl->mutex);
if (auto entry = impl->pools.find(config_name); entry != impl->pools.end())
{
return *(entry->second.get());
}
else
{
std::string entry_name = getPoolEntryName(config, config_name);
if (auto id = impl->pools_by_ids.find(entry_name); id != impl->pools_by_ids.end())
{
entry = impl->pools.find(id->second);
std::shared_ptr<PoolWithFailover> pool = entry->second;
impl->pools.insert_or_assign(config_name, pool);
return *pool;
}
auto pool = std::make_shared<PoolWithFailover>(config, config_name, default_connections, max_connections, max_tries);
// Check the pool will be shared
if (!entry_name.empty())
{
// Store shared pool
impl->pools.insert_or_assign(config_name, pool);
impl->pools_by_ids.insert_or_assign(entry_name, config_name);
}
return *(pool.get());
}
}
void PoolFactory::reset()
{
std::lock_guard<std::mutex> lock(impl->mutex);
impl->pools.clear();
impl->pools_by_ids.clear();
}
PoolFactory::PoolFactory() : impl(std::make_unique<PoolFactory::Impl>()) {}
PoolFactory & PoolFactory::instance()
{
static PoolFactory ret;
return ret;
}
}

View File

@ -15,6 +15,7 @@ PoolWithFailover::PoolWithFailover(const Poco::Util::AbstractConfiguration & cfg
const unsigned max_connections, const size_t max_tries)
: max_tries(max_tries)
{
shareable = cfg.getBool(config_name + ".share_connection", false);
if (cfg.has(config_name + ".replica"))
{
Poco::Util::AbstractConfiguration::Keys replica_keys;
@ -48,15 +49,22 @@ PoolWithFailover::PoolWithFailover(const std::string & config_name, const unsign
{}
PoolWithFailover::PoolWithFailover(const PoolWithFailover & other)
: max_tries{other.max_tries}
: max_tries{other.max_tries}, config_name{other.config_name}, shareable{other.shareable}
{
for (const auto & priority_replicas : other.replicas_by_priority)
if (shareable)
{
Replicas replicas;
replicas.reserve(priority_replicas.second.size());
for (const auto & pool : priority_replicas.second)
replicas.emplace_back(std::make_shared<Pool>(*pool));
replicas_by_priority.emplace(priority_replicas.first, std::move(replicas));
replicas_by_priority = other.replicas_by_priority;
}
else
{
for (const auto & priority_replicas : other.replicas_by_priority)
{
Replicas replicas;
replicas.reserve(priority_replicas.second.size());
for (const auto & pool : priority_replicas.second)
replicas.emplace_back(std::make_shared<Pool>(*pool));
replicas_by_priority.emplace(priority_replicas.first, std::move(replicas));
}
}
}
@ -81,7 +89,7 @@ PoolWithFailover::Entry PoolWithFailover::Get()
try
{
Entry entry = pool->tryGet();
Entry entry = shareable ? pool->Get() : pool->tryGet();
if (!entry.isNull())
{

View File

@ -105,6 +105,7 @@ namespace ErrorCodes
extern const int UNEXPECTED_PACKET_FROM_SERVER;
extern const int CLIENT_OUTPUT_FORMAT_SPECIFIED;
extern const int INVALID_USAGE_OF_INPUT;
extern const int DEADLOCK_AVOIDED;
}
@ -906,9 +907,34 @@ private:
query = serializeAST(*parsed_query);
}
connection->sendQuery(connection_parameters.timeouts, query, query_id, QueryProcessingStage::Complete, &context.getSettingsRef(), nullptr, true);
sendExternalTables();
receiveResult();
static constexpr size_t max_retries = 10;
for (size_t retry = 0; retry < max_retries; ++retry)
{
try
{
connection->sendQuery(
connection_parameters.timeouts,
query,
query_id,
QueryProcessingStage::Complete,
&context.getSettingsRef(),
nullptr,
true);
sendExternalTables();
receiveResult();
break;
}
catch (const Exception & e)
{
/// Retry when the server said "Client should retry" and no rows has been received yet.
if (processed_rows == 0 && e.code() == ErrorCodes::DEADLOCK_AVOIDED && retry + 1 < max_retries)
continue;
throw;
}
}
}

View File

@ -74,4 +74,13 @@ const BlockMissingValues::RowsBitMask & BlockMissingValues::getDefaultsBitmask(s
return none;
}
bool BlockMissingValues::hasDefaultBits(size_t column_idx) const
{
auto it = rows_mask_by_column_id.find(column_idx);
if (it == rows_mask_by_column_id.end())
return false;
const auto & col_mask = it->second;
return std::find(col_mask.begin(), col_mask.end(), true) != col_mask.end();
}
}

View File

@ -51,7 +51,10 @@ class BlockMissingValues
public:
using RowsBitMask = std::vector<bool>; /// a bit per row for a column
/// Get mask for column, column_idx is index inside corresponding block
const RowsBitMask & getDefaultsBitmask(size_t column_idx) const;
/// Check that we have to replace default value at least in one of columns
bool hasDefaultBits(size_t column_idx) const;
void setBit(size_t column_idx, size_t row_idx);
bool empty() const { return rows_mask_by_column_id.empty(); }
size_t size() const { return rows_mask_by_column_id.size(); }

View File

@ -56,11 +56,20 @@ Block AddingDefaultsBlockInputStream::readImpl()
if (block_missing_values.empty())
return res;
/// res block alredy has all columns values, with default value for type
/// (not value specified in table). We identify which columns we need to
/// recalculate with help of block_missing_values.
Block evaluate_block{res};
/// remove columns for recalculation
for (const auto & column : column_defaults)
{
if (evaluate_block.has(column.first))
evaluate_block.erase(column.first);
{
size_t column_idx = res.getPositionByName(column.first);
if (block_missing_values.hasDefaultBits(column_idx))
evaluate_block.erase(column.first);
}
}
if (!evaluate_block.columns())
evaluate_block.insert({ColumnConst::create(ColumnUInt8::create(1, 0), res.rows()), std::make_shared<DataTypeUInt8>(), "_dummy"});

View File

@ -12,7 +12,7 @@ namespace DB
* Mostly the same as Int64.
* But also tagged with interval kind.
*
* Intended isage is for temporary elements in expressions,
* Intended usage is for temporary elements in expressions,
* not for storing values in tables.
*/
class DataTypeInterval final : public DataTypeNumberBase<Int64>

View File

@ -257,7 +257,7 @@ template class DataTypeNumberBase<UInt8>;
template class DataTypeNumberBase<UInt16>;
template class DataTypeNumberBase<UInt32>;
template class DataTypeNumberBase<UInt64>;
template class DataTypeNumberBase<UInt128>;
template class DataTypeNumberBase<UInt128>; // used only in UUID
template class DataTypeNumberBase<Int8>;
template class DataTypeNumberBase<Int16>;
template class DataTypeNumberBase<Int32>;

View File

@ -51,6 +51,7 @@ namespace
const ASTCreateQuery & query,
DatabaseOrdinary & database,
const String & database_name,
const String & metadata_path,
bool has_force_restore_data_flag)
{
assert(!query.is_dictionary);
@ -64,7 +65,9 @@ namespace
}
catch (Exception & e)
{
e.addMessage("Cannot attach table '" + backQuote(query.table) + "' from query " + serializeAST(query));
e.addMessage("Cannot attach table " + backQuote(database_name) + "." + backQuote(query.table)
+ " from metadata file " + metadata_path
+ " from query " + serializeAST(query));
throw;
}
}
@ -110,7 +113,6 @@ void DatabaseOrdinary::loadStoredObjects(
Context & context,
bool has_force_restore_data_flag)
{
/** Tables load faster if they are loaded in sorted (by name) order.
* Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order,
* which does not correspond to order tables creation and does not correspond to order of their location on disk.
@ -124,7 +126,7 @@ void DatabaseOrdinary::loadStoredObjects(
String full_path = getMetadataPath() + file_name;
try
{
auto ast = parseQueryFromMetadata(context, full_path, /*throw_on_error*/ true, /*remove_empty*/false);
auto ast = parseQueryFromMetadata(context, full_path, /*throw_on_error*/ true, /*remove_empty*/ false);
if (ast)
{
auto * create_query = ast->as<ASTCreateQuery>();
@ -157,7 +159,7 @@ void DatabaseOrdinary::loadStoredObjects(
if (!create_query.is_dictionary)
pool.scheduleOrThrowOnError([&]()
{
tryAttachTable(context, create_query, *this, getDatabaseName(), has_force_restore_data_flag);
tryAttachTable(context, create_query, *this, getDatabaseName(), getMetadataPath() + name_with_query.first, has_force_restore_data_flag);
/// Messages, so that it's not boring to wait for the server to load for a long time.
logAboutProgress(log, ++tables_processed, total_tables, watch);

View File

@ -46,6 +46,7 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
# include <common/logger_useful.h>
# include <Formats/MySQLBlockInputStream.h>
# include "readInvalidateQuery.h"
# include <mysqlxx/PoolFactory.h>
namespace DB
{
@ -66,11 +67,11 @@ MySQLDictionarySource::MySQLDictionarySource(
, update_field{config.getString(config_prefix + ".update_field", "")}
, dont_check_update_time{config.getBool(config_prefix + ".dont_check_update_time", false)}
, sample_block{sample_block_}
, pool{config, config_prefix}
, pool{mysqlxx::PoolFactory::instance().Get(config, config_prefix)}
, query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks}
, load_all_query{query_builder.composeLoadAllQuery()}
, invalidate_query{config.getString(config_prefix + ".invalidate_query", "")}
, close_connection{config.getBool(config_prefix + ".close_connection", false)}
, close_connection{config.getBool(config_prefix + ".close_connection", false) || config.getBool(config_prefix + ".share_connection", false)}
{
}
@ -114,19 +115,21 @@ std::string MySQLDictionarySource::getUpdateFieldAndDate()
BlockInputStreamPtr MySQLDictionarySource::loadAll()
{
last_modification = getLastModification();
auto connection = pool.Get();
last_modification = getLastModification(connection, false);
LOG_TRACE(log, load_all_query);
return std::make_shared<MySQLBlockInputStream>(pool.Get(), load_all_query, sample_block, max_block_size, close_connection);
return std::make_shared<MySQLBlockInputStream>(connection, load_all_query, sample_block, max_block_size, close_connection);
}
BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll()
{
last_modification = getLastModification();
auto connection = pool.Get();
last_modification = getLastModification(connection, false);
std::string load_update_query = getUpdateFieldAndDate();
LOG_TRACE(log, load_update_query);
return std::make_shared<MySQLBlockInputStream>(pool.Get(), load_update_query, sample_block, max_block_size, close_connection);
return std::make_shared<MySQLBlockInputStream>(connection, load_update_query, sample_block, max_block_size, close_connection);
}
BlockInputStreamPtr MySQLDictionarySource::loadIds(const std::vector<UInt64> & ids)
@ -158,8 +161,8 @@ bool MySQLDictionarySource::isModified() const
if (dont_check_update_time)
return true;
return getLastModification() > last_modification;
auto connection = pool.Get();
return getLastModification(connection, true) > last_modification;
}
bool MySQLDictionarySource::supportsSelectiveLoad() const
@ -199,7 +202,7 @@ std::string MySQLDictionarySource::quoteForLike(const std::string s)
return out.str();
}
LocalDateTime MySQLDictionarySource::getLastModification() const
LocalDateTime MySQLDictionarySource::getLastModification(mysqlxx::Pool::Entry & connection, bool allow_connection_closure) const
{
LocalDateTime modification_time{std::time(nullptr)};
@ -208,7 +211,6 @@ LocalDateTime MySQLDictionarySource::getLastModification() const
try
{
auto connection = pool.Get();
auto query = connection->query("SHOW TABLE STATUS LIKE " + quoteForLike(table));
LOG_TRACE(log, query.str());
@ -233,6 +235,11 @@ LocalDateTime MySQLDictionarySource::getLastModification() const
++fetched_rows;
}
if (close_connection && allow_connection_closure)
{
connection.disconnect();
}
if (0 == fetched_rows)
LOG_ERROR(log, "Cannot find table in SHOW TABLE STATUS result.");
@ -243,7 +250,6 @@ LocalDateTime MySQLDictionarySource::getLastModification() const
{
tryLogCurrentException("MySQLDictionarySource");
}
/// we suppose failure to get modification time is not an error, therefore return current time
return modification_time;
}

View File

@ -62,7 +62,7 @@ private:
static std::string quoteForLike(const std::string s);
LocalDateTime getLastModification() const;
LocalDateTime getLastModification(mysqlxx::Pool::Entry & connection, bool allow_connection_closure) const;
// execute invalidate_query. expects single cell in result
std::string doInvalidateQuery(const std::string & request) const;

View File

@ -59,6 +59,7 @@ DiskSelectorPtr DiskSelector::updateFromConfig(const Poco::Util::AbstractConfigu
std::shared_ptr<DiskSelector> result = std::make_shared<DiskSelector>(*this);
constexpr auto default_disk_name = "default";
std::set<String> old_disks_minus_new_disks;
for (const auto & [disk_name, _] : result->disks)
{
@ -84,6 +85,8 @@ DiskSelectorPtr DiskSelector::updateFromConfig(const Poco::Util::AbstractConfigu
}
}
old_disks_minus_new_disks.erase(default_disk_name);
if (!old_disks_minus_new_disks.empty())
{
WriteBufferFromOwnString warning;
@ -465,9 +468,11 @@ StoragePolicySelectorPtr StoragePolicySelector::updateFromConfig(const Poco::Uti
std::shared_ptr<StoragePolicySelector> result = std::make_shared<StoragePolicySelector>(config, config_prefix, disks);
constexpr auto default_storage_policy_name = "default";
for (const auto & [name, policy] : policies)
{
if (result->policies.count(name) == 0)
if (name != default_storage_policy_name && result->policies.count(name) == 0)
throw Exception("Storage policy " + backQuote(name) + " is missing in new configuration", ErrorCodes::BAD_ARGUMENTS);
policy->checkCompatibleWith(result->policies[name]);

View File

@ -1,5 +1,10 @@
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Dictionaries/DictionaryFactory.h>
#include "config_core.h"
#if USE_MYSQL
# include <mysqlxx/PoolFactory.h>
#endif
namespace DB
{
@ -24,4 +29,12 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create(
bool dictionary_from_database = !repository_name.empty();
return DictionaryFactory::instance().create(name, config, key_in_config, context, dictionary_from_database);
}
void ExternalDictionariesLoader::resetAll()
{
#if USE_MYSQL
mysqlxx::PoolFactory::instance().reset();
#endif
}
}

View File

@ -4,7 +4,6 @@
#include <Interpreters/ExternalLoader.h>
#include <memory>
namespace DB
{
class Context;
@ -29,6 +28,8 @@ public:
return std::static_pointer_cast<const IDictionaryBase>(tryLoad(name));
}
static void resetAll();
protected:
LoadablePtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config,
const std::string & key_in_config, const std::string & repository_name) const override;

View File

@ -62,7 +62,6 @@ namespace ErrorCodes
extern const int UNKNOWN_DATABASE_ENGINE;
extern const int DUPLICATE_COLUMN;
extern const int DATABASE_ALREADY_EXISTS;
extern const int THERE_IS_NO_DEFAULT_VALUE;
extern const int BAD_DATABASE_FOR_TEMPORARY_TABLE;
extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY;
extern const int DICTIONARY_ALREADY_EXISTS;
@ -315,15 +314,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(const ASTExpres
Block defaults_sample_block;
/// set missing types and wrap default_expression's in a conversion-function if necessary
if (!default_expr_list->children.empty())
{
auto syntax_analyzer_result = SyntaxAnalyzer(context).analyze(default_expr_list, column_names_and_types);
const auto actions = ExpressionAnalyzer(default_expr_list, syntax_analyzer_result, context).getActions(true);
for (auto & action : actions->getActions())
if (action.type == ExpressionAction::Type::JOIN || action.type == ExpressionAction::Type::ARRAY_JOIN)
throw Exception("Cannot CREATE table. Unsupported default value that requires ARRAY JOIN or JOIN action", ErrorCodes::THERE_IS_NO_DEFAULT_VALUE);
defaults_sample_block = actions->getSampleBlock();
}
defaults_sample_block = validateColumnsDefaultsAndGetSampleBlock(default_expr_list, column_names_and_types, context);
ColumnsDescription res;
auto name_type_it = column_names_and_types.begin();

View File

@ -216,6 +216,7 @@ BlockIO InterpreterSystemQuery::execute()
case Type::RELOAD_DICTIONARY:
context.checkAccess(AccessType::RELOAD_DICTIONARY);
system_context.getExternalDictionariesLoader().loadOrReload(query.target_dictionary);
ExternalDictionariesLoader::resetAll();
break;
case Type::RELOAD_DICTIONARIES:
context.checkAccess(AccessType::RELOAD_DICTIONARY);
@ -223,6 +224,7 @@ BlockIO InterpreterSystemQuery::execute()
[&] () { system_context.getExternalDictionariesLoader().reloadAllTriedToLoad(); },
[&] () { system_context.getEmbeddedDictionaries().reload(); }
);
ExternalDictionariesLoader::resetAll();
break;
case Type::RELOAD_EMBEDDED_DICTIONARIES:
context.checkAccess(AccessType::RELOAD_DICTIONARY);

View File

@ -13,6 +13,7 @@
#include <Parsers/ASTFunction.h>
#include <utility>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/RequiredSourceColumnsVisitor.h>
namespace DB
@ -32,10 +33,29 @@ ASTPtr defaultRequiredExpressions(Block & block, const NamesAndTypesList & requi
const auto it = column_defaults.find(column.name);
/// expressions must be cloned to prevent modification by the ExpressionAnalyzer
if (it != column_defaults.end())
{
auto cast_func = makeASTFunction("CAST", it->second.expression->clone(), std::make_shared<ASTLiteral>(column.type->getName()));
/// expressions must be cloned to prevent modification by the ExpressionAnalyzer
auto column_default_expr = it->second.expression->clone();
/// Our default may depend on columns with ALIAS as default expr which not present in block
/// we can easily add them from column_defaults struct
RequiredSourceColumnsVisitor::Data columns_context;
RequiredSourceColumnsVisitor(columns_context).visit(column_default_expr);
NameSet required_columns_names = columns_context.requiredColumns();
for (const auto & required_column_name : required_columns_names)
{
/// If we have such default column and it's alias than we should
/// add it into default_expression_list
if (auto rit = column_defaults.find(required_column_name);
rit != column_defaults.end() && rit->second.kind == ColumnDefaultKind::Alias)
{
default_expr_list->children.emplace_back(setAlias(rit->second.expression->clone(), required_column_name));
}
}
auto cast_func = makeASTFunction("CAST", column_default_expr, std::make_shared<ASTLiteral>(column.type->getName()));
default_expr_list->children.emplace_back(setAlias(cast_func, it->first));
}
}

View File

@ -9,6 +9,7 @@
#include <DataTypes/NestedUtils.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/addTypeConversionToAST.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Parsers/ASTAlterQuery.h>
@ -664,6 +665,8 @@ void AlterCommands::prepare(const StorageInMemoryMetadata & metadata)
void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Context & context) const
{
auto all_columns = metadata.columns;
/// Default expression for all added/modified columns
ASTPtr default_expr_list = std::make_shared<ASTExpressionList>();
for (size_t i = 0; i < size(); ++i)
{
auto & command = (*this)[i];
@ -684,9 +687,6 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con
throw Exception{"Data type have to be specified for column " + backQuote(column_name) + " to add",
ErrorCodes::BAD_ARGUMENTS};
if (command.default_expression)
validateDefaultExpressionForColumn(command.default_expression, column_name, command.data_type, all_columns, context);
all_columns.add(ColumnDescription(column_name, command.data_type, false));
}
else if (command.type == AlterCommand::MODIFY_COLUMN)
@ -699,22 +699,6 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con
else
continue;
}
auto column_in_table = metadata.columns.get(column_name);
if (command.default_expression)
{
if (!command.data_type)
validateDefaultExpressionForColumn(
command.default_expression, column_name, column_in_table.type, all_columns, context);
else
validateDefaultExpressionForColumn(
command.default_expression, column_name, command.data_type, all_columns, context);
}
else if (column_in_table.default_desc.expression && command.data_type)
{
validateDefaultExpressionForColumn(
column_in_table.default_desc.expression, column_name, command.data_type, all_columns, context);
}
}
else if (command.type == AlterCommand::DROP_COLUMN)
{
@ -756,31 +740,52 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con
if (metadata.settings_ast == nullptr)
throw Exception{"Cannot alter settings, because table engine doesn't support settings changes", ErrorCodes::BAD_ARGUMENTS};
}
}
}
void AlterCommands::validateDefaultExpressionForColumn(
const ASTPtr default_expression,
const String & column_name,
const DataTypePtr column_type,
const ColumnsDescription & all_columns,
const Context & context) const
{
/// Collect default expressions for MODIFY and ADD comands
if (command.type == AlterCommand::MODIFY_COLUMN || command.type == AlterCommand::ADD_COLUMN)
{
if (command.default_expression)
{
/// If we modify default, but not type
if (!command.data_type)
{
default_expr_list->children.emplace_back(setAlias(command.default_expression->clone(), column_name));
}
else
{
const auto & final_column_name = column_name;
const auto tmp_column_name = final_column_name + "_tmp";
const auto data_type_ptr = command.data_type;
try
{
String tmp_column_name = "__tmp" + column_name;
auto copy_expression = default_expression->clone();
auto default_with_cast = makeASTFunction("CAST", copy_expression, std::make_shared<ASTLiteral>(column_type->getName()));
auto query_with_alias = setAlias(default_with_cast, tmp_column_name);
auto syntax_result = SyntaxAnalyzer(context).analyze(query_with_alias, all_columns.getAll());
ExpressionAnalyzer(query_with_alias, syntax_result, context).getActions(true);
}
catch (Exception & ex)
{
ex.addMessage("default expression and column type are incompatible. Cannot alter column " + backQuote(column_name));
throw;
default_expr_list->children.emplace_back(setAlias(
addTypeConversionToAST(std::make_shared<ASTIdentifier>(tmp_column_name), data_type_ptr->getName()),
final_column_name));
default_expr_list->children.emplace_back(setAlias(command.default_expression->clone(), tmp_column_name));
}
} /// if we change data type for column with default
else if (metadata.columns.has(column_name) && command.data_type)
{
auto column_in_table = metadata.columns.get(column_name);
/// Column doesn't have a default, nothing to check
if (!column_in_table.default_desc.expression)
continue;
const auto & final_column_name = column_name;
const auto tmp_column_name = final_column_name + "_tmp";
const auto data_type_ptr = command.data_type;
default_expr_list->children.emplace_back(setAlias(
addTypeConversionToAST(std::make_shared<ASTIdentifier>(tmp_column_name), data_type_ptr->getName()), final_column_name));
default_expr_list->children.emplace_back(setAlias(column_in_table.default_desc.expression->clone(), tmp_column_name));
}
}
}
validateColumnsDefaultsAndGetSampleBlock(default_expr_list, all_columns.getAll(), context);
}
bool AlterCommands::isModifyingData() const

View File

@ -127,16 +127,6 @@ class AlterCommands : public std::vector<AlterCommand>
{
private:
bool prepared = false;
private:
/// Validate that default expression and type are compatible, i.e. default
/// expression result can be casted to column_type
void validateDefaultExpressionForColumn(
const ASTPtr default_expression,
const String & column_name,
const DataTypePtr column_type,
const ColumnsDescription & all_columns,
const Context & context) const;
public:
/// Validate that commands can be applied to metadata.

View File

@ -5,6 +5,9 @@
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadBuffer.h>
@ -20,7 +23,9 @@
#include <Storages/IStorage.h>
#include <Common/typeid_cast.h>
#include <Compression/CompressionFactory.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
namespace DB
{
@ -30,6 +35,7 @@ namespace ErrorCodes
extern const int NO_SUCH_COLUMN_IN_TABLE;
extern const int ILLEGAL_COLUMN;
extern const int CANNOT_PARSE_TEXT;
extern const int THERE_IS_NO_DEFAULT_VALUE;
}
ColumnDescription::ColumnDescription(String name_, DataTypePtr type_, bool is_virtual_)
@ -421,4 +427,28 @@ ColumnsDescription ColumnsDescription::parse(const String & str)
return result;
}
Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const NamesAndTypesList & all_columns, const Context & context)
{
for (const auto & child : default_expr_list->children)
if (child->as<ASTSelectQuery>() || child->as<ASTSelectWithUnionQuery>() || child->as<ASTSubquery>())
throw Exception("Select query is not allowed in columns DEFAULT expression", ErrorCodes::THERE_IS_NO_DEFAULT_VALUE);
try
{
auto syntax_analyzer_result = SyntaxAnalyzer(context).analyze(default_expr_list, all_columns);
const auto actions = ExpressionAnalyzer(default_expr_list, syntax_analyzer_result, context).getActions(true);
for (auto & action : actions->getActions())
if (action.type == ExpressionAction::Type::JOIN || action.type == ExpressionAction::Type::ARRAY_JOIN)
throw Exception("Unsupported default value that requires ARRAY JOIN or JOIN action", ErrorCodes::THERE_IS_NO_DEFAULT_VALUE);
return actions->getSampleBlock();
}
catch (Exception & ex)
{
ex.addMessage("default expression and column type are incompatible.");
throw;
}
}
}

View File

@ -114,4 +114,9 @@ private:
Container columns;
};
/// Validate default expressions and corresponding types compatibility, i.e.
/// default expression result can be casted to column_type. Also checks, that we
/// don't have strange constructions in default expression like SELECT query or
/// arrayJoin function.
Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const NamesAndTypesList & all_columns, const Context & context);
}

View File

@ -0,0 +1,491 @@
#include <Storages/IStorage.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/StorageGenerate.h>
#include <Storages/StorageFactory.h>
#include <DataStreams/OneBlockInputStream.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <Processors/Pipe.h>
#include <Parsers/ASTLiteral.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeDecimalBase.h>
#include <DataTypes/DataTypeArray.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnTuple.h>
#include <Common/SipHash.h>
#include <Common/randomSeed.h>
#include <pcg_random.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int LOGICAL_ERROR;
extern const int BAD_TYPE_OF_FIELD;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
void fillColumnWithRandomData(IColumn & column, const DataTypePtr type, UInt64 limit,
UInt64 max_array_length, UInt64 max_string_length, pcg32 & generator, pcg64_fast & generator64)
{
TypeIndex idx = type->getTypeId();
switch (idx)
{
case TypeIndex::Nothing:
throw Exception("Random Generator not implemented for type 'Nothing'.", ErrorCodes::NOT_IMPLEMENTED);
case TypeIndex::UInt8:
{
auto & data = typeid_cast<ColumnVector<UInt8> &>(column).getData();
data.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
data[i] = static_cast<UInt8>(generator());
}
break;
}
case TypeIndex::UInt16:
{
auto & data = typeid_cast<ColumnVector<UInt16> &>(column).getData();
data.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
data[i] = static_cast<UInt16>(generator());
}
break;
}
case TypeIndex::UInt32:
{
auto & data = typeid_cast<ColumnVector<UInt32> &>(column).getData();
data.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
data[i] = static_cast<UInt32>(generator());
}
break;
}
case TypeIndex::UInt64:
{
auto & data = typeid_cast<ColumnVector<UInt64> &>(column).getData();
data.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
UInt64 a = static_cast<UInt64>(generator64());
data[i] = static_cast<UInt64>(a);
}
break;
}
case TypeIndex::UInt128:
throw Exception("There is no DataType 'UInt128' support.", ErrorCodes::NOT_IMPLEMENTED);
case TypeIndex::Int8:
{
auto & data = typeid_cast<ColumnVector<Int8> &>(column).getData();
data.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
data[i] = static_cast<Int8>(generator());
}
break;
}
case TypeIndex::Int16:
{
auto & data = typeid_cast<ColumnVector<Int16> &>(column).getData();
data.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
data[i] = static_cast<Int16>(generator());
}
break;
}
case TypeIndex::Int32:
{
auto & data = typeid_cast<ColumnVector<Int32> &>(column).getData();
data.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
data[i] = static_cast<Int32>(generator());
}
break;
}
case TypeIndex::Int64:
{
auto & data = typeid_cast<ColumnVector<Int64> &>(column).getData();
data.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
data[i] = static_cast<Int64>(generator64());
}
break;
}
case TypeIndex::Int128:
throw Exception("There is no DataType 'Int128' support.", ErrorCodes::NOT_IMPLEMENTED);
case TypeIndex::Float32:
{
auto & data = typeid_cast<ColumnVector<Float32> &>(column).getData();
data.resize(limit);
double d = 1.0;
for (UInt64 i = 0; i < limit; ++i)
{
d = std::numeric_limits<float>::max();
data[i] = (d / pcg32::max()) * generator();
}
break;
}
case TypeIndex::Float64:
{
auto & data = typeid_cast<ColumnVector<Float64> &>(column).getData();
data.resize(limit);
double d = 1.0;
for (UInt64 i = 0; i < limit; ++i)
{
d = std::numeric_limits<double>::max();
data[i] = (d / pcg64::max()) * generator64();
}
break;
}
case TypeIndex::Date:
{
auto & data = typeid_cast<ColumnVector<UInt16> &>(column).getData();
data.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
data[i] = static_cast<UInt16>(generator());
}
break;
}
case TypeIndex::DateTime:
{
auto & data = typeid_cast<ColumnVector<UInt32> &>(column).getData();
data.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
data[i] = static_cast<UInt32>(generator());
}
break;
}
case TypeIndex::DateTime64:
{
UInt32 scale;
if (auto * ptype = typeid_cast<const DataTypeDateTime64 *>(type.get()))
scale = ptype->getScale();
else
throw Exception("Static cast to DataTypeDateTime64 failed ", ErrorCodes::BAD_TYPE_OF_FIELD);
auto & data = typeid_cast<ColumnDecimal<Decimal64> &>(column).getData();
data.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
UInt32 fractional = static_cast<UInt32>(generator()) % intExp10(scale);
UInt32 whole = static_cast<UInt32>(generator());
DateTime64 dt = DecimalUtils::decimalFromComponents<DateTime64>(whole, fractional, scale);
data[i] = dt;
}
break;
}
case TypeIndex::String:
{
auto & column_string = typeid_cast<ColumnString &>(column);
auto & offsets = column_string.getOffsets();
auto & chars = column_string.getChars();
UInt64 offset = 0;
{
offsets.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
offset += 1 + static_cast<UInt64>(generator()) % max_string_length;
offsets[i] = offset;
}
chars.resize(offset);
for (UInt64 i = 0; i < offset; ++i)
{
if (offset - i > 5)
{
UInt32 r = generator();
chars[i] = 32 + (r & 0x7F) % 95;
chars[i + 1] = 32 + ((r >> 7) & 0x7F) % 95;
chars[i + 2] = 32 + ((r >> 14) & 0x7F) % 95;
chars[i + 3] = 32 + ((r >> 21) & 0x7F) % 95;
chars[i + 4] = 32 + (r >> 28);
i += 4;
}
else
{
UInt32 r = generator();
chars[i] = 32 + (r % 95);
}
}
// add terminating zero char
for (auto & i : offsets)
{
chars[i - 1] = 0;
}
}
break;
}
case TypeIndex::FixedString:
{
auto & column_string = typeid_cast<ColumnFixedString &>(column);
const size_t len = column_string.sizeOfValueIfFixed();
auto & chars = column_string.getChars();
UInt64 num_chars = static_cast<UInt64>(len) * limit;
{
chars.resize(num_chars);
for (UInt64 i = 0; i < num_chars; ++i)
{
chars[i] = static_cast<UInt8>(generator());
}
}
break;
}
case TypeIndex::Enum8:
{
auto values = typeid_cast<const DataTypeEnum<Int8> *>(type.get())->getValues();
auto & data = typeid_cast<ColumnVector<Int8> &>(column).getData();
data.resize(limit);
UInt8 size = values.size();
UInt8 off;
for (UInt64 i = 0; i < limit; ++i)
{
off = static_cast<UInt8>(generator()) % size;
data[i] = values[off].second;
}
break;
}
case TypeIndex::Enum16:
{
auto values = typeid_cast<const DataTypeEnum<Int16> *>(type.get())->getValues();
auto & data = typeid_cast<ColumnVector<Int16> &>(column).getData();
data.resize(limit);
UInt16 size = values.size();
UInt8 off;
for (UInt64 i = 0; i < limit; ++i)
{
off = static_cast<UInt16>(generator()) % size;
data[i] = values[off].second;
}
break;
}
case TypeIndex::Decimal32:
{
auto & data = typeid_cast<ColumnDecimal<Decimal32> &>(column).getData();
data.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
data[i] = static_cast<Int32>(generator());
}
break;
}
case TypeIndex::Decimal64:
{
auto & data = typeid_cast<ColumnDecimal<Decimal64> &>(column).getData();
data.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
UInt64 a = static_cast<UInt64>(generator()) << 32 | static_cast<UInt64>(generator());
data[i] = a;
}
break;
}
case TypeIndex::Decimal128:
{
auto & data = typeid_cast<ColumnDecimal<Decimal128> &>(column).getData();
data.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
Int128 x = static_cast<Int128>(generator64()) << 64 | static_cast<Int128>(generator64());
data[i] = x;
}
break;
}
case TypeIndex::UUID:
{
auto & data = typeid_cast<ColumnVector<UInt128> &>(column).getData();
data.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
UInt64 a = static_cast<UInt64>(generator64());
UInt64 b = static_cast<UInt64>(generator64());
auto x = UInt128(a, b);
data[i] = x;
}
break;
}
case TypeIndex::Array:
{
auto & column_array = typeid_cast<ColumnArray &>(column);
auto nested_type = typeid_cast<const DataTypeArray *>(type.get())->getNestedType();
auto & offsets = column_array.getOffsets();
IColumn & data = column_array.getData();
UInt64 offset = 0;
{
offsets.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
offset += static_cast<UInt64>(generator()) % max_array_length;
offsets[i] = offset;
}
}
fillColumnWithRandomData(data, nested_type, offset, max_array_length, max_string_length, generator, generator64);
break;
}
case TypeIndex::Tuple:
{
auto &column_tuple = typeid_cast<ColumnTuple &>(column);
auto elements = typeid_cast<const DataTypeTuple *>(type.get())->getElements();
for (size_t i = 0; i < column_tuple.tupleSize(); ++i)
{
fillColumnWithRandomData(column_tuple.getColumn(i), elements[i], limit, max_array_length, max_string_length, generator, generator64);
}
break;
}
case TypeIndex::Set:
throw Exception("Type 'Set' can not be stored in a table.", ErrorCodes::LOGICAL_ERROR);
case TypeIndex::Interval:
throw Exception("Type 'Interval' can not be stored in a table.", ErrorCodes::LOGICAL_ERROR);
case TypeIndex::Nullable:
{
auto & column_nullable = typeid_cast<ColumnNullable &>(column);
auto nested_type = typeid_cast<const DataTypeNullable *>(type.get())->getNestedType();
auto & null_map = column_nullable.getNullMapData();
IColumn & nested_column = column_nullable.getNestedColumn();
fillColumnWithRandomData(nested_column, nested_type, limit, max_array_length, max_string_length, generator, generator64);
null_map.resize(limit);
for (UInt64 i = 0; i < limit; ++i)
{
null_map[i] = generator() < 1024; /// No real motivation for this.
}
break;
}
case TypeIndex::Function:
throw Exception("Type 'Function' can not be stored in a table.", ErrorCodes::LOGICAL_ERROR);
case TypeIndex::AggregateFunction:
throw Exception("Random Generator not implemented for type 'AggregateFunction'.", ErrorCodes::NOT_IMPLEMENTED);
case TypeIndex::LowCardinality:
throw Exception("Random Generator not implemented for type 'LowCardinality'.", ErrorCodes::NOT_IMPLEMENTED);
}
}
StorageGenerate::StorageGenerate(const StorageID & table_id_, const ColumnsDescription & columns_,
UInt64 max_array_length_, UInt64 max_string_length_, UInt64 random_seed_)
: IStorage(table_id_), max_array_length(max_array_length_), max_string_length(max_string_length_)
{
random_seed = random_seed_ ? random_seed_ : randomSeed();
setColumns(columns_);
}
class GenerateSource : public SourceWithProgress
{
public:
GenerateSource(UInt64 block_size_, UInt64 max_array_length_, UInt64 max_string_length_, UInt64 random_seed_, Block block_header_)
: SourceWithProgress(block_header_), block_size(block_size_), max_array_length(max_array_length_), max_string_length(max_string_length_)
, block_header(block_header_), r32(random_seed_), r64(random_seed_) {}
String getName() const override { return "Generate"; }
protected:
Chunk generate() override
{
auto columns = block_header.cloneEmptyColumns();
DataTypes types = block_header.getDataTypes();
auto cur_type = types.cbegin();
for (auto & col : columns)
{
fillColumnWithRandomData(col->assumeMutableRef(), *cur_type, block_size, max_array_length, max_string_length, r32, r64);
++cur_type;
}
return {std::move(columns), block_size};
}
private:
UInt64 block_size;
UInt64 max_array_length;
UInt64 max_string_length;
Block block_header;
pcg32 r32;
pcg64_fast r64;
};
void registerStorageGenerate(StorageFactory & factory)
{
factory.registerStorage("Generate", [](const StorageFactory::Arguments & args)
{
ASTs & engine_args = args.engine_args;
if (engine_args.size() > 3)
throw Exception("Storage Generate requires at most three arguments: "\
"max_array_length, max_string_length, random_seed.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
UInt64 max_array_length_ = 10;
UInt64 max_string_length_ = 10;
UInt64 random_seed_ = 0; // zero for random
/// Parsing second argument if present
if (engine_args.size() >= 1)
max_array_length_ = engine_args[0]->as<ASTLiteral &>().value.safeGet<UInt64>();
if (engine_args.size() >= 2)
max_string_length_ = engine_args[1]->as<ASTLiteral &>().value.safeGet<UInt64>();
if (engine_args.size() == 3)
random_seed_ = engine_args[2]->as<ASTLiteral &>().value.safeGet<UInt64>();
return StorageGenerate::create(args.table_id, args.columns, max_array_length_, max_string_length_, random_seed_);
});
}
Pipes StorageGenerate::read(
const Names & column_names,
const SelectQueryInfo & /*query_info*/,
const Context & /*context*/,
QueryProcessingStage::Enum /*processed_stage*/,
size_t max_block_size,
unsigned num_streams)
{
check(column_names, true);
Pipes pipes;
pipes.reserve(num_streams);
const ColumnsDescription & columns_ = getColumns();
Block block_header;
for (const auto & name : column_names)
{
const auto & name_type = columns_.get(name);
MutableColumnPtr column = name_type.type->createColumn();
block_header.insert({std::move(column), name_type.type, name_type.name});
}
pcg32 generate(random_seed);
for (UInt64 i = 0; i < num_streams; ++i)
{
pipes.emplace_back(std::make_shared<GenerateSource>(max_block_size, max_array_length, max_string_length, generate(), block_header));
}
return pipes;
}
}

View File

@ -0,0 +1,35 @@
#pragma once
#include <ext/shared_ptr_helper.h>
#include <Storages/IStorage.h>
namespace DB
{
/* Generates random data for given schema.
*/
class StorageGenerate : public ext::shared_ptr_helper<StorageGenerate>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageGenerate>;
public:
std::string getName() const override { return "Generate"; }
Pipes read(
const Names & column_names,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
unsigned num_streams) override;
private:
UInt64 max_array_length = 10;
UInt64 max_string_length = 10;
UInt64 random_seed = 0;
protected:
StorageGenerate(const StorageID & table_id_, const ColumnsDescription & columns_,
UInt64 max_array_length, UInt64 max_string_length, UInt64 random_seed);
};
}

View File

@ -29,8 +29,9 @@ void registerStorages()
registerStorageView(factory);
registerStorageMaterializedView(factory);
registerStorageLiveView(factory);
registerStorageGenerate(factory);
#if USE_AWS_S3
#if USE_AWS_S3
registerStorageS3(factory);
#endif

View File

@ -23,6 +23,7 @@ void registerStorageJoin(StorageFactory & factory);
void registerStorageView(StorageFactory & factory);
void registerStorageMaterializedView(StorageFactory & factory);
void registerStorageLiveView(StorageFactory & factory);
void registerStorageGenerate(StorageFactory & factory);
#if USE_AWS_S3
void registerStorageS3(StorageFactory & factory);

View File

@ -0,0 +1,78 @@
#include <Common/typeid_cast.h>
#include <Common/Exception.h>
#include <Core/Block.h>
#include <Storages/StorageGenerate.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <TableFunctions/ITableFunction.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/TableFunctionGenerate.h>
#include <TableFunctions/parseColumnsListForTableFunction.h>
#include "registerTableFunctions.h"
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int LOGICAL_ERROR;
}
StoragePtr TableFunctionGenerate::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const
{
ASTs & args_func = ast_function->children;
if (args_func.size() != 1)
throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::LOGICAL_ERROR);
ASTs & args = args_func.at(0)->children;
if (args.size() < 1)
throw Exception("Table function '" + getName() + "' requires at least one argument: "
" structure(, max_array_length, max_string_length, random_seed).",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (args.size() > 4)
throw Exception("Table function '" + getName() + "' requires at most four arguments: "
" structure, max_array_length, max_string_length, random_seed.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
/// Parsing first argument as table structure and creating a sample block
std::string structure = args[0]->as<ASTLiteral &>().value.safeGet<String>();
UInt64 max_array_length = 10;
UInt64 max_string_length = 10;
UInt64 random_seed = 0; // zero for random
/// Parsing second argument if present
if (args.size() >= 2)
max_array_length = args[1]->as<ASTLiteral &>().value.safeGet<UInt64>();
if (args.size() >= 3)
max_string_length = args[2]->as<ASTLiteral &>().value.safeGet<UInt64>();
if (args.size() == 4)
random_seed = args[3]->as<ASTLiteral &>().value.safeGet<UInt64>();
ColumnsDescription columns = parseColumnsListFromString(structure, context);
auto res = StorageGenerate::create(StorageID(getDatabaseName(), table_name), columns, max_array_length, max_string_length, random_seed);
res->startup();
return res;
}
void registerTableFunctionGenerate(TableFunctionFactory & factory)
{
factory.registerFunction<TableFunctionGenerate>(TableFunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,19 @@
#pragma once
#include <TableFunctions/ITableFunction.h>
namespace DB
{
/* generate(structure, [max_array_length, max_string_length, random_seed]) - creates a temporary storage that generates columns with random data
*/
class TableFunctionGenerate : public ITableFunction
{
public:
static constexpr auto name = "generate";
std::string getName() const override { return name; }
private:
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override;
};
}

View File

@ -15,6 +15,7 @@ void registerTableFunctions()
registerTableFunctionURL(factory);
registerTableFunctionValues(factory);
registerTableFunctionInput(factory);
registerTableFunctionGenerate(factory);
#if USE_AWS_S3
registerTableFunctionS3(factory);

View File

@ -12,6 +12,7 @@ void registerTableFunctionFile(TableFunctionFactory & factory);
void registerTableFunctionURL(TableFunctionFactory & factory);
void registerTableFunctionValues(TableFunctionFactory & factory);
void registerTableFunctionInput(TableFunctionFactory & factory);
void registerTableFunctionGenerate(TableFunctionFactory & factory);
#if USE_AWS_S3
void registerTableFunctionS3(TableFunctionFactory & factory);

View File

@ -0,0 +1,30 @@
<?xml version="1.0"?>
<yandex>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
<size>1000M</size>
<count>10</count>
</logger>
<tcp_port>9000</tcp_port>
<listen_host>127.0.0.1</listen_host>
<openSSL>
<client>
<cacheSessions>true</cacheSessions>
<verificationMode>none</verificationMode>
<invalidCertificateHandler>
<name>AcceptCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
<max_concurrent_queries>500</max_concurrent_queries>
<mark_cache_size>5368709120</mark_cache_size>
<path>./clickhouse/</path>
<users_config>users.xml</users_config>
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
</yandex>

View File

@ -0,0 +1,75 @@
<?xml version="1.0"?>
<yandex>
<dictionary>
<name>dict0</name>
<source>
<mysql >
<db>test</db>
<host>mysql1</host>
<port>3306</port>
<user>root</user>
<password>clickhouse</password>
<table>test0</table>
<close_connection>true</close_connection>
<share_connection>true</share_connection>
</mysql>
</source>
<layout>
<hashed/>
</layout>
<structure>
<id>
<name>id</name>
<type>UInt32</type>
<expression>CAST(id AS UNSIGNED)</expression>
</id>
<attribute>
<name>id</name>
<type>Int32</type>
<null_value></null_value>
</attribute>
<attribute>
<name>value</name>
<type>String</type>
<null_value>(UNDEFINED)</null_value>
</attribute>
</structure>
<lifetime>0</lifetime>
</dictionary>
<dictionary>
<name>dict1</name>
<source>
<mysql >
<db>test</db>
<host>mysql1</host>
<port>3306</port>
<user>root</user>
<password>clickhouse</password>
<table>test1</table>
<close_connection>true</close_connection>
<share_connection>true</share_connection>
</mysql>
</source>
<layout>
<hashed/>
</layout>
<structure>
<id>
<name>id</name>
<type>UInt32</type>
<expression>CAST(id AS UNSIGNED)</expression>
</id>
<attribute>
<name>id</name>
<type>Int32</type>
<null_value></null_value>
</attribute>
<attribute>
<name>value</name>
<type>String</type>
<null_value>(UNDEFINED)</null_value>
</attribute>
</structure>
<lifetime>0</lifetime>
</dictionary>
</yandex>

View File

@ -0,0 +1,113 @@
<?xml version="1.0"?>
<yandex>
<dictionary>
<name>dict2</name>
<source>
<mysql >
<db>test</db>
<host>mysql1</host>
<port>3306</port>
<user>root</user>
<password>clickhouse</password>
<table>test2</table>
<close_connection>true</close_connection>
<share_connection>true</share_connection>
</mysql>
</source>
<layout>
<hashed/>
</layout>
<structure>
<id>
<name>id</name>
<type>UInt32</type>
<expression>CAST(id AS UNSIGNED)</expression>
</id>
<attribute>
<name>id</name>
<type>Int32</type>
<null_value></null_value>
</attribute>
<attribute>
<name>value</name>
<type>String</type>
<null_value>(UNDEFINED)</null_value>
</attribute>
</structure>
<lifetime>0</lifetime>
</dictionary>
<dictionary>
<name>dict3</name>
<source>
<mysql >
<db>test</db>
<host>mysql1</host>
<port>3306</port>
<user>root</user>
<password>clickhouse</password>
<table>test3</table>
<close_connection>true</close_connection>
<share_connection>true</share_connection>
</mysql>
</source>
<layout>
<hashed/>
</layout>
<structure>
<id>
<name>id</name>
<type>UInt32</type>
<expression>CAST(id AS UNSIGNED)</expression>
</id>
<attribute>
<name>id</name>
<type>Int32</type>
<null_value></null_value>
</attribute>
<attribute>
<name>value</name>
<type>String</type>
<null_value>(UNDEFINED)</null_value>
</attribute>
</structure>
<lifetime>0</lifetime>
</dictionary>
<dictionary>
<name>dict4</name>
<source>
<mysql >
<db>test</db>
<host>mysql1</host>
<port>3306</port>
<user>root</user>
<password>clickhouse</password>
<table>test4</table>
<close_connection>true</close_connection>
<share_connection>true</share_connection>
</mysql>
</source>
<layout>
<hashed/>
</layout>
<structure>
<id>
<name>id</name>
<type>UInt32</type>
<expression>CAST(id AS UNSIGNED)</expression>
</id>
<attribute>
<name>id</name>
<type>Int32</type>
<null_value></null_value>
</attribute>
<attribute>
<name>value</name>
<type>String</type>
<null_value>(UNDEFINED)</null_value>
</attribute>
</structure>
<lifetime>0</lifetime>
</dictionary>
</yandex>

View File

@ -0,0 +1,12 @@
<yandex>
<remote_servers>
<test_cluster>
<shard>
<replica>
<host>instance</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
</yandex>

View File

@ -0,0 +1,23 @@
<?xml version="1.0"?>
<yandex>
<profiles>
<default>
</default>
</profiles>
<users>
<default>
<password></password>
<networks incl="networks" replace="replace">
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
</default>
</users>
<quotas>
<default>
</default>
</quotas>
</yandex>

View File

@ -0,0 +1,95 @@
import pytest
import os
import time
## sudo -H pip install PyMySQL
import pymysql.cursors
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
CONFIG_FILES = ['configs/dictionaries/mysql_dict1.xml', 'configs/dictionaries/mysql_dict2.xml', 'configs/remote_servers.xml']
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
instance = cluster.add_instance('instance', main_configs=CONFIG_FILES, with_mysql = True)
create_table_mysql_template = """
CREATE TABLE IF NOT EXISTS `test`.`{}` (
`id` int(11) NOT NULL,
`value` varchar(50) NOT NULL,
PRIMARY KEY (`id`)
) ENGINE=InnoDB;
"""
create_clickhouse_dictionary_table_template = """
CREATE TABLE IF NOT EXISTS `test`.`dict_table_{}` (`id` Int32, `value` String) ENGINE = Dictionary({})
"""
@pytest.fixture(scope="module")
def started_cluster():
try:
#time.sleep(30)
cluster.start()
# Create a MySQL database
mysql_connection = get_mysql_conn()
create_mysql_db(mysql_connection, 'test')
mysql_connection.close()
# Create database in ClickHouse
instance.query("CREATE DATABASE IF NOT EXISTS test")
# Create database in ClickChouse using MySQL protocol (will be used for data insertion)
instance.query("CREATE DATABASE clickhouse_mysql ENGINE = MySQL('mysql1:3306', 'test', 'root', 'clickhouse')")
yield cluster
finally:
cluster.shutdown()
def test_load_mysql_dictionaries(started_cluster):
# Load dictionaries
query = instance.query
query("SYSTEM RELOAD DICTIONARIES")
for n in range(0, 5):
# Create MySQL tables, fill them and create CH dict tables
prepare_mysql_table('test', str(n))
# Check dictionaries are loaded and have correct number of elements
for n in range(0, 100):
# Force reload of dictionaries (each 10 iteration)
if (n % 10) == 0:
query("SYSTEM RELOAD DICTIONARIES")
# Check number of row
assert query("SELECT count() FROM `test`.`dict_table_{}`".format('test' + str(n % 5))).rstrip() == '10000'
def create_mysql_db(mysql_connection, name):
with mysql_connection.cursor() as cursor:
cursor.execute("CREATE DATABASE IF NOT EXISTS {} DEFAULT CHARACTER SET 'utf8'".format(name))
def prepare_mysql_table(table_name, index):
mysql_connection = get_mysql_conn()
# Create table
create_mysql_table(mysql_connection, table_name + str(index))
# Insert rows using CH
query = instance.query
query("INSERT INTO `clickhouse_mysql`.{}(id, value) select number, concat('{} value ', toString(number)) from numbers(10000) ".format(table_name + str(index), table_name + str(index)))
assert query("SELECT count() FROM `clickhouse_mysql`.{}".format(table_name + str(index))).rstrip() == '10000'
mysql_connection.close()
#Create CH Dictionary tables based on MySQL tables
query(create_clickhouse_dictionary_table_template.format(table_name + str(index), 'dict' + str(index)))
def get_mysql_conn():
conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.10', port=3308)
return conn
def create_mysql_table(conn, table_name):
with conn.cursor() as cursor:
cursor.execute(create_table_mysql_template.format(table_name))

View File

@ -0,0 +1,30 @@
<test>
<type>loop</type>
<stop_conditions>
<any_of>
<average_speed_not_changing_for_ms>4000</average_speed_not_changing_for_ms>
<total_time_ms>10000</total_time_ms>
</any_of>
</stop_conditions>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('ui64 UInt64, i64 Int64, ui32 UInt32, i32 Int32, ui16 UInt16, i16 Int16, ui8 UInt8, i8 Int8') LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('ui64 UInt64, i64 Int64, ui32 UInt32, i32 Int32, ui16 UInt16, i16 Int16, ui8 UInt8, i8 Int8', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Enum8(\'hello\' = 1, \'world\' = 5)', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Array(Nullable(Enum8(\'hello\' = 1, \'world\' = 5)))', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Nullable(Enum16(\'h\' = 1, \'w\' = 5 , \'o\' = -200)))', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('d Date, dt DateTime, dtm DateTime(\'Europe/Moscow\')', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('dt64 DateTime64, dts64 DateTime64(6), dtms64 DateTime64(6 ,\'Europe/Moscow\')', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('f32 Float32, f64 Float64', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('d32 Decimal32(4), d64 Decimal64(8), d128 Decimal128(16)', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Tuple(Int32, Int64)', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Array(Int8)', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Array(Nullable(Int32))', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Tuple(Int32, Array(Int64))', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Nullable(String)', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Array(String)', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i UUID', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Array(Nullable(UUID))', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i FixedString(4)', 10, 10, 1) LIMIT 100000);</query>
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i String', 10, 10, 1) LIMIT 100000);</query>
</test>

View File

@ -5,7 +5,6 @@ col3 UInt64 MATERIALIZED col1 + 2
col4 UInt64 ALIAS col1 + 3
10 11
12 13
99
payload String
date Date MATERIALIZED today()
key UInt64 MATERIALIZED 0 * rand()

View File

@ -11,10 +11,7 @@ select * from defaulted;
select col3, col4 from defaulted;
drop table defaulted;
create table defaulted (col1 Int8, col2 UInt64 default (SELECT dummy+99 from system.one)) engine=Memory;
insert into defaulted (col1) values (0);
select col2 from defaulted;
drop table defaulted;
create table defaulted (col1 Int8, col2 UInt64 default (SELECT dummy+99 from system.one)) engine=Memory; --{serverError 116}
create table defaulted (payload String, date materialized today(), key materialized 0 * rand()) engine=MergeTree(date, key, 8192);
desc table defaulted;

View File

@ -48,14 +48,8 @@ echo
echo
${CLICKHOUSE_LOCAL} -q "CREATE TABLE sophisticated_default
(
a UInt8 DEFAULT
(
SELECT number FROM system.numbers LIMIT 3,1
),
b UInt8 ALIAS
(
SELECT dummy+9 FROM system.one
),
a UInt8 DEFAULT 3,
b UInt8 ALIAS a + 5,
c UInt8
) ENGINE = Memory; SELECT count() FROM system.tables WHERE name='sophisticated_default';"

View File

@ -63,7 +63,6 @@ $CLICKHOUSE_CLIENT $settings -q "
SELECT
-- max(thread_realtime), $query_elapsed, max(thread_time_user_system_io),
0.9 * $query_elapsed <= max(thread_realtime) AND max(thread_realtime) <= 1.1 * $query_elapsed,
0.7 * $query_elapsed <= max(thread_time_user_system_io) AND max(thread_time_user_system_io) <= 1.3 * $query_elapsed,
uniqExact(thread_id) = $threads
FROM
(

View File

@ -1,65 +1,68 @@
SET send_logs_level = 'none';
SET replication_alter_partitions_sync = 2;
DROP TABLE IF EXISTS test.alter_compression_codec1;
DROP TABLE IF EXISTS test.alter_compression_codec2;
DROP TABLE IF EXISTS alter_compression_codec1;
DROP TABLE IF EXISTS alter_compression_codec2;
CREATE TABLE test.alter_compression_codec1 (
CREATE TABLE alter_compression_codec1 (
somedate Date CODEC(LZ4),
id UInt64 CODEC(NONE)
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/alter_compression_codecs', '1') PARTITION BY somedate ORDER BY id;
CREATE TABLE test.alter_compression_codec2 (
CREATE TABLE alter_compression_codec2 (
somedate Date CODEC(LZ4),
id UInt64 CODEC(NONE)
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/alter_compression_codecs', '2') PARTITION BY somedate ORDER BY id;
INSERT INTO test.alter_compression_codec1 VALUES('2018-01-01', 1);
INSERT INTO test.alter_compression_codec1 VALUES('2018-01-01', 2);
SYSTEM SYNC REPLICA test.alter_compression_codec2;
INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 1);
INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 2);
SYSTEM SYNC REPLICA alter_compression_codec2;
SELECT * FROM test.alter_compression_codec1 ORDER BY id;
SELECT * FROM test.alter_compression_codec2 ORDER BY id;
SELECT * FROM alter_compression_codec1 ORDER BY id;
SELECT * FROM alter_compression_codec2 ORDER BY id;
ALTER TABLE test.alter_compression_codec1 ADD COLUMN alter_column String DEFAULT 'default_value' CODEC(ZSTD);
SYSTEM SYNC REPLICA test.alter_compression_codec2;
ALTER TABLE alter_compression_codec1 ADD COLUMN alter_column String DEFAULT 'default_value' CODEC(ZSTD);
SYSTEM SYNC REPLICA alter_compression_codec1;
SYSTEM SYNC REPLICA alter_compression_codec2;
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'alter_compression_codec1' AND name = 'alter_column';
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'alter_compression_codec2' AND name = 'alter_column';
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column';
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column';
INSERT INTO test.alter_compression_codec1 VALUES('2018-01-01', 3, '3');
INSERT INTO test.alter_compression_codec1 VALUES('2018-01-01', 4, '4');
SYSTEM SYNC REPLICA test.alter_compression_codec2;
INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 3, '3');
INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 4, '4');
SYSTEM SYNC REPLICA alter_compression_codec1;
SYSTEM SYNC REPLICA alter_compression_codec2;
SELECT * FROM test.alter_compression_codec1 ORDER BY id;
SELECT * FROM test.alter_compression_codec2 ORDER BY id;
SELECT * FROM alter_compression_codec1 ORDER BY id;
SELECT * FROM alter_compression_codec2 ORDER BY id;
ALTER TABLE test.alter_compression_codec1 MODIFY COLUMN alter_column CODEC(NONE);
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'alter_compression_codec1' AND name = 'alter_column';
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'alter_compression_codec2' AND name = 'alter_column';
ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column CODEC(NONE);
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column';
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column';
INSERT INTO test.alter_compression_codec2 VALUES('2018-01-01', 5, '5');
INSERT INTO test.alter_compression_codec2 VALUES('2018-01-01', 6, '6');
SYSTEM SYNC REPLICA test.alter_compression_codec1;
SELECT * FROM test.alter_compression_codec1 ORDER BY id;
SELECT * FROM test.alter_compression_codec2 ORDER BY id;
INSERT INTO alter_compression_codec2 VALUES('2018-01-01', 5, '5');
INSERT INTO alter_compression_codec2 VALUES('2018-01-01', 6, '6');
SYSTEM SYNC REPLICA alter_compression_codec1;
SELECT * FROM alter_compression_codec1 ORDER BY id;
SELECT * FROM alter_compression_codec2 ORDER BY id;
ALTER TABLE test.alter_compression_codec1 MODIFY COLUMN alter_column CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE);
SYSTEM SYNC REPLICA test.alter_compression_codec1;
SYSTEM SYNC REPLICA test.alter_compression_codec2;
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'alter_compression_codec1' AND name = 'alter_column';
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'alter_compression_codec2' AND name = 'alter_column';
ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column CODEC(ZSTD, LZ4HC, LZ4, LZ4, NONE);
SYSTEM SYNC REPLICA alter_compression_codec1;
SYSTEM SYNC REPLICA alter_compression_codec2;
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column';
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column';
INSERT INTO test.alter_compression_codec1 VALUES('2018-01-01', 7, '7');
INSERT INTO test.alter_compression_codec2 VALUES('2018-01-01', 8, '8');
SYSTEM SYNC REPLICA test.alter_compression_codec2;
SYSTEM SYNC REPLICA test.alter_compression_codec1;
SELECT * FROM test.alter_compression_codec1 ORDER BY id;
SELECT * FROM test.alter_compression_codec2 ORDER BY id;
INSERT INTO alter_compression_codec1 VALUES('2018-01-01', 7, '7');
INSERT INTO alter_compression_codec2 VALUES('2018-01-01', 8, '8');
SYSTEM SYNC REPLICA alter_compression_codec2;
SYSTEM SYNC REPLICA alter_compression_codec1;
SELECT * FROM alter_compression_codec1 ORDER BY id;
SELECT * FROM alter_compression_codec2 ORDER BY id;
ALTER TABLE test.alter_compression_codec1 MODIFY COLUMN alter_column FixedString(100);
SYSTEM SYNC REPLICA test.alter_compression_codec2;
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'alter_compression_codec1' AND name = 'alter_column';
SELECT compression_codec FROM system.columns WHERE database = 'test' AND table = 'alter_compression_codec2' AND name = 'alter_column';
ALTER TABLE alter_compression_codec1 MODIFY COLUMN alter_column FixedString(100);
SYSTEM SYNC REPLICA alter_compression_codec2;
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec1' AND name = 'alter_column';
SELECT compression_codec FROM system.columns WHERE table = 'alter_compression_codec2' AND name = 'alter_column';
DROP TABLE IF EXISTS test.alter_compression_codec1;
DROP TABLE IF EXISTS test.alter_compression_codec2;
DROP TABLE IF EXISTS alter_compression_codec1;
DROP TABLE IF EXISTS alter_compression_codec2;

View File

@ -0,0 +1,6 @@
1 1
1 1 1
2 2 4
2 2 2 4
3 3 9
3 3 3 9 27

View File

@ -0,0 +1,30 @@
DROP TABLE IF EXISTS table_with_defaults_on_aliases;
CREATE TABLE table_with_defaults_on_aliases (col1 UInt32, col2 ALIAS col1, col3 DEFAULT col2) Engine = MergeTree() ORDER BY tuple();
INSERT INTO table_with_defaults_on_aliases (col1) VALUES (1);
SELECT * FROM table_with_defaults_on_aliases WHERE col1 = 1;
SELECT col1, col2, col3 FROM table_with_defaults_on_aliases WHERE col1 = 1;
ALTER TABLE table_with_defaults_on_aliases ADD COLUMN col4 UInt64 DEFAULT col2 * col3;
INSERT INTO table_with_defaults_on_aliases (col1) VALUES (2);
SELECT * FROM table_with_defaults_on_aliases WHERE col1 = 2;
SELECT col1, col2, col3, col4 FROM table_with_defaults_on_aliases WHERE col1 = 2;
ALTER TABLE table_with_defaults_on_aliases ADD COLUMN col5 UInt64 ALIAS col2 * col4;
INSERT INTO table_with_defaults_on_aliases (col1) VALUES (3);
SELECT * FROM table_with_defaults_on_aliases WHERE col1 = 3;
SELECT col1, col2, col3, col4, col5 FROM table_with_defaults_on_aliases WHERE col1 = 3;
ALTER TABLE table_with_defaults_on_aliases ADD COLUMN col6 UInt64 MATERIALIZED col2 * col4;
DROP TABLE IF EXISTS table_with_defaults_on_aliases;

View File

@ -0,0 +1,103 @@
100
-
[] -183162.1041 ('2074-01-15 19:36:42.413','0345f8ad-8936-8cc9-9ff2-394f225fc318')
[3] -39049.0845 ('2045-07-04 15:01:09.380','f79d47d8-4030-9916-54b9-495a5ccc7202')
[-95,-104] -14324.1685 ('2023-10-02 06:05:53.887','ea94157b-737b-a272-acd5-c7ab9c6f07c3')
[] -182420.0194 ('2002-01-08 01:42:50.396','ac579c0f-8523-144f-aa4c-c9587cc27144')
[] 127157.2100 ('2006-11-23 14:25:39.542','d07c5204-ef95-6804-83df-01dedaf32522')
[28] -198600.4267 ('2060-09-25 02:57:41.504','b13ff007-c245-d737-85b2-1fa003e57127')
[] -87232.0739 ('2027-05-12 20:26:59.405','a2f2cbf4-b11b-6976-7b91-14b6964acbe2')
[] -110349.8987 ('2042-11-01 10:51:30.039','445a77b5-0a27-3485-8dd8-c7cc35d2692f')
[32,73] 123253.5669 ('2016-06-21 00:23:02.917','c12095e6-b82c-d81c-4629-acd80e02b080')
[-109,85] 34320.8302 ('2080-12-31 16:49:32.509','ebbbe70a-0321-ff18-89de-2bc9a9e4c454')
[68,76] 160458.5593 ('2030-05-23 03:33:29.681','805b0a62-9ada-a47e-2d5e-63cb5923549c')
[69] -189322.2887 ('2084-06-19 03:31:58.508','429df3a4-ff18-28d5-9ad8-dcdd78e8b1ae')
[] 189158.4731 ('1985-12-21 16:36:51.092','d63c5cbb-9418-ce59-000c-056f88157bfa')
[-120] 114890.5905 ('2089-02-19 22:30:18.216','2bc4860a-7214-300a-851e-b61011c346ef')
[] 54493.1631 ('2062-08-18 04:57:01.348','c00d218a-913f-b657-1ff9-99927741f7ab')
[103] 88891.6006 ('2071-05-23 21:46:45.644','036d2746-f7aa-b5a4-b716-b8e8a5e041da')
[28] -41173.9863 ('2044-03-11 10:06:47.659','6bf54ef0-9bad-54d4-5ca3-02d79883b697')
[] -64809.0652 ('2010-11-02 23:46:46.150','ae5cafb4-fe3c-71a5-9a76-0314c44180de')
[125,74] 28139.7661 ('1989-07-19 22:10:13.477','ee9b8173-4426-8615-97eb-a904266847e5')
[-101] -29032.2527 ('2052-08-13 08:55:15.045','ab8ee358-ff53-de7e-f012-cd0eed754ff2')
[82] 170334.6151 ('2034-02-23 18:50:46.847','f1402791-1d23-e56d-25f0-5a51a3cb245a')
[27,56] 168648.5067 ('2004-05-31 19:47:19.448','5019365e-f74d-b31e-aacb-63d8189e3e3e')
[-74] 89577.4738 ('1997-01-15 09:05:57.691','d144325c-24a9-411e-cc10-78b0637f75a7')
[100] 178532.5772 ('2055-04-22 16:20:11.498','41439c27-fba1-1b66-13a7-cf79fded4d9a')
[32] 123101.7871 ('2103-02-18 15:53:42.748','25b804d3-a73e-ed14-e6e1-eafb0d9473cd')
[-115,-85] -208371.1662 ('2039-10-22 18:06:50.235','41fba85a-5080-48bb-e18d-a8af04a890aa')
[-12,101] -7791.5577 ('2073-07-27 11:35:45.239','c00be55f-59ac-762c-af0a-9e33cf30a1f4')
[-127] -18602.1918 ('2024-02-05 19:54:00.798','ea85cbed-66f2-197b-4e63-dfbdcd306cce')
[-78,125] -112158.3556 ('2016-08-12 06:46:17.173','15809e25-b003-010b-c63c-9e880568736a')
[] 151191.1081 ('1982-11-12 17:59:10.171','d6bbbe2c-fca0-53c8-22a6-de9e0715d3cc')
[46] 58689.9611 ('2085-04-13 09:13:07.230','fe4be382-eb78-4cf9-fa57-c6eccf955419')
[-4,42] -88292.1046 ('1980-03-06 08:29:12.503','2633522e-ff9c-b837-1b9b-6559875c13b0')
[-2] 64983.6649 ('2034-07-07 11:20:23.903','d19f5c4d-e444-2e5b-a55d-5280d1760b94')
[-31,-110] -25638.6649 ('2025-05-17 21:45:25.519','3654a15e-bfa3-6075-b5b8-07e25310de1f')
[25] -140469.2476 ('2083-12-13 23:55:25.450','940f7441-ae40-d810-f6c3-e2fff468050c')
[-99] 128186.7318 ('1995-09-15 04:26:33.803','4cfd264f-ff00-4190-929c-b675826607d3')
[] -52961.0340 ('2046-03-19 14:15:50.245','314de821-308b-c61c-e256-9f6afed5d4f3')
[] -155852.9334 ('2052-01-29 06:31:08.957','5be2ccd2-b5bb-921d-5b5e-4a0e22385de7')
[-74,81] 29366.0091 ('1978-03-05 19:24:49.193','ab9b6a39-89ac-9280-c76f-60d598ce65c6')
[9] 56134.8951 ('2104-04-03 10:27:33.053','339a7f6d-0e0b-e039-78c0-2d045457d821')
[-61] 68841.1188 ('2059-07-26 12:14:33.197','c817bbb3-d091-b73c-1b9a-53f8a03bffb6')
[89] 168034.0459 ('2028-05-29 08:02:02.393','bc217a73-e802-1772-80b5-d8c827295799')
[124] 11648.6762 ('2084-12-25 12:10:35.676','77390177-1dd6-a5c0-dd35-4f85e38bcb2c')
[-47,-125] -120893.6705 ('2012-10-18 22:52:57.524','472814b2-4033-c5a5-7d86-fb36079e88fb')
[35] 153250.6252 ('2006-11-06 00:05:25.456','de0d6ed9-eca6-e01e-eb1c-c46c8ad6e33e')
[-43,70] -141086.3184 ('2013-02-03 23:07:11.759','65d48b24-cdc0-f7db-cb16-d0ad03279bcc')
[120,-57] -93351.1404 ('2000-02-03 14:39:00.466','6991722b-90dc-e9dd-c5e7-f28bd1d4f0d8')
[34,43] 187780.4567 ('2014-02-10 05:22:19.250','3db77bc5-d877-b22e-6667-955bf36d2e08')
[73] -90148.5697 ('2014-10-05 18:34:31.419','5a0f919e-38c9-0a68-e805-977db04d0acb')
[] -179121.0029 ('2077-01-23 07:57:55.365','fcf79336-a6dc-44fd-8c78-7e74e07b60fa')
[-69,120] 119321.8003 ('1989-07-01 13:11:35.185','92f6a362-250c-cfcd-acd7-99399cbf88ad')
[] 208864.8324 ('1991-02-17 03:04:00.682','b0dc8e88-ea6f-c2da-c116-3e4873dc8d54')
[22,-14] -127735.4391 ('2036-08-10 08:33:03.806','5ab1ab2b-913d-ff8a-6f8f-86387e77ed5c')
[83,-70] -142476.9847 ('2074-11-22 19:27:13.085','51b9d30a-3b10-265c-4086-1ac35b634ec7')
[] -128052.2443 ('2088-01-02 10:58:36.999','745e8226-d906-7fb3-33f4-9a079037bdcd')
[12,-116] -88390.1399 ('2074-02-18 17:46:45.208','fb5f827e-1809-6cab-2855-d45df20ecd92')
[] -84110.2097 ('2039-03-24 17:08:15.660','88e18c93-6276-d176-dad1-7db72e340ca7')
[] 202866.8175 ('2104-01-25 13:42:41.758','10faa33e-d383-c6b3-399d-44c06ebb00f5')
[-21] 151775.1601 ('1995-10-20 15:44:53.296','7ccaf135-787d-2ac0-09c0-7545c798ee14')
[-19] -15498.5738 ('2097-08-02 18:34:16.406','cf97f268-02c0-24fc-bbf3-c7b272632c14')
[116] -72670.9713 ('2020-08-31 18:10:41.904','f9cdd931-e2ed-0584-d4b9-67a6df717a4c')
[] 124014.7040 ('1975-07-23 11:17:25.176','ccf33ba5-8fd8-c8b5-ccc4-a9cb892d4b55')
[-56] -204745.8115 ('2037-11-13 01:03:12.923','6dc83c7b-7782-57b4-a293-18ca8aba331d')
[] -28535.2534 ('2105-04-07 20:51:09.990','0d9f3a2f-d4f2-a330-7b6e-001ea3aacbde')
[-124,-128] -31519.7583 ('1993-02-14 23:06:10.338','a073dafb-6f1f-273e-acf9-88200f82af6d')
[46] -154950.9257 ('2032-06-04 23:16:16.051','e6aa3b80-9f53-6c10-0cc8-622622f964b4')
[] 206914.3454 ('2003-10-05 10:44:30.786','137ed3be-2d40-d1c1-7aff-b32f7e21c0da')
[-47] 91521.1349 ('2006-09-01 04:06:32.496','52e4ef43-9379-4864-8f63-8e205875a096')
[121] 161456.7813 ('2027-11-03 10:20:30.670','f1abbd17-f399-657c-1a47-1dd627578b53')
[99] -127959.4741 ('2084-08-18 06:04:41.942','2a3b92c3-75ed-bd20-5a77-b77cbe1ce479')
[-97] 82020.4570 ('2061-10-25 06:16:50.814','8625d479-6e81-318f-5077-a9deb13c50e0')
[71] -121599.1388 ('2010-04-02 11:05:18.877','0ec279cf-c9b2-dc65-40c0-2d0f390b1102')
[] 98975.6469 ('2049-03-06 08:56:25.010','845340d7-a1df-9ddf-b737-9eb90ca6344c')
[92,81] 135864.7854 ('2040-12-30 21:17:28.184','ea224755-198e-c9ae-c59b-0517a7459d7c')
[81] -154620.5037 ('1984-06-07 02:36:28.734','52d3b727-043f-1d43-6f48-51e8abdc2127')
[38] 33379.3375 ('2057-10-19 17:03:44.317','e709bfc2-0915-9e4e-4d01-c10b24795e30')
[] 7491.1071 ('1971-04-29 09:30:25.245','26bcd2ab-6d0b-fc20-27eb-084c4248af7d')
[-122] -135635.3813 ('2010-03-04 23:05:25.982','66ed96eb-fc6e-653e-0353-ac4477ea60a6')
[] -174748.4115 ('2020-10-28 07:39:33.461','e17fa9ba-2595-c0f9-2f85-d6bbdc2f6f6a')
[72,106] 25749.2190 ('2008-06-15 04:03:39.682','0e47b616-da80-091e-664d-2a35bc57a480')
[-84,97] 109277.9244 ('1998-10-27 10:40:00.442','9488bce4-46d7-8249-78aa-540b8be43937')
[-120,-107] -64113.5210 ('2091-12-03 06:46:11.903','325fcb1c-8552-b434-b349-732d62be19f1')
[] -66141.6000 ('2085-10-05 08:08:11.830','4c66022b-75b9-b0a8-3897-b9de8ea851f1')
[-34,-102] -142314.4437 ('2038-01-27 12:04:29.739','91e9eb11-5679-02ef-6ea6-2c9fdcb12ed9')
[103] 96187.7213 ('1978-10-07 13:57:43.616','7c02e8e3-9e98-5043-8029-34e32ad1af61')
[] -21344.8423 ('2085-01-13 00:10:52.538','52cb36f8-987a-f414-7e0f-93ddccc5c377')
[16] -95098.4107 ('2074-02-19 18:56:00.878','821e4b10-f70a-4bee-ef0c-ac12eab994f3')
[21,86] 27954.7748 ('2033-10-18 03:15:38.815','bfe4d932-c5ed-45c0-9f50-72a6394d49af')
[] 149788.2085 ('2073-09-10 20:42:48.693','5e7d825e-5c88-7c89-4235-0e7934739a12')
[33,116] -148302.8732 ('2044-08-10 22:05:18.943','a53d4b07-5529-7472-3cca-3770f52b3648')
[] -98384.4505 ('2070-01-28 05:17:35.804','4833b839-51a3-87b8-7709-30676f697aa4')
[] -75597.1523 ('2075-02-04 19:24:01.477','d64becff-5c08-b0a0-e7f1-b86eaf5f1913')
[] 179005.6113 ('2100-05-27 21:54:12.965','d87ce81c-c471-b6b3-93b7-05225cb577be')
[] -134366.9213 ('2054-11-16 18:19:00.801','c348fced-6700-f0f6-cda0-14aef7ea6948')
[10] 82182.0343 ('2017-03-04 09:41:21.249','e19f0022-49ab-2d41-872d-be35669a79bc')
[-28] 90333.8564 ('2032-11-19 01:23:37.107','e2586be2-e968-21d0-d1b1-b438c55a59a3')
[-73] 185647.6735 ('2001-01-23 16:20:26.442','24b04f39-f272-24ff-538d-41e636a1a37a')
[-79,7] -87628.8007 ('2005-03-25 04:17:49.969','38a10e9d-7086-f358-8e50-c72b278bec42')
[119,-55] -208591.8591 ('1976-11-14 15:17:57.569','d0935dc7-7f56-71db-67f2-1b4e52770ba9')
[-108,-124] 181408.0349 ('2056-10-27 05:07:32.393','29d655c1-c35a-1245-25e2-65b4f233cb9c')
-

View File

@ -0,0 +1,17 @@
DROP TABLE IF EXISTS test_table;
CREATE TABLE test_table(a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)) ENGINE=Generate();
SELECT COUNT(*) FROM (SELECT * FROM test_table LIMIT 100);
DROP TABLE IF EXISTS test_table;
SELECT '-';
DROP TABLE IF EXISTS test_table_2;
CREATE TABLE test_table_2(a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)) ENGINE=Generate(3, 5, 10);
SELECT * FROM test_table_2 LIMIT 100;
SELECT '-';
DROP TABLE IF EXISTS test_table_2;

View File

@ -0,0 +1,238 @@
UInt64 Int64 UInt32 Int32 UInt16 Int16 UInt8 Int8
5443401583997919274 956654340036924402 2956613447 2041372187 46025 26509 247 -34
14051730854243326159 340055300607421421 579798001 915264595 58925 22498 36 -57
12126660396637528292 -9182366379883086416 535113873 -1583603936 45790 6066 230 91
5198178071978083704 -3549936112074464250 3354362520 -1732019372 41330 -27737 13 -47
9045663333607591872 -5069075924065328373 741246230 -1830932765 29642 -11720 41 7
18192666371709191624 -5005976579831091773 671021725 1851158245 38613 -27838 57 3
4333039311970693040 -7294587049092886539 2106347821 2101852759 24058 9107 85 94
1398111012802844853 1131449717368086026 1687614855 -1193084417 9803 -18141 198 115
15838944643191192696 6226099517671026657 1300309956 468322781 17216 -2375 184 -102
15170414162889419078 3337938833953948518 3603117877 -1297530274 25534 8264 36 16
-
Enum8(\'hello\' = 1, \'world\' = 5)
world
world
world
hello
hello
world
world
world
hello
world
-
Array(Nullable(Enum8(\'hello\' = 1, \'world\' = 5)))
['world','world','hello','hello','world','world','world']
['world']
['world','hello','world']
[]
[]
['world','hello','hello','hello','world']
['hello']
['world','hello','hello','world','hello']
['hello','world','hello','hello','world','world']
['world','hello','world','hello','hello','world','world']
-
Nullable(Enum16(\'o\' = -200, \'h\' = 1, \'w\' = 5))
w
h
h
o
w
w
o
w
h
o
-
Date DateTime DateTime(\'Europe/Moscow\')
2031-03-05 2034-09-09 02:49:47 2061-06-26 03:46:01
1972-10-06 1999-01-02 11:09:55 2064-03-18 05:47:09
2004-01-16 2055-12-02 15:29:20 2090-08-18 23:04:46
2061-07-14 2051-03-20 20:58:44 1973-04-20 21:20:34
2063-04-13 2048-01-31 01:02:11 2051-02-07 03:11:54
2106-02-07 2028-08-29 13:37:25 2054-10-20 03:48:21
2026-11-24 2036-08-09 02:59:19 2065-10-12 06:39:38
2106-02-07 2068-04-17 13:07:59 2101-04-03 08:48:59
1997-11-15 1984-11-03 12:39:41 1998-04-01 17:38:08
2008-09-11 2064-12-25 16:23:42 2031-10-18 03:20:14
-
DateTime64(3) DateTime64(6) DateTime64(6, \'Europe/Moscow\')
1988-05-16 19:00:01.447 2064-03-18 05:47:09.972361 2104-06-20 09:26:44.845879
2076-04-17 18:22:00.873 1973-04-20 21:20:34.769886 2052-08-01 07:14:05.921510
1991-04-07 13:55:25.230 2054-10-20 03:48:21.341514 2013-02-07 18:37:45.437737
2023-06-24 16:54:15.821 2101-04-03 08:48:59.544378 2039-07-05 08:51:02.770005
2084-03-05 21:04:37.956 2031-10-18 03:20:14.437888 2076-03-16 14:08:20.993528
1999-01-02 11:09:55.187 2054-01-01 16:49:22.580109 1997-01-09 20:11:35.889758
2051-03-20 20:58:44.360 1975-02-11 06:38:15.042546 2015-10-21 23:47:13.191963
2028-08-29 13:37:25.531 1975-02-14 07:25:38.319928 2103-09-16 20:57:23.033927
2068-04-17 13:07:59.759 2024-03-06 21:42:43.711891 2045-04-22 19:38:11.140126
2064-12-25 16:23:42.781 2025-08-18 15:44:56.149625 2093-09-26 16:30:56.744858
-
Float32 Float64
2.3424705e38 5.304765772621186e307
4.5936326e37 1.3693852957827914e308
4.2396088e37 1.1817811347484115e308
2.6575997e38 5.065787759860024e307
5.8727575e37 8.815282962741328e307
5.3163816e37 1.7729324649694315e308
1.6688205e38 4.2226828718895e307
1.3370661e38 1.3625030842560206e307
1.0302116e38 1.5435548915708008e308
2.8546838e38 1.4784044970034722e308
-
Decimal32(4) Decimal64(8) Decimal64(8)
-133835.3849 87676267830.44260947 10041303591043480341650.6377217747572943
57979.8001 -68015271123.73929132 -11658496611537681782723.8256877955807880
53511.3873 -78637963449.98695195 16686303649199763212696.4854950355256776
-94060.4776 90273888640.14252543 7993046724924589483272.0796323974797493
74124.6230 20114310313.64207198 -4810540869033768101015.4448286464595642
67102.1725 -60472921957.85611731 1764715777766465744700.9237855716355053
210634.7821 -20967919098.37725326 -16938476260073815366594.8118263905360890
168761.4855 -74544559691.08355371 -9350794626143586522954.2962771754340925
130030.9956 -54650148153.48939189 -13456138041801265081736.4812607484010998
-69184.9419 38286965773.25360062 11485126437992390872631.7990315807376230
-
Tuple(Int32, Int64)
(-1338353849,5443401583997919274)
(579798001,-4395013219466225457)
(535113873,-6320083677072023324)
(-940604776,5198178071978083704)
(741246230,9045663333607591872)
(671021725,-254077702000359992)
(2106347821,4333039311970693040)
(1687614855,1398111012802844853)
(1300309956,-2607799430518358920)
(-691849419,-3276329910820132538)
-
Array(Int8)
[27,83,32,84,-29,-27,87]
[-1]
[-35,94,-55]
[]
[]
[45,-34,114,-54,-43]
[-6]
[75,64,-66,-115,-30]
[-78,-89,56,66,-109,35]
[-71,72,-9,36,-26,13,41]
-
Array(Nullable(Int32))
[2041372187,915264595,-1583603936,-1732019372,-1830932765,1851158245,2101852759]
[-1193084417]
[468322781,-1297530274,-1407994935]
[]
[]
[-1321933267,-488197410,104178034,-1735625782,-1618897195]
[-1272422918]
[-153016757,891437888,1950049214,6580109,-1644079134]
[790042546,161321895,1074319928,161583938,515711891,1709750563]
[-149817671,1755521096,815845879,-51580892,1361921510,-1688868851,-1185529559]
-
Tuple(Int32, Array(Int64))
(-1338353849,[5443401583997919274,-4395013219466225457,-6320083677072023324,5198178071978083704,9045663333607591872,-254077702000359992,4333039311970693040])
(579798001,[1398111012802844853,-2607799430518358920,-3276329910820132538,956654340036924402,340055300607421421])
(535113873,[])
(-940604776,[-9182366379883086416,-3549936112074464250,-5069075924065328373,-5005976579831091773])
(741246230,[-7294587049092886539])
(671021725,[1131449717368086026,6226099517671026657,3337938833953948518,-104956130729581604,515805789944032293])
(2106347821,[2731028582309582302,-8197314279937271385,7439592879615992239,-8726726222408049230,-4046170041070917399,-8162695179087422573,7147712321550951494,-2473105312361834401,2871941532606538254])
(1687614855,[7045950974355040215,8128475529675984757,3862453874713979777,8584893221699499395,-4344095019439049735,7221768832555831190,5712009283210486481,8657278465574644253,-4620821897447975309])
(1300309956,[-3580736586972265629])
(-691849419,[7980379733974797651,-548434416689229144])
-
Nullable(String)
;\\Sm\'sH
T
@^1
7-f)$
9
)&}y3
w5>+\'@
+g+N^g$
-
Array(String)
['Y9n(%ub','\\\'f%7','','X0O@','D','4^,~q','a(gmt6#{X']
['@#q4?Q%\'.']
['h','#B','{']
[]
[]
['~vYP/4f9.',':,7u.0',';e.<','^O,i','3']
['!y1/Z\'5D']
['&- KDN%>[','>-xM./ B','?+//','M,.71QR#_','~N']
['z9P/%m','7q\'!k','Q%] #.*3','U:&XeP{*',',','s.3']
['+k.=%','8\'nb=P','-uY ,h8(w','=\'W$','','m<+%l','<~+@ Vw']
-
UUID
4b8ad8e6-77fe-a02a-c301-c6c33e91d8cf
a84a8d61-8560-a0e4-4823-a36a537a8578
7d88a6e0-3d74-cfc0-fc79-55ad1aebc5c8
3c220c4b-fbd0-efb0-1367-168f02acd8b5
dbcf3c3c-127c-bc78-d288-234e0a942946
0d46b80d-bebc-93f2-04b8-1e7e84b1abed
8091ae6d-8194-3db0-cebc-17ea18786406
b9a7064d-de99-e30b-ba87-32d2cd3e2dc3
9ac46bba-2ba7-4ff5-0fb3-b785f4f0de0a
56678c86-2703-2fe1-2e52-bdaf2fce8366
-
Array(Nullable(UUID))
['4b8ad8e6-77fe-a02a-c301-c6c33e91d8cf','a84a8d61-8560-a0e4-4823-a36a537a8578','7d88a6e0-3d74-cfc0-fc79-55ad1aebc5c8','3c220c4b-fbd0-efb0-1367-168f02acd8b5','dbcf3c3c-127c-bc78-d288-234e0a942946','0d46b80d-bebc-93f2-04b8-1e7e84b1abed','8091ae6d-8194-3db0-cebc-17ea18786406']
['b9a7064d-de99-e30b-ba87-32d2cd3e2dc3']
['9ac46bba-2ba7-4ff5-0fb3-b785f4f0de0a','56678c86-2703-2fe1-2e52-bdaf2fce8366','fe8b1ef4-86dd-23dc-0728-82a212b42c25']
[]
[]
['25e69006-a800-55de-8e3d-4a17f81a19a7','673ebe4e-af09-61af-86e4-70bca5481db2','c7d91dc9-0123-e8e9-8eb8-47fb80b35b93','6331c67c-7aba-2446-ddad-c3d24c8a985f','27db2f9b-92b4-220e-61c8-3f14833fe7d7']
['70ce256d-7fca-cf75-359a-3155bba86b81']
['7723aae6-820e-b583-c3b6-ac9887cab3f9','6438e07d-9a7a-4f96-4f45-222df77ea2d1','7824d4e5-6e37-2a1d-bfdf-8af768445673','ce4eab0b-b346-0363-6ec0-0116104b4d53','f86391cd-c2a3-d2a8-ea85-27ed3508504c']
['b87d39f3-e56c-7128-7a66-4e516e2ce1c0','af25bac3-d662-673a-3516-022e687643ed','657c9997-3b29-f51c-7193-6a3b9a0c18eb','19bb38b5-6f97-a81c-2d56-57f189119a1a','0c7e416d-c669-dc04-1130-ff950fbbf44b','9705bc44-8d08-c734-6f47-8edcc1608a81']
['7c20103f-659a-f845-399c-abdc8dc88ba0','bf479e85-1a0f-66c3-66fa-f6029e4ee2a8','f14af1a0-823c-b414-eb8e-e6b05b019868','ce353e45-2a9e-492c-1c54-d50459160ecf','1e232279-77ad-db7e-82f6-b4b3e30cdc2e','991111af-30a3-1ff7-e15a-023dfa0a8a6e','a749ef39-dc02-d05c-e8b2-129a7cccfd24']
-
FixedString(4)
˜
<16>-‡
Ä5S
Tãå
WÿÝ^
É-Þr
ÊÕúK
<>â
²§8B
“#¹H
-
String
;\\Sm\'sH
T
@^1
7-f)$
9
)&}y3
w5>+\'@
+g+N^g$
-
[27] -119308.4417 ('1998-04-01 17:38:08.539','4b8ad8e6-77fe-a02a-c301-c6c33e91d8cf')
[83] 46832.2781 ('1970-03-18 06:48:29.214','a84a8d61-8560-a0e4-4823-a36a537a8578')
[32] -129753.0274 ('1995-01-14 03:15:46.162','7d88a6e0-3d74-cfc0-fc79-55ad1aebc5c8')
[] -140799.4935 ('2004-01-17 09:12:08.895','3c220c4b-fbd0-efb0-1367-168f02acd8b5')
[] -132193.3267 ('1986-05-06 01:18:11.938','dbcf3c3c-127c-bc78-d288-234e0a942946')
[84] -48819.7410 ('2101-05-10 09:27:05.563','0d46b80d-bebc-93f2-04b8-1e7e84b1abed')
[-29] 10417.8034 ('1995-11-08 18:51:19.096','8091ae6d-8194-3db0-cebc-17ea18786406')
[-27] -173562.5782 ('2013-02-27 03:31:50.404','b9a7064d-de99-e30b-ba87-32d2cd3e2dc3')
[] -161889.7195 ('2068-07-13 23:42:17.445','9ac46bba-2ba7-4ff5-0fb3-b785f4f0de0a')
[87] -127242.2918 ('2033-08-04 15:06:45.865','56678c86-2703-2fe1-2e52-bdaf2fce8366')
-
[] 3608695403 ZL 109414.2847 h 2.2986075276244747e306 ('1985-05-10','2009-10-28 20:06:11','1993-01-03 17:51:52.981','b13ff007-c245-d737-85b2-1fa003e57127') .ü
[85] 4204173796 ], -199466.5471 h 1.1231803213254798e308 ('2075-04-03','1983-02-12 23:57:05','2060-06-06 20:15:08.751','a2f2cbf4-b11b-6976-7b91-14b6964acbe2') *þ
[-94,100] 32713522 8D$ 102255.5602 h 1.738807291208415e308 ('2029-07-12','2056-08-07 23:18:32','2081-01-25 13:13:30.589','445a77b5-0a27-3485-8dd8-c7cc35d2692f') á’
[] 4117557956 0b>+ 65942.4942 w 5.949505844751135e307 ('2048-03-05','2074-01-22 02:32:44','2073-12-04 05:05:06.955','c12095e6-b82c-d81c-4629-acd80e02b080') <19>
[] 1511604199 Il= -96352.6064 o 1.6472659147355216e308 ('2024-06-01','2024-12-26 00:54:40','2038-04-14 05:21:44.387','ebbbe70a-0321-ff18-89de-2bc9a9e4c454') ¶Q
[-18] 2278197196 ~ 193977.7666 o 1.213689191969361e308 ('2060-10-04','1992-10-24 16:31:53','1983-06-10 08:51:48.294','805b0a62-9ada-a47e-2d5e-63cb5923549c') \tÃ
[] 3761265784 N"(6 -59230.0369 o 1.2102282609858645e308 ('2106-02-07','2060-07-09 20:14:59','2007-03-17 04:51:09.288','429df3a4-ff18-28d5-9ad8-dcdd78e8b1ae') Y|
[] 66606254 6x&+ 130635.2269 o 1.1958868988757417e308 ('2088-10-07','2070-03-01 21:30:45','1978-05-22 14:28:52.523','d63c5cbb-9418-ce59-000c-056f88157bfa') у
[-27,-12] 4089193163 )+.8 -111081.7896 o 1.464035857434812e308 ('2106-02-07','2007-04-27 23:04:36','1987-07-21 04:32:01.821','2bc4860a-7214-300a-851e-b61011c346ef') #Õ
[14,-43] 3638976325 #" 116961.4294 o 9.260305126207595e307 ('2042-06-11','2087-12-28 00:21:16','2071-04-01 21:44:13.058','c00d218a-913f-b657-1ff9-99927741f7ab') Fx
-

View File

@ -0,0 +1,190 @@
SELECT
toTypeName(ui64), toTypeName(i64),
toTypeName(ui32), toTypeName(i32),
toTypeName(ui16), toTypeName(i16),
toTypeName(ui8), toTypeName(i8)
FROM generate('ui64 UInt64, i64 Int64, ui32 UInt32, i32 Int32, ui16 UInt16, i16 Int16, ui8 UInt8, i8 Int8')
LIMIT 1;
SELECT
ui64, i64,
ui32, i32,
ui16, i16,
ui8, i8
FROM generate('ui64 UInt64, i64 Int64, ui32 UInt32, i32 Int32, ui16 UInt16, i16 Int16, ui8 UInt8, i8 Int8', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(i)
FROM generate('i Enum8(\'hello\' = 1, \'world\' = 5)')
LIMIT 1;
SELECT
i
FROM generate('i Enum8(\'hello\' = 1, \'world\' = 5)', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(i)
FROM generate('i Array(Nullable(Enum8(\'hello\' = 1, \'world\' = 5)))')
LIMIT 1;
SELECT
i
FROM generate('i Array(Nullable(Enum8(\'hello\' = 1, \'world\' = 5)))', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(i)s
FROM generate('i Nullable(Enum16(\'h\' = 1, \'w\' = 5 , \'o\' = -200)))')
LIMIT 1;
SELECT
i
FROM generate('i Nullable(Enum16(\'h\' = 1, \'w\' = 5 , \'o\' = -200)))', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(d), toTypeName(dt), toTypeName(dtm)
FROM generate('d Date, dt DateTime, dtm DateTime(\'Europe/Moscow\')')
LIMIT 1;
SELECT
d, dt, dtm
FROM generate('d Date, dt DateTime, dtm DateTime(\'Europe/Moscow\')', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(dt64), toTypeName(dts64), toTypeName(dtms64)
FROM generate('dt64 DateTime64, dts64 DateTime64(6), dtms64 DateTime64(6 ,\'Europe/Moscow\')')
LIMIT 1;
SELECT
dt64, dts64, dtms64
FROM generate('dt64 DateTime64, dts64 DateTime64(6), dtms64 DateTime64(6 ,\'Europe/Moscow\')', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(f32), toTypeName(f64)
FROM generate('f32 Float32, f64 Float64')
LIMIT 1;
SELECT
f32, f64
FROM generate('f32 Float32, f64 Float64', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(d32), toTypeName(d64), toTypeName(d64)
FROM generate('d32 Decimal32(4), d64 Decimal64(8), d128 Decimal128(16)')
LIMIT 1;
SELECT
d32, d64, d128
FROM generate('d32 Decimal32(4), d64 Decimal64(8), d128 Decimal128(16)', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(i)
FROM generate('i Tuple(Int32, Int64)')
LIMIT 1;
SELECT
i
FROM generate('i Tuple(Int32, Int64)', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(i)
FROM generate('i Array(Int8)')
LIMIT 1;
SELECT
i
FROM generate('i Array(Int8)', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(i)
FROM generate('i Array(Nullable(Int32))')
LIMIT 1;
SELECT
i
FROM generate('i Array(Nullable(Int32))', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(i)
FROM generate('i Tuple(Int32, Array(Int64))')
LIMIT 1;
SELECT
i
FROM generate('i Tuple(Int32, Array(Int64))', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(i)
FROM generate('i Nullable(String)', 1)
LIMIT 1;
SELECT
i
FROM generate('i Nullable(String)', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(i)
FROM generate('i Array(String)')
LIMIT 1;
SELECT
i
FROM generate('i Array(String)', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(i)
FROM generate('i UUID')
LIMIT 1;
SELECT
i
FROM generate('i UUID', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(i)
FROM generate('i Array(Nullable(UUID))')
LIMIT 1;
SELECT
i
FROM generate('i Array(Nullable(UUID))', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(i)
FROM generate('i FixedString(4)')
LIMIT 1;
SELECT
i
FROM generate('i FixedString(4)', 10, 10, 1)
LIMIT 10;
SELECT '-';
SELECT
toTypeName(i)
FROM generate('i String')
LIMIT 1;
SELECT
i
FROM generate('i String', 10, 10, 1)
LIMIT 10;
SELECT '-';
DROP TABLE IF EXISTS test_table;
CREATE TABLE test_table(a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)) ENGINE=Memory;
INSERT INTO test_table SELECT * FROM generate('a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)', 2, 10, 1)
LIMIT 10;
SELECT * FROM test_table;
DROP TABLE IF EXISTS test_table;
SELECT '-';
DROP TABLE IF EXISTS test_table_2;
CREATE TABLE test_table_2(a Array(Int8), b UInt32, c Nullable(String), d Decimal32(4), e Nullable(Enum16('h' = 1, 'w' = 5 , 'o' = -200)), f Float64, g Tuple(Date, DateTime, DateTime64, UUID), h FixedString(2)) ENGINE=Memory;
INSERT INTO test_table_2 SELECT * FROM generate('a Array(Int8), b UInt32, c Nullable(String), d Decimal32(4), e Nullable(Enum16(\'h\' = 1, \'w\' = 5 , \'o\' = -200)), f Float64, g Tuple(Date, DateTime, DateTime64, UUID), h FixedString(2)', 3, 5, 10)
LIMIT 10;
SELECT * FROM test_table_2;
SELECT '-';
DROP TABLE IF EXISTS test_table_2;

View File

@ -0,0 +1,2 @@
0 0 test0
ClickHouse is great ClickHouse is fast

View File

@ -0,0 +1,26 @@
DROP TABLE IF EXISTS table_with_complex_default;
CREATE TABLE table_with_complex_default (i Int8, n UInt8 DEFAULT 42, s String DEFAULT concat('test', CAST(n, 'String'))) ENGINE=TinyLog;
INSERT INTO table_with_complex_default FORMAT JSONEachRow {"i":0, "n": 0}
SELECT * FROM table_with_complex_default;
DROP TABLE IF EXISTS table_with_complex_default;
DROP TABLE IF EXISTS test_default_using_alias;
CREATE TABLE test_default_using_alias
(
what String,
a String DEFAULT concat(c, ' is great'),
b String DEFAULT concat(c, ' is fast'),
c String ALIAS concat(what, 'House')
)
ENGINE = TinyLog;
INSERT INTO test_default_using_alias(what) VALUES ('Click');
SELECT a, b FROM test_default_using_alias;
DROP TABLE IF EXISTS test_default_using_alias;

View File

@ -0,0 +1,19 @@
CREATE TABLE test
(
`a1` UInt64 DEFAULT a + 1,
`a1` UInt64 DEFAULT a + 1,
`a2` UInt64 DEFAULT a3 + a4,
`a3` UInt64 DEFAULT a2 + 1,
`a4` UInt64 ALIAS a3 + 1
)
ENGINE = Log; -- { serverError 174 }
CREATE TABLE pythagoras
(
`a` Float64 DEFAULT sqrt((c * c) - (b * b)),
`b` Float64 DEFAULT sqrt((c * c) - (a * a)),
`c` Float64 DEFAULT sqrt((a * a) + (b * b))
)
ENGINE = Log; -- { serverError 174 }
-- TODO: It works but should not: CREATE TABLE test (a DEFAULT b, b DEFAULT a) ENGINE = Memory

View File

@ -6,26 +6,71 @@
| Company | Industry | Usecase | Cluster Size | (Un)Compressed Data Size<abbr title="of single replica"><sup>*</sup></abbr> | Reference |
| --- | --- | --- | --- | --- | --- |
| [2gis](https://2gis.ru) | Maps | Monitoring | — | — | [Talk in Russian, July 2019](https://youtu.be/58sPkXfq6nw) |
| [Aloha Browser](https://alohabrowser.com/) | Mobile App | Browser backend | — | — | [Slides in Russian, May 2019](https://github.com/yandex/clickhouse-presentations/blob/master/meetup22/aloha.pdf) |
| [Amadeus](https://amadeus.com/) | Travel | Analytics | — | — | [Press Release, April 2018](https://www.altinity.com/blog/2018/4/5/amadeus-technologies-launches-investment-and-insights-tool-based-on-machine-learning-and-strategy-algorithms) |
| [Appsflyer](https://www.appsflyer.com) | Mobile analytics | Main product | — | — | [Talk in Russian, July 2019](https://www.youtube.com/watch?v=M3wbRlcpBbY) |
| [ArenaData](https://arenadata.tech/) | Data Platform | Main product | — | — | [Slides in Russian, December 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup38/indexes.pdf) |
| [Badoo](https://badoo.com) | Dating | Timeseries | — | — | [Slides in Russian, December 2019](https://presentations.clickhouse.tech/meetup38/forecast.pdf) |
| [Benocs](https://www.benocs.com/) | Network Telemetry and Analytics | Main Product | — | — | [Slides in English, October 2017](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup9/lpm.pdf) |
| [Bloomberg](https://www.bloomberg.com/) | Finance, Media | Monitoring | 102 servers | — | [Slides, May 2018](https://www.slideshare.net/Altinity/http-analytics-for-6m-requests-per-second-using-clickhouse-by-alexander-bocharov) |
| [Bloxy](https://bloxy.info) | Blockchain | Analytics | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/4_bloxy.pptx) |
| `Dataliance/UltraPower` | Telecom | Analytics | — | — | [Slides in Chinese, January 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup12/telecom.pdf) |
| [CARTO](https://carto.com/) | Business Intelligence | Geo analytics | — | — | [Geospatial processing with Clickhouse](https://carto.com/blog/geospatial-processing-with-clickhouse/) |
| [CERN](http://public.web.cern.ch/public/) | Research | Experiment | — | — | [Press release, April 2012](https://www.yandex.com/company/press_center/press_releases/2012/2012-04-10/) |
| [Cisco](http://public.web.cern.ch/public/) | Networking | Traffic analysis | — | — | [Lightning talk, October 2019](https://youtu.be/-hI1vDR2oPY?t=5057) |
| [Cisco](http://cisco.com/) | Networking | Traffic analysis | — | — | [Lightning talk, October 2019](https://youtu.be/-hI1vDR2oPY?t=5057) |
| [Citadel Securities](https://www.citadelsecurities.com/) | Finance | — | — | — | [Contribution, March 2019](https://github.com/ClickHouse/ClickHouse/pull/4774) |
| [Citymobil](https://city-mobil.ru) | Taxi | Analytics | — | — | [Blog Post in Russian, March 2020](https://habr.com/en/company/citymobil/blog/490660/) |
| [ContentSquare](https://contentsquare.com) | Web analytics | Main product | — | — | [Blog post in French, November 2018](http://souslecapot.net/2018/11/21/patrick-chatain-vp-engineering-chez-contentsquare-penser-davantage-amelioration-continue-que-revolution-constante/) |
| [Cloudflare](https://cloudflare.com) | CDN | Traffic analysis | 36 servers | — | [Blog post, May 2017](https://blog.cloudflare.com/how-cloudflare-analyzes-1m-dns-queries-per-second/), [Blog post, March 2018](https://blog.cloudflare.com/http-analytics-for-6m-requests-per-second-using-clickhouse/) |
| [Corunet](https://coru.net/) | Analytics | Main product | — | — | [Slides in English, April 2019 ](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup21/predictive_models.pdf) |
| [CraiditX 氪信](https://creditx.com) | Finance AI | Analysis | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/udf.pptx) |
| [Criteo/Storetail](https://www.criteo.com/) | Retail | Main product | — | — | [Slides in English, October 2018 ](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup18/3_storetail.pptx) |
| [Deutsche Bank](https://db.com) | Finance | BI Analytics | — | — | [Slides in English, October 2019](https://bigdatadays.ru/wp-content/uploads/2019/10/D2-H3-3_Yakunin-Goihburg.pdf) |
| [Diva-e](https://www.diva-e.com) | Digital consulting | Main Product | — | — | [Slides in English, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup29/ClickHouse-MeetUp-Unusual-Applications-sd-2019-09-17.pdf) |
| [Exness](https://www.exness.com) | Trading | Metrics, Logging | — | — | [Talk in Russian, May 2019](https://youtu.be/_rpU-TvSfZ8?t=3215) |
| [Geniee](https://geniee.co.jp) | Ad network | Main product | — | — | [Blog post in Japanese, July 2017](https://tech.geniee.co.jp/entry/2017/07/20/160100) |
| [LifeStreet](https://cloudflare.com) | Ad network | Main product | — | — | [Blog post in Russian, February 2017](https://habr.com/en/post/322620/) |
| [HUYA](https://www.huya.com/) | Video Streaming | Analytics | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/7.%20ClickHouse万亿数据分析实践%20李本旺(sundy-li)%20虎牙.pdf) |
| [Idealista](https://www.idealista.com) | Real Estate | Analytics | — | — | [Blog Post in English, April 2019](https://clickhouse.yandex/blog/en/clickhouse-meetup-in-madrid-on-april-2-2019) |
| [Infovista](https://www.infovista.com/) | Networks | Analytics | — | — | [Slides in English, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup30/infovista.pdf) |
| [InnoGames](https://www.innogames.com) | Games | Metrics, Logging | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/graphite_and_clickHouse.pdf) |
| [Integros](https://integros.com) | Platform for video services | Analytics | — | — | [Slides in Russian, May 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup22/strategies.pdf) |
| [Kodiak Data](https://www.kodiakdata.com/) | Clouds | Main product | — | — | [Slides in Engish, April 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup13/kodiak_data.pdf) |
| [Kontur](https://kontur.ru) | Software Development | Metrics | — | — | [Talk in Russian, November 2018](https://www.youtube.com/watch?v=U4u4Bd0FtrY) |
| [LifeStreet](https://cloudflare.com) | Ad network | Main product | 60 servers in 3 replicas | 2-2.5 PiB | [Blog post in Russian, February 2017](https://habr.com/en/post/322620/) |
| [Mail.ru Cloud Solutions](https://mcs.mail.ru/) | Cloud services | Main product | — | — | [Running ClickHouse Instance, in Russian](https://mcs.mail.ru/help/db-create/clickhouse#) |
| [MessageBird](https://www.messagebird.com) | Telecommunications | Statistics | — | — | [Slides in English, November 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup20/messagebird.pdf) |
| [MGID](https://www.mgid.com/) | Ad network | Web-analytics | — | — | [Our experience in implementing analytical DBMS ClickHouse, in Russian](http://gs-studio.com/news-about-it/32777----clickhouse---c) |
| [OneAPM](https://www.oneapm.com/) | Monitorings and Data Analysis | Main product | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/8.%20clickhouse在OneAPM的应用%20杜龙.pdf) |
| [Pragma Innovation](http://www.pragma-innovation.fr/) | Telemetry and Big Data Analysis | Main product | — | — | [Slides in English, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup18/4_pragma_innovation.pdf) |
| [QINGCLOUD](https://www.qingcloud.com/) | Cloud services | Main product | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/4.%20Cloud%20%2B%20TSDB%20for%20ClickHouse%20张健%20QingCloud.pdf) |
| [Qrator](https://qrator.net) | DDoS protection | Main product | — | — | [Blog Post, March 2019](https://blog.qrator.net/en/clickhouse-ddos-mitigation_37/) |
| [Beijing PERCENT Information Technology Co., Ltd.](https://www.percent.cn/) | Analytics | Main Product | — | — | [Slides in Chinese, June 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/4.%20ClickHouse万亿数据双中心的设计与实践%20.pdf) |
| [Rambler](https://rambler.ru) | Internet services | Analytics | — | — | [Talk in Russian, April 2018](https://medium.com/@ramblertop/разработка-api-clickhouse-для-рамблер-топ-100-f4c7e56f3141) |
| [Tencent](https://www.tencent.com) | Messaging | Logging | — | — | [Talk in Chinese, November 2019](https://youtu.be/T-iVQRuw-QY?t=5050) |
| [Traffic Stars](https://trafficstars.com/) | AD network | — | — | — | [Slides in Russian, May 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup15/lightning/ninja.pdf) |
| [S7 Airlines](https://www.s7.ru) | Airlines | Metrics, Logging | — | — | [Talk in Russian, March 2019](https://www.youtube.com/watch?v=nwG68klRpPg&t=15s) |
| [SEMrush](https://www.semrush.com/) | Marketing | Main product | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/5_semrush.pdf) |
| [scireum GmbH](https://www.scireum.de/) | e-Commerce | Main product | — | — | [Talk in German, February 2020](https://www.youtube.com/watch?v=7QWAn5RbyR4) |
| [Sentry](https://sentry.io/) | Software developer | Backend for product | — | — | [Blog Post in English, May 2019](https://blog.sentry.io/2019/05/16/introducing-snuba-sentrys-new-search-infrastructure) |
| [SGK](http://www.sgk.gov.tr/wps/portal/sgk/tr) | Goverment Social Security | Analytics | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/ClickHouse%20Meetup-Ramazan%20POLAT.pdf) |
| [seo.do](https://seo.do/) | Analytics | Main product | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/CH%20Presentation-%20Metehan%20Çetinkaya.pdf) |
| [Sina](http://english.sina.com/index.html) | News | — | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/6.%20ClickHouse最佳实践%20高鹏_新浪.pdf) |
| [SMI2](https://smi2.ru/) | News | Analytics | — | — | [Blog Post in Russian, November 2017](https://habr.com/ru/company/smi2/blog/314558/) |
| [Splunk](https://www.splunk.com/) | Business Analytics | Main product | — | — | [Slides in English, January 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup12/splunk.pdf) |
| [Spotify](https://www.spotify.com) | Music | Experimentation | — | — | [Slides, July 2018](https://www.slideshare.net/glebus/using-clickhouse-for-experimentation-104247173) |
| [Tencent](https://www.tencent.com) | Big Data | Data processing | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/5.%20ClickHouse大数据集群应用_李俊飞腾讯网媒事业部.pdf) |
| [Uber](https://www.uber.com) | Taxi | Logging | — | — | [Slides, February 2020](https://presentations.clickhouse.tech/meetup40/ml.pdf) |
| [VKontakte](https://vk.com) | Social Network | Statistics, Logging | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/3_vk.pdf) |
| [Wisebits](https://wisebits.com/) | IT Solutions | Analytics | — | — | [Slides in Russian, May 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup22/strategies.pdf) |
| [Xiaoxin Tech.](https://www.xiaoheiban.cn/) | Education | Common purpose | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/sync-clickhouse-with-mysql-mongodb.pptx) |
| [Ximalaya](https://www.ximalaya.com/) | Audio sharing | OLAP | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/ximalaya.pdf) |
| [Yandex Cloud](https://cloud.yandex.ru/services/managed-clickhouse) | Public Cloud | Main product | — | — | [Talk in Russian, December 2019](https://www.youtube.com/watch?v=pgnak9e_E0o) |
| [Yandex DataLens](https://cloud.yandex.ru/services/datalens) | Business Intelligence | Main product | — | — | [Slides in Russian, December 2019](https://presentations.clickhouse.tech/meetup38/datalens.pdf) |
| [Yandex Market](https://market.yandex.ru/) | e-Commerce | Metrics, Logging | — | — | [Talk in Russian, January 2019](https://youtu.be/_l1qP0DyBcA?t=478) |
| [Yandex Metrica](https://metrica.yandex.com) | Web analytics | Main product | 360 servers in one cluster, 1862 servers in one department | 66.41 PiB / 5.68 PiB | [Slides, February 2020](https://presentations.clickhouse.tech/meetup40/introduction/#13) |
| [ЦВТ](https://htc-cs.ru/) | Software Development | Metrics, Logging | — | — | [Blog Post, March 2019, in Russian](https://vc.ru/dev/62715-kak-my-stroili-monitoring-na-prometheus-clickhouse-i-elk) |
| [МКБ](https://mkb.ru/) | Bank | Web-system monitoring | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/mkb.pdf) |
| [金数据](https://jinshuju.net) | BI Analytics | Main product | — | — | [Slides in Chinese, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/3.%20金数据数据架构调整方案Public.pdf) |
[Original article](https://clickhouse.tech/docs/en/introduction/adopters/) <!--hide-->

View File

@ -96,7 +96,7 @@ Maximum number of bytes before sorting.
What to do if the number of rows received before sorting exceeds one of the limits: 'throw' or 'break'. By default, throw.
## max_result_rows
## max_result_rows {#setting-max_result_rows}
Limit on the number of rows in the result. Also checked for subqueries, and on remote servers when running parts of a distributed query.
@ -107,7 +107,27 @@ Limit on the number of bytes in the result. The same as the previous setting.
## result_overflow_mode
What to do if the volume of the result exceeds one of the limits: 'throw' or 'break'. By default, throw.
Using 'break' is similar to using LIMIT.
Using 'break' is similar to using LIMIT. Break interrupts execution only at the block level. This means that amount of returned rows is greater than [max_result_rows](#setting-max_result_rows), multiple of [max_block_size](settings.md#setting-max_block_size) and depends on [max_threads](settings.md#settings-max_threads).
Пример:
```sql
SET max_threads = 3, max_block_size = 3333;
SET max_result_rows = 3334, result_overflow_mode = 'break';
SELECT *
FROM numbers_mt(100000)
FORMAT Null;
```
Результат:
```text
6666 rows in set. ...
```
## max_execution_time

View File

@ -406,7 +406,7 @@ Possible values:
Default value: 1.
## max_block_size
## max_block_size {#setting-max_block_size}
In ClickHouse, data is processed by blocks (sets of column parts). The internal processing cycles for a single block are efficient enough, but there are noticeable expenditures on each block. The `max_block_size` setting is a recommendation for what size of block (in number of rows) to load from tables. The block size shouldn't be too small, so that the expenditures on each block are still noticeable, but not too large, so that the query with LIMIT that is completed after the first block is processed quickly. The goal is to avoid consuming too much memory when extracting a large number of columns in multiple threads, and to preserve at least some cache locality.

View File

@ -0,0 +1,53 @@
# Generate {#table_engines-generate}
The Generate table engine produces random data for given table schema.
Usage examples:
- Use in test to populate reproducible large table.
- Generate random input for fuzzing tests.
## Usage in ClickHouse Server
```sql
Generate(max_array_length, max_string_length, random_seed)
```
The `max_array_length` and `max_string_length` parameters specify maximum length of all
array columns and strings correspondingly in generated data.
Generate table engine supports only `SELECT` queries.
It supports all [DataTypes](../../data_types/index.md) that can be stored in a table except `LowCardinality` and `AggregateFunction`.
**Example:**
**1.** Set up the `generate_engine_table` table:
```sql
CREATE TABLE generate_engine_table (name String, value UInt32) ENGINE=Generate(3, 5, 1)
```
**2.** Query the data:
```sql
SELECT * FROM generate_engine_table LIMIT 3
```
```text
┌─name─┬──────value─┐
│ c4xJ │ 1412771199 │
│ r │ 1791099446 │
│ 7#$ │ 124312908 │
└──────┴────────────┘
```
## Details of Implementation
- Not supported:
- `ALTER`
- `SELECT ... SAMPLE`
- `INSERT`
- Indices
- Replication
[Original article](https://clickhouse.tech/docs/en/operations/table_engines/generate/) <!--hide-->

View File

@ -1034,7 +1034,7 @@ Alias: `medianExactWeighted`.
- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` value in the range of `[0.01, 0.99]`. Default value: 0.5. At `level=0.5` the function calculates [median](https://en.wikipedia.org/wiki/Median).
- `expr` — Expression over the column values resulting in numeric [data types](../../data_types/index.md#data_types), [Date](../../data_types/date.md) or [DateTime](../../data_types/datetime.md).
- `weight` — Column with weights of sequence elements. Weight is a number of value occurrences.
- `weight` — Column with weights of sequence members. Weight is a number of value occurrences.
**Returned value**
@ -1300,7 +1300,7 @@ Result:
## quantileTDigestWeighted {#quantiletdigestweighted}
Computes an approximate [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence using the [t-digest](https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf) algorithm. The function takes into account the weight of each sequence number. The maximum error is 1%. Memory consumption is `log(n)`, where `n` is a number of values.
Computes an approximate [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence using the [t-digest](https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf) algorithm. The function takes into account the weight of each sequence member. The maximum error is 1%. Memory consumption is `log(n)`, where `n` is a number of values.
The performance of the function is lower than performance of [quantile](#quantile) or [quantileTiming](#quantiletiming). In terms of the ratio of State size to precision, this function is much better than `quantile`.

View File

@ -0,0 +1,38 @@
# generate
Generates random data with given schema.
Allows to populate test tables with data.
Supports all data types that can be stored in table except `LowCardinality` and `AggregateFunction`.
```sql
generate('name TypeName[, name TypeName]...', 'limit'[, 'max_array_length'[, 'max_string_length'[, 'random_seed']]]);
```
**Parameters**
- `name` — Name of corresponding column.
- `TypeName` — Type of corresponding column.
- `limit` — Number of rows to generate.
- `max_array_length` — Maximum array length for all generated arrays. Defaults to `10`.
- `max_string_length` — Maximum string length for all generated strings. Defaults to `10`.
- `random_seed` — Specify random seed manually to produce stable results. Defaults to `0` — seed is randomly generated.
**Returned Value**
A table object with requested schema.
## Usage Example
```sql
SELECT * FROM generate('a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)', 3, 2, 10, 1);
```
```text
┌─a────────┬────────────d─┬─c──────────────────────────────────────────────────────────────────┐
│ [77] │ -124167.6723 │ ('2061-04-17 21:59:44.573','3f72f405-ec3e-13c8-44ca-66ef335f7835') │
│ [32,110] │ -141397.7312 │ ('1979-02-09 03:43:48.526','982486d1-5a5d-a308-e525-7bd8b80ffa73') │
│ [68] │ -67417.0770 │ ('2080-03-12 14:17:31.269','110425e5-413f-10a6-05ba-fa6b3e929f15') │
└──────────┴──────────────┴────────────────────────────────────────────────────────────────────┘
```
[Original article](https://clickhouse.tech/docs/en/query_language/table_functions/generate/) <!--hide-->

View File

@ -0,0 +1 @@
../../../en/operations/table_engines/generate.md

View File

@ -0,0 +1 @@
../../../en/query_language/table_functions/generate.md

View File

@ -1 +0,0 @@
../../en/introduction/features_considered_disadvantages.md

View File

@ -0,0 +1,8 @@
# 欠点と考えられるClickHouseの機能
1. 本格的なトランザクションはありません。
2. 既に挿入されたデータの変更または削除を、高頻度かつ低遅延に行う機能はありません。 [GDPR](https://gdpr-info.eu)に準拠するなど、データをクリーンアップまたは変更するために、バッチ削除およびバッチ更新が利用可能です。
3. インデックスが疎であるため、ClickHouseは、キーで単一行を取得するようなクエリにはあまり適していません。
[Original article](https://clickhouse.yandex/docs/en/introduction/features_considered_disadvantages/) <!--hide-->

View File

@ -1 +0,0 @@
../../en/introduction/history.md

View File

@ -0,0 +1,46 @@
# ClickHouseの歴史
ClickHouseは元々、 [世界で2番目に大きなWeb分析プラットフォーム ](http://w3techs.com/technologies/overview/traffic_analysis/all) である [Yandex.Metrica](https://metrica.yandex.com/)を強化するために開発されたもので、このシステムのコアコンポーネントであり続けています。データベースには13兆を超えるレコードがあり、毎日200億を超えるイベントが発生しますが、ClickHouseでは集計されていないデータから直接カスタムレポートを生成できます。この記事では、ClickHouseの開発の初期段階におけるClickHouseの目標について簡単に説明します。
Yandex.Metricaは、ユーザーが定義した任意のセグメントを使用して、ヒットとセッションに基づいてカスタマイズされたレポートをその都度作成します。これには、多くの場合、一意のユーザー数などの複雑な集計を作成する必要があり、レポートを作成するための新しいデータがリアルタイムで受信されます。
2014年4月の時点で、Yandex.Metricaは毎日約120億のイベント(ページビューとクリック)を追跡していました。カスタムレポートを作成するには、これらすべてのイベントを保存する必要があります。単一のクエリで、数百ミリ秒以内に数百万行をスキャンしたり、わずか数秒で数億行をスキャンする必要があります。
## Yandex.Metricaおよびその他のYandexサービスでの用途
ClickHouseは、Yandex.Metricaで複数の目的に使用されます。その主なタスクは、非集計データを使用してオンラインでレポートを作成することです。 374台のサーバーからなるクラスターを使用し、20.3兆行をデータベースに保存します。圧縮されたデータの量は、重複データとレプリケーションを除いて約2PBです。非圧縮データ(TSV形式)の量は約17PBにもなります。
ClickHouseは以下の目的にも使用されます。
- Yandex.Metricaのデータをセッションリプレイのために保存する。
- 中間データを処理する。
- Analyticsを使用したグローバルレポートの作成。
- Yandex.Metricaエンジンをデバッグするためのクエリの実行。
- APIおよびユーザーインターフェイスからのログの分析。
ClickHouseは少なくとも、そのほか12のYandexのサービス(検索分野、Market、Direct、ビジネス分析、モバイル開発、AdFox、パーソナルサービスなど)で利用されています。
## 集約されたデータと非集約データ
統計を効果的に計算するには、データの量を減らすため、データを集計する必要があるという一般的な意見があります。
ただし次の理由により、データ集約は非常に限られた解決策です。
- ユーザーが必要とするレポートの事前定義リストが必要です。
- ユーザーはカスタムレポートを作成できません。
- 大量のキーを集約する場合、データ量は削減されず、集約は役に立ちません。
- 多数のレポートの場合、集計のバリエーションが多すぎます(組み合わせ爆発)
- カーディナリティの高いキー(URLなど)を集約する場合、データの量はそれほど減少しません(たかだか半分程度)。
- このため、集約されたデータの量は減少するどころか増加する場合があります。
- 生成した全てのレポートをユーザが見るわけではありません。計算の大部分は無駄になっています。
- データの論理的な整合性は、さまざまな集計に対して違反する可能性があります。
何も集約せず、集約されていないデータを操作する場合、実際には計算量が減る可能性があります。
しかしながら集約は、その大部分がオフラインで実行され、比較的ゆったりと処理します。対照的に、オンライン計算では、ユーザーが結果を待っているため、できるだけ高速に計算する必要があります。
Yandex.Metricaには、Metrageと呼ばれるデータを集計するための特別なシステムがあり、これはほとんどのレポートで使用されています。 2009年以降、Yandex.Metricaは、以前にレポートビルダーで使用されていたOLAPServerと呼ばれる非集計データ用の特殊なOLAPデータベースも使用しました。 OLAPServerは非集計データに対してはうまく機能しましたが、多くの制限があり、必要に応じてすべてのレポートに使用することはできませんでした。制限とは、(数値のみしか扱えない)データ型サポートの欠如や、リアルタイムでデータを段階的に更新できないこと(毎日データを書き換えることによってのみ更新可能)がありました。 OLAPServerはDBMSではなく、特殊なDBだったのです。
OLAPServerの制限を取り除き、レポートのための非集計データを扱う問題を解決するために、私達は ClickHouse DBMSを開発しました。
[Original article](https://clickhouse.yandex/docs/en/introduction/history/) <!--hide-->

View File

@ -0,0 +1 @@
../../../en/operations/table_engines/generate.md

View File

@ -0,0 +1 @@
../../../en/query_language/table_functions/generate.md

View File

@ -97,7 +97,7 @@
Что делать, если количество строк, полученное перед сортировкой, превысило одно из ограничений: throw или break. По умолчанию: throw.
## max_result_rows
## max_result_rows {#setting-max_result_rows}
Ограничение на количество строк результата. Проверяются также для подзапросов и на удалённых серверах при выполнении части распределённого запроса.
@ -108,7 +108,24 @@
## result_overflow_mode
Что делать, если объём результата превысил одно из ограничений: throw или break. По умолчанию: throw.
Использование break по смыслу похоже на LIMIT.
Использование break по смыслу похоже на LIMIT. Break прерывает выполнение только на уровне блока. Т.е. число строк которые вернет запрос будет больше чем ограничение [max_result_rows](#setting-max_result_rows), кратно [max_block_size](settings.md#setting-max_block_size) и зависит от [max_threads](settings.md#settings-max_threads).
Пример:
```sql
SET max_threads = 3, max_block_size = 3333;
SET max_result_rows = 3334, result_overflow_mode = 'break';
SELECT *
FROM numbers_mt(100000)
FORMAT Null;
```
Результат:
```text
6666 rows in set. ...
```
## max_execution_time

View File

@ -377,7 +377,7 @@ Ok.
Значение по умолчанию: 0.
## max_block_size
## max_block_size {#setting-max_block_size}
Данные в ClickHouse обрабатываются по блокам (наборам кусочков столбцов). Внутренние циклы обработки для одного блока достаточно эффективны, но есть заметные издержки на каждый блок. Настройка `max_block_size` — это рекомендация, какой размер блока (в количестве строк) загружать из таблиц. Размер блока не должен быть слишком маленьким, чтобы затраты на каждый блок были заметны, но не слишком велики, чтобы запрос с LIMIT, который завершается после первого блока, обрабатывался быстро. Цель состоит в том, чтобы не использовалось слишком много оперативки при вынимании большого количества столбцов в несколько потоков; чтобы оставалась хоть какая-нибудь кэш-локальность.

View File

@ -0,0 +1 @@
../../../en/operations/table_engines/generate.md

View File

@ -860,7 +860,7 @@ FROM t
quantile(level)(expr)
```
Альяс: `median`.
Алиас: `median`.
**Параметры**
@ -870,7 +870,7 @@ quantile(level)(expr)
**Возвращаемое значение**
- Приблизительную квантиль заданного уровня.
- Приблизительный квантиль заданного уровня.
Тип:
@ -926,7 +926,7 @@ SELECT quantile(val) FROM t
quantileDeterministic(level)(expr, determinator)
```
Альяс: `medianDeterministic`.
Алиас: `medianDeterministic`.
**Параметры**
@ -936,7 +936,7 @@ quantileDeterministic(level)(expr, determinator)
**Возвращаемое значение**
- Приблизительную квантиль заданного уровня.
- Приблизительный квантиль заданного уровня.
Тип:
@ -993,7 +993,7 @@ SELECT quantileDeterministic(val, 1) FROM t
quantileExact(level)(expr)
```
Альяс: `medianExact`.
Алиас: `medianExact`.
**Параметры**
@ -1046,7 +1046,7 @@ SELECT quantileExact(number) FROM numbers(10)
quantileExactWeighted(level)(expr, weight)
```
Альяс: `medianExactWeighted`.
Алиас: `medianExactWeighted`.
**Параметры**
@ -1110,7 +1110,7 @@ SELECT quantileExactWeighted(n, val) FROM t
quantileTiming(level)(expr)
```
Альяс: `medianTiming`.
Алиас: `medianTiming`.
**Параметры**
@ -1192,7 +1192,7 @@ SELECT quantileTiming(response_time) FROM t
quantileTimingWeighted(level)(expr, weight)
```
Альяс: `medianTimingWeighted`.
Алиас: `medianTimingWeighted`.
**Параметры**
@ -1276,7 +1276,7 @@ SELECT quantileTimingWeighted(response_time, weight) FROM t
quantileTDigest(level)(expr)
```
Альяс: `medianTDigest`.
Алиас: `medianTDigest`.
**Параметры**
@ -1333,7 +1333,7 @@ SELECT quantileTDigest(number) FROM numbers(10)
quantileTDigestWeighted(level)(expr, weight)
```
Альяс: `medianTDigest`.
Алиас: `medianTDigest`.
**Параметры**
@ -1343,7 +1343,7 @@ quantileTDigestWeighted(level)(expr, weight)
**Возвращаемое значение**
- Приблизительную квантиль заданного уровня.
- Приблизительный квантиль заданного уровня.
Тип:

View File

@ -0,0 +1,37 @@
# generate
Генерирует случайные данные с заданной схемой.
Позволяет заполнять тестовые таблицы данными.
Поддерживает все типы данных, которые могут храниться в таблице, за исключением `LowCardinality` и `AggregateFunction`.
```sql
generate('name TypeName[, name TypeName]...', 'limit'[, 'max_array_length'[, 'max_string_length'[, 'random_seed']]]);
```
**Входные параметры**
- `name` — название соответствующего столбца.
- `TypeName` — тип соответствующего столбца.
- `limit` — количество строк для генерации.
- `max_array_length` — максимальная длина массива для всех сгенерированных массивов. По умолчанию `10`.
- `max_string_length` — максимальная длина строки для всех генерируемых строк. По умолчанию `10`.
- `random_seed` — укажите состояние генератора случайных чисел вручную, чтобы получить стабильные результаты. По умолчанию `0` - генератор инициализируется случайным состоянием.
**Возвращаемое значение**
Объект таблицы с запрошенной схемой.
## Пример
```sql
SELECT * FROM generate('a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)', 3, 2, 10, 1);
```
```text
┌─a────────┬────────────d─┬─c──────────────────────────────────────────────────────────────────┐
│ [77] │ -124167.6723 │ ('2061-04-17 21:59:44.573','3f72f405-ec3e-13c8-44ca-66ef335f7835') │
│ [32,110] │ -141397.7312 │ ('1979-02-09 03:43:48.526','982486d1-5a5d-a308-e525-7bd8b80ffa73') │
│ [68] │ -67417.0770 │ ('2080-03-12 14:17:31.269','110425e5-413f-10a6-05ba-fa6b3e929f15') │
└──────────┴──────────────┴────────────────────────────────────────────────────────────────────┘
```
[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/table_functions/generate/) <!--hide-->

View File

@ -75,6 +75,7 @@ nav:
- 'MaterializedView': 'operations/table_engines/materializedview.md'
- 'Memory': 'operations/table_engines/memory.md'
- 'Buffer': 'operations/table_engines/buffer.md'
- 'Generate': 'operations/table_engines/generate.md'
- 'Database Engines':
- 'Introduction': 'database_engines/index.md'
@ -143,6 +144,7 @@ nav:
- 'odbc': 'query_language/table_functions/odbc.md'
- 'hdfs': 'query_language/table_functions/hdfs.md'
- 'input': 'query_language/table_functions/input.md'
- 'generate': 'query_language/table_functions/generate.md'
- 'Dictionaries':
- 'Introduction': 'query_language/dicts/index.md'
- 'External Dictionaries':

View File

@ -109,6 +109,7 @@ nav:
- 'MaterializedView': 'operations/table_engines/materializedview.md'
- 'Memory': 'operations/table_engines/memory.md'
- 'Buffer': 'operations/table_engines/buffer.md'
- 'Generate': 'operations/table_engines/generate.md'
- 'SQL Reference':
- 'hidden': 'query_language/index.md'
@ -170,6 +171,7 @@ nav:
- 'odbc': 'query_language/table_functions/odbc.md'
- 'hdfs': 'query_language/table_functions/hdfs.md'
- 'input': 'query_language/table_functions/input.md'
- 'generate': 'query_language/table_functions/generate.md'
- 'Dictionaries':
- 'Introduction': 'query_language/dicts/index.md'
- 'External Dictionaries':

View File

@ -79,7 +79,7 @@ nav:
- 'MaterializedView': 'operations/table_engines/materializedview.md'
- 'Memory': 'operations/table_engines/memory.md'
- 'Buffer': 'operations/table_engines/buffer.md'
- 'Generate': 'operations/table_engines/generate.md'
- 'SQL Reference':
- 'hidden': 'query_language/index.md'
- 'Syntax': 'query_language/syntax.md'
@ -142,6 +142,7 @@ nav:
- 'odbc': 'query_language/table_functions/odbc.md'
- 'hdfs': 'query_language/table_functions/hdfs.md'
- 'input': 'query_language/table_functions/input.md'
- 'generate': 'query_language/table_functions/generate.md'
- 'Dictionaries':
- 'Introduction': 'query_language/dicts/index.md'
- 'External Dictionaries':

View File

@ -80,6 +80,7 @@ nav:
- 'MaterializedView': 'operations/table_engines/materializedview.md'
- 'Memory': 'operations/table_engines/memory.md'
- 'Buffer': 'operations/table_engines/buffer.md'
- 'Generate': 'operations/table_engines/generate.md'
- 'Справка по SQL':
- 'hidden': 'query_language/index.md'
@ -143,6 +144,7 @@ nav:
- 'odbc': 'query_language/table_functions/odbc.md'
- 'hdfs': 'query_language/table_functions/hdfs.md'
- 'input': 'query_language/table_functions/input.md'
- 'generate': 'query_language/table_functions/generate.md'
- 'Словари':
- 'Введение': 'query_language/dicts/index.md'
- 'Внешние словари':

View File

@ -109,6 +109,7 @@ nav:
- 'MaterializedView': 'operations/table_engines/materializedview.md'
- 'Memory': 'operations/table_engines/memory.md'
- 'Buffer': 'operations/table_engines/buffer.md'
- 'Generate': 'operations/table_engines/generate.md'
- 'SQL语法':
- 'hidden': 'query_language/index.md'
@ -170,6 +171,7 @@ nav:
- 'odbc': 'query_language/table_functions/odbc.md'
- 'hdfs': 'query_language/table_functions/hdfs.md'
- 'input': 'query_language/table_functions/input.md'
- 'generate': 'query_language/table_functions/generate.md'
- '字典':
- '介绍': 'query_language/dicts/index.md'
- '外部字典':

View File

@ -0,0 +1 @@
../../../en/operations/table_engines/generate.md

View File

@ -0,0 +1 @@
../../../en/query_language/table_functions/generate.md

View File

@ -2172,6 +2172,57 @@ var results =
[0.033, 0.025, 0.011]
]
},
{
"system": "AMD EPYC 7502P / 128G DDR4 / 2NVME SAMSUNG MZQLB960HAJR",
"time": "2020-03-05 00:00:00",
"result":
[
[0.012, 0.019, 0.009],
[0.042, 0.026, 0.038],
[0.026, 0.032, 0.017],
[0.058, 0.025, 0.027],
[0.095, 0.080, 0.087],
[0.143, 0.125, 0.124],
[0.018, 0.010, 0.016],
[0.013, 0.012, 0.013],
[0.201, 0.182, 0.182],
[0.228, 0.204, 0.204],
[0.093, 0.078, 0.077],
[0.100, 0.080, 0.081],
[0.241, 0.222, 0.218],
[0.291, 0.265, 0.270],
[0.268, 0.254, 0.256],
[0.255, 0.241, 0.242],
[0.623, 0.593, 0.599],
[0.373, 0.343, 0.339],
[1.354, 1.318, 1.311],
[0.054, 0.020, 0.022],
[0.495, 0.247, 0.242],
[0.520, 0.258, 0.248],
[0.957, 0.646, 0.652],
[null, null, null],
[0.149, 0.105, 0.099],
[0.091, 0.070, 0.069],
[0.150, 0.096, 0.094],
[0.499, 0.315, 0.309],
[0.437, 0.354, 0.357],
[1.002, 0.996, 0.991],
[0.234, 0.205, 0.207],
[0.380, 0.305, 0.305],
[1.733, 1.651, 1.655],
[1.230, 1.134, 1.132],
[1.217, 1.130, 1.114],
[0.396, 0.385, 0.383],
[0.156, 0.148, 0.160],
[0.065, 0.062, 0.063],
[0.057, 0.052, 0.052],
[0.368, 0.342, 0.336],
[0.030, 0.025, 0.027],
[0.022, 0.017, 0.019],
[0.005, 0.004, 0.004]
]
},
];
</script>
@ -2602,6 +2653,7 @@ Results for AWS are from <b>Wolf Kreuzerkrieg</b>.<br/>
Results for Huawei Taishan are from <b>Peng Gao</b> in sina.com.<br/>
Results for Selectel and AMD EPYC 7402P are from <b>Andrey Dudin</b>.<br/>
Results for ProLiant are from <b>Denis Ustinov</b>.<br/>
Results for AMD EPYC 7502P are from <b>Kostiantyn Velychkovskyi</b>.<br/>
Xeon Gold 6230 server is using 4 x SAMSUNG datacenter class SSD in RAID-10.<br/>
Results for Yandex Managed ClickHouse for "cold cache" are biased and should not be compared, because cache was not flushed for every next query.<br/>
</div>