mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge branch 'master' into negative
This commit is contained in:
commit
5294c1ec9e
@ -1,11 +1,11 @@
|
||||
# This strings autochanged from release_lib.sh:
|
||||
set(VERSION_REVISION 54397 CACHE STRING "")
|
||||
set(VERSION_REVISION 54399 CACHE STRING "")
|
||||
set(VERSION_MAJOR 18 CACHE STRING "")
|
||||
set(VERSION_MINOR 2 CACHE STRING "")
|
||||
set(VERSION_MINOR 4 CACHE STRING "")
|
||||
set(VERSION_PATCH 0 CACHE STRING "")
|
||||
set(VERSION_GITHASH 6ad677d7d6961a0c9088ccd9eff55779cfdaa654 CACHE STRING "")
|
||||
set(VERSION_DESCRIBE v18.2.0-testing CACHE STRING "")
|
||||
set(VERSION_STRING 18.2.0 CACHE STRING "")
|
||||
set(VERSION_GITHASH 8bc95412b66b360fdbef5bb0cec5217378f066a6 CACHE STRING "")
|
||||
set(VERSION_DESCRIBE v18.4.0-testing CACHE STRING "")
|
||||
set(VERSION_STRING 18.4.0 CACHE STRING "")
|
||||
# end of autochange
|
||||
|
||||
set(VERSION_EXTRA "" CACHE STRING "")
|
||||
|
@ -646,7 +646,7 @@ struct AggregateFunctionAnyHeavyData : Data
|
||||
}
|
||||
else
|
||||
{
|
||||
if (counter < to.counter)
|
||||
if ((!this->has() && to.has()) || counter < to.counter)
|
||||
{
|
||||
this->change(to, arena);
|
||||
return true;
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <sys/utsname.h>
|
||||
#include <cerrno>
|
||||
#include <cstdlib>
|
||||
#include <cstring>
|
||||
#include <algorithm>
|
||||
#include <iostream>
|
||||
@ -103,7 +104,9 @@ static ElementIdentifier getElementIdentifier(Node * element)
|
||||
{
|
||||
const Node * node = attrs->item(i);
|
||||
std::string name = node->nodeName();
|
||||
if (name == "replace" || name == "remove" || name == "incl" || name == "from_zk")
|
||||
auto subst_name_pos = std::find(ConfigProcessor::SUBSTITUTION_ATTRS.begin(), ConfigProcessor::SUBSTITUTION_ATTRS.end(), name);
|
||||
if (name == "replace" || name == "remove" ||
|
||||
subst_name_pos != ConfigProcessor::SUBSTITUTION_ATTRS.end())
|
||||
continue;
|
||||
std::string value = node->nodeValue();
|
||||
attrs_kv.push_back(std::make_pair(name, value));
|
||||
@ -267,12 +270,18 @@ void ConfigProcessor::doIncludesRecursive(
|
||||
return;
|
||||
}
|
||||
|
||||
std::map<std::string, const Node *> attr_nodes;
|
||||
NamedNodeMapPtr attributes = node->attributes();
|
||||
const Node * incl_attribute = attributes->getNamedItem("incl");
|
||||
const Node * from_zk_attribute = attributes->getNamedItem("from_zk");
|
||||
size_t substs_count = 0;
|
||||
for (const auto & attr_name : SUBSTITUTION_ATTRS)
|
||||
{
|
||||
auto subst = attributes->getNamedItem(attr_name);
|
||||
attr_nodes[attr_name] = subst;
|
||||
substs_count += static_cast<size_t>(subst == nullptr);
|
||||
}
|
||||
|
||||
if (incl_attribute && from_zk_attribute)
|
||||
throw Poco::Exception("both incl and from_zk attributes set for element <" + node->nodeName() + ">");
|
||||
if (substs_count < SUBSTITUTION_ATTRS.size() - 1) /// only one substitution is allowed
|
||||
throw Poco::Exception("several substitutions attributes set for element <" + node->nodeName() + ">");
|
||||
|
||||
/// Replace the original contents, not add to it.
|
||||
bool replace = attributes->getNamedItem("replace");
|
||||
@ -296,8 +305,8 @@ void ConfigProcessor::doIncludesRecursive(
|
||||
{
|
||||
Element & element = dynamic_cast<Element &>(*node);
|
||||
|
||||
element.removeAttribute("incl");
|
||||
element.removeAttribute("from_zk");
|
||||
for (const auto & attr_name : SUBSTITUTION_ATTRS)
|
||||
element.removeAttribute(attr_name);
|
||||
|
||||
if (replace)
|
||||
{
|
||||
@ -324,16 +333,19 @@ void ConfigProcessor::doIncludesRecursive(
|
||||
}
|
||||
};
|
||||
|
||||
auto get_incl_node = [&](const std::string & name)
|
||||
if (attr_nodes["incl"]) // we have include subst
|
||||
{
|
||||
return include_from ? include_from->getNodeByPath("yandex/" + name) : nullptr;
|
||||
};
|
||||
if (incl_attribute)
|
||||
process_include(incl_attribute, get_incl_node, "Include not found: ");
|
||||
auto get_incl_node = [&](const std::string & name)
|
||||
{
|
||||
return include_from ? include_from->getNodeByPath("yandex/" + name) : nullptr;
|
||||
};
|
||||
|
||||
if (from_zk_attribute)
|
||||
process_include(attr_nodes["incl"], get_incl_node, "Include not found: ");
|
||||
}
|
||||
|
||||
if (attr_nodes["from_zk"]) /// we have zookeeper subst
|
||||
{
|
||||
contributing_zk_paths.insert(from_zk_attribute->getNodeValue());
|
||||
contributing_zk_paths.insert(attr_nodes["from_zk"]->getNodeValue());
|
||||
|
||||
if (zk_node_cache)
|
||||
{
|
||||
@ -349,10 +361,27 @@ void ConfigProcessor::doIncludesRecursive(
|
||||
return getRootNode(zk_document.get());
|
||||
};
|
||||
|
||||
process_include(from_zk_attribute, get_zk_node, "Could not get ZooKeeper node: ");
|
||||
process_include(attr_nodes["from_zk"], get_zk_node, "Could not get ZooKeeper node: ");
|
||||
}
|
||||
}
|
||||
|
||||
if (attr_nodes["from_env"]) /// we have env subst
|
||||
{
|
||||
XMLDocumentPtr env_document;
|
||||
auto get_env_node = [&](const std::string & name) -> const Node *
|
||||
{
|
||||
const char * env_val = std::getenv(name.c_str());
|
||||
if (env_val == nullptr)
|
||||
return nullptr;
|
||||
|
||||
env_document = dom_parser.parseString("<from_env>" + std::string{env_val} + "</from_env>");
|
||||
|
||||
return getRootNode(env_document.get());
|
||||
};
|
||||
|
||||
process_include(attr_nodes["from_env"], get_env_node, "Env variable is not set: ");
|
||||
}
|
||||
|
||||
if (included_something)
|
||||
doIncludesRecursive(config, include_from, node, zk_node_cache, contributing_zk_paths);
|
||||
else
|
||||
|
@ -95,6 +95,8 @@ public:
|
||||
/// Is the file named as result of config preprocessing, not as original files.
|
||||
static bool isPreprocessedFile(const std::string & config_path);
|
||||
|
||||
static inline const auto SUBSTITUTION_ATTRS = {"incl", "from_zk", "from_env"};
|
||||
|
||||
private:
|
||||
const std::string path;
|
||||
const std::string preprocessed_path;
|
||||
|
@ -93,16 +93,16 @@ void SelectStreamFactory::createForShard(
|
||||
if (shard_info.isLocal())
|
||||
{
|
||||
StoragePtr main_table_storage;
|
||||
|
||||
|
||||
if (table_func_ptr)
|
||||
{
|
||||
auto table_function = static_cast<ASTFunction *>(table_func_ptr.get());
|
||||
auto table_function = static_cast<const ASTFunction *>(table_func_ptr.get());
|
||||
main_table_storage = TableFunctionFactory::instance().get(table_function->name, context)->execute(table_func_ptr, context);
|
||||
}
|
||||
else
|
||||
else
|
||||
main_table_storage = context.tryGetTable(main_table.database, main_table.table);
|
||||
|
||||
|
||||
|
||||
|
||||
if (!main_table_storage) /// Table is absent on a local server.
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable);
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
QueryProcessingStage::Enum processed_stage_,
|
||||
QualifiedTableName main_table_,
|
||||
const Tables & external_tables);
|
||||
|
||||
|
||||
/// TableFunction in a query.
|
||||
SelectStreamFactory(
|
||||
const Block & header_,
|
||||
|
@ -559,7 +559,7 @@ void ExpressionAnalyzer::analyzeAggregation()
|
||||
const auto & col = block.getByName(column_name);
|
||||
|
||||
/// Constant expressions have non-null column pointer at this stage.
|
||||
if (const auto is_constexpr = col.column)
|
||||
if (col.column && col.column->isColumnConst())
|
||||
{
|
||||
/// But don't remove last key column if no aggregate functions, otherwise aggregation will not work.
|
||||
if (!aggregate_descriptions.empty() || size > 1)
|
||||
|
@ -57,13 +57,12 @@ ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const Context &
|
||||
/// Branch with string in qery.
|
||||
if (typeid_cast<const ASTLiteral *>(node.get()))
|
||||
return node;
|
||||
|
||||
|
||||
/// Branch with TableFunction in query.
|
||||
if (auto table_func_ptr = typeid_cast<ASTFunction *>(node.get()))
|
||||
if (TableFunctionFactory::instance().isTableFunctionName(table_func_ptr->name))
|
||||
|
||||
return node;
|
||||
|
||||
|
||||
return std::make_shared<ASTLiteral>(evaluateConstantExpression(node, context).first);
|
||||
}
|
||||
|
||||
|
@ -372,9 +372,9 @@ void ASTSelectQuery::replaceDatabaseAndTable(const String & database_name, const
|
||||
children.emplace_back(tables_list);
|
||||
table_expression = table_expr.get();
|
||||
}
|
||||
|
||||
|
||||
ASTPtr table = std::make_shared<ASTIdentifier>(table_name, ASTIdentifier::Table);
|
||||
|
||||
|
||||
if (!database_name.empty())
|
||||
{
|
||||
ASTPtr database = std::make_shared<ASTIdentifier>(database_name, ASTIdentifier::Database);
|
||||
@ -405,7 +405,7 @@ void ASTSelectQuery::addTableFunction(ASTPtr & table_function_ptr)
|
||||
children.emplace_back(tables_list);
|
||||
table_expression = table_expr.get();
|
||||
}
|
||||
|
||||
|
||||
table_expression->table_function = table_function_ptr;
|
||||
table_expression->database_and_table_name = nullptr;
|
||||
}
|
||||
|
@ -244,19 +244,19 @@ BlockInputStreams StorageDistributed::read(
|
||||
processed_stage = result_size == 1
|
||||
? QueryProcessingStage::Complete
|
||||
: QueryProcessingStage::WithMergeableState;
|
||||
|
||||
|
||||
|
||||
const auto & modified_query_ast = rewriteSelectQuery(
|
||||
query_info.query, remote_database, remote_table, remote_table_function_ptr);
|
||||
|
||||
Block header = materializeBlock(InterpreterSelectQuery(query_info.query, context, Names{}, processed_stage).getSampleBlock());
|
||||
|
||||
|
||||
ClusterProxy::SelectStreamFactory select_stream_factory = remote_table_function_ptr ?
|
||||
ClusterProxy::SelectStreamFactory(
|
||||
header, processed_stage, remote_table_function_ptr, context.getExternalTables())
|
||||
: ClusterProxy::SelectStreamFactory(
|
||||
header, processed_stage, QualifiedTableName{remote_database, remote_table}, context.getExternalTables());
|
||||
|
||||
|
||||
return ClusterProxy::executeQuery(
|
||||
select_stream_factory, cluster, modified_query_ast, context, settings);
|
||||
}
|
||||
|
@ -41,7 +41,7 @@ public:
|
||||
const String & remote_table_, /// The name of the table on the remote servers.
|
||||
ClusterPtr owned_cluster_,
|
||||
const Context & context_);
|
||||
|
||||
|
||||
static StoragePtr createWithOwnCluster(
|
||||
const std::string & table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
@ -155,7 +155,7 @@ protected:
|
||||
const ASTPtr & sharding_key_,
|
||||
const String & data_path_,
|
||||
bool attach);
|
||||
|
||||
|
||||
StorageDistributed(
|
||||
const String & database_name,
|
||||
const String & table_name_,
|
||||
|
@ -128,8 +128,9 @@ class ReadBufferFromKafkaConsumer : public ReadBuffer
|
||||
public:
|
||||
ReadBufferFromKafkaConsumer(rd_kafka_t * consumer_, Poco::Logger * log_, char row_delimiter_)
|
||||
: ReadBuffer(nullptr, 0), consumer(consumer_), current(nullptr),
|
||||
current_pending(false), log(log_), read_messages(0), row_delimiter(row_delimiter_) {
|
||||
LOG_TRACE(log, "row delimiter is :" << row_delimiter);
|
||||
current_pending(false), log(log_), read_messages(0), row_delimiter(row_delimiter_)
|
||||
{
|
||||
LOG_TRACE(log, "Row delimiter is: " << row_delimiter);
|
||||
}
|
||||
|
||||
~ReadBufferFromKafkaConsumer() { reset(); }
|
||||
|
@ -29,14 +29,14 @@ ColumnsDescription getStructureOfRemoteTable(
|
||||
{
|
||||
/// Send to the first any remote shard.
|
||||
const auto & shard_info = cluster.getAnyShardInfo();
|
||||
|
||||
|
||||
String query;
|
||||
|
||||
if (table_func_ptr)
|
||||
{
|
||||
if (shard_info.isLocal())
|
||||
{
|
||||
auto table_function = static_cast<ASTFunction *>(table_func_ptr.get());
|
||||
auto table_function = static_cast<const ASTFunction *>(table_func_ptr.get());
|
||||
return TableFunctionFactory::instance().get(table_function->name, context)->execute(table_func_ptr, context)->getColumns();
|
||||
}
|
||||
|
||||
|
@ -42,7 +42,7 @@ public:
|
||||
TableFunctionPtr get(
|
||||
const std::string & name,
|
||||
const Context & context) const;
|
||||
|
||||
|
||||
bool isTableFunctionName(const std::string & name) const;
|
||||
|
||||
const TableFunctions & getAllTableFunctions() const
|
||||
|
@ -231,17 +231,18 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
|
||||
++arg_num;
|
||||
|
||||
args[arg_num] = evaluateConstantExpressionOrIdentifierAsLiteral(args[arg_num], context);
|
||||
|
||||
const auto table_function = static_cast<ASTFunction *>(args[arg_num].get());
|
||||
|
||||
if (TableFunctionFactory::instance().isTableFunctionName(table_function->name))
|
||||
|
||||
const auto function = typeid_cast<const ASTFunction *>(args[arg_num].get());
|
||||
|
||||
if (function && TableFunctionFactory::instance().isTableFunctionName(function->name))
|
||||
{
|
||||
remote_table_function_ptr = args[arg_num];
|
||||
++arg_num;
|
||||
}
|
||||
else {
|
||||
else
|
||||
{
|
||||
remote_database = static_cast<const ASTLiteral &>(*args[arg_num]).value.safeGet<String>();
|
||||
|
||||
|
||||
++arg_num;
|
||||
|
||||
size_t dot = remote_database.find('.');
|
||||
@ -254,12 +255,13 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
|
||||
else
|
||||
{
|
||||
if (arg_num >= args.size())
|
||||
{
|
||||
throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
else
|
||||
{
|
||||
args[arg_num] = evaluateConstantExpressionOrIdentifierAsLiteral(args[arg_num], context);
|
||||
remote_table = static_cast<const ASTLiteral &>(*args[arg_num]).value.safeGet<String>();
|
||||
remote_database = remote_database;
|
||||
++arg_num;
|
||||
}
|
||||
}
|
||||
@ -315,15 +317,14 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
|
||||
}
|
||||
|
||||
auto structure_remote_table = getStructureOfRemoteTable(*cluster, remote_database, remote_table, context, remote_table_function_ptr);
|
||||
|
||||
StoragePtr res = remote_table_function_ptr ?
|
||||
StorageDistributed::createWithOwnCluster(
|
||||
|
||||
StoragePtr res = remote_table_function_ptr
|
||||
? StorageDistributed::createWithOwnCluster(
|
||||
getName(),
|
||||
structure_remote_table,
|
||||
remote_table_function_ptr,
|
||||
cluster,
|
||||
context)
|
||||
|
||||
: StorageDistributed::createWithOwnCluster(
|
||||
getName(),
|
||||
structure_remote_table,
|
||||
@ -336,6 +337,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
TableFunctionRemote::TableFunctionRemote(const std::string & name_)
|
||||
: name(name_)
|
||||
{
|
||||
|
@ -20,8 +20,16 @@ from .client import Client, CommandRequest
|
||||
|
||||
|
||||
HELPERS_DIR = p.dirname(__file__)
|
||||
DEFAULT_ENV_NAME = 'env_file'
|
||||
|
||||
|
||||
def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME):
|
||||
full_path = os.path.join(path, fname)
|
||||
with open(full_path, 'w') as f:
|
||||
for var, value in variables.items():
|
||||
f.write("=".join([var, value]) + "\n")
|
||||
return full_path
|
||||
|
||||
class ClickHouseCluster:
|
||||
"""ClickHouse cluster with several instances and (possibly) ZooKeeper.
|
||||
|
||||
@ -55,12 +63,12 @@ class ClickHouseCluster:
|
||||
self.with_zookeeper = False
|
||||
self.with_mysql = False
|
||||
self.with_kafka = False
|
||||
|
||||
|
||||
self.docker_client = None
|
||||
self.is_up = False
|
||||
|
||||
|
||||
def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={}, with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None, hostname=None):
|
||||
def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={}, with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None, hostname=None, env_variables={}):
|
||||
"""Add an instance to the cluster.
|
||||
|
||||
name - the name of the instance directory and the value of the 'instance' macro in ClickHouse.
|
||||
@ -78,7 +86,7 @@ class ClickHouseCluster:
|
||||
|
||||
instance = ClickHouseInstance(
|
||||
self, self.base_dir, name, config_dir, main_configs, user_configs, macros, with_zookeeper,
|
||||
self.zookeeper_config_path, with_mysql, with_kafka, self.base_configs_dir, self.server_bin_path, clickhouse_path_dir, hostname=hostname)
|
||||
self.zookeeper_config_path, with_mysql, with_kafka, self.base_configs_dir, self.server_bin_path, clickhouse_path_dir, hostname=hostname, env_variables=env_variables)
|
||||
|
||||
self.instances[name] = instance
|
||||
self.base_cmd.extend(['--file', instance.docker_compose_path])
|
||||
@ -87,7 +95,7 @@ class ClickHouseCluster:
|
||||
self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_zookeeper.yml')])
|
||||
self.base_zookeeper_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
|
||||
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_zookeeper.yml')]
|
||||
|
||||
|
||||
if with_mysql and not self.with_mysql:
|
||||
self.with_mysql = True
|
||||
self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_mysql.yml')])
|
||||
@ -219,13 +227,15 @@ services:
|
||||
- --log-file=/var/log/clickhouse-server/clickhouse-server.log
|
||||
- --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log
|
||||
depends_on: {depends_on}
|
||||
env_file:
|
||||
- {env_file}
|
||||
'''
|
||||
|
||||
|
||||
class ClickHouseInstance:
|
||||
def __init__(
|
||||
self, cluster, base_path, name, custom_config_dir, custom_main_configs, custom_user_configs, macros,
|
||||
with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, base_configs_dir, server_bin_path, clickhouse_path_dir, hostname=None):
|
||||
with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, base_configs_dir, server_bin_path, clickhouse_path_dir, hostname=None, env_variables={}):
|
||||
|
||||
self.name = name
|
||||
self.base_cmd = cluster.base_cmd[:]
|
||||
@ -249,6 +259,7 @@ class ClickHouseInstance:
|
||||
|
||||
self.path = p.join(self.cluster.instances_dir, name)
|
||||
self.docker_compose_path = p.join(self.path, 'docker_compose.yml')
|
||||
self.env_variables = env_variables
|
||||
|
||||
self.docker_client = None
|
||||
self.ip_address = None
|
||||
@ -396,6 +407,8 @@ class ClickHouseInstance:
|
||||
depends_on.append("zoo2")
|
||||
depends_on.append("zoo3")
|
||||
|
||||
env_file = _create_env_file(os.path.dirname(self.docker_compose_path), self.env_variables)
|
||||
|
||||
with open(self.docker_compose_path, 'w') as docker_compose:
|
||||
docker_compose.write(DOCKER_COMPOSE_TEMPLATE.format(
|
||||
name=self.name,
|
||||
@ -406,7 +419,8 @@ class ClickHouseInstance:
|
||||
config_d_dir=config_d_dir,
|
||||
db_dir=db_dir,
|
||||
logs_dir=logs_dir,
|
||||
depends_on=str(depends_on)))
|
||||
depends_on=str(depends_on),
|
||||
env_file=env_file))
|
||||
|
||||
|
||||
def destroy_dir(self):
|
||||
|
@ -0,0 +1,14 @@
|
||||
<yandex>
|
||||
<profiles>
|
||||
<default>
|
||||
<max_query_size from_env="MAX_QUERY_SIZE" />
|
||||
</default>
|
||||
</profiles>
|
||||
<users>
|
||||
<default>
|
||||
<password></password>
|
||||
<profile>default</profile>
|
||||
<quota>default</quota>
|
||||
</default>
|
||||
</users>
|
||||
</yandex>
|
@ -0,0 +1,15 @@
|
||||
<yandex>
|
||||
<include_from>/etc/clickhouse-server/config.d/max_query_size.xml</include_from>
|
||||
<profiles>
|
||||
<default>
|
||||
<max_query_size incl="mqs" />
|
||||
</default>
|
||||
</profiles>
|
||||
<users>
|
||||
<default>
|
||||
<password></password>
|
||||
<profile>default</profile>
|
||||
<quota>default</quota>
|
||||
</default>
|
||||
</users>
|
||||
</yandex>
|
@ -0,0 +1,14 @@
|
||||
<yandex>
|
||||
<profiles>
|
||||
<default>
|
||||
<max_query_size>33333</max_query_size>
|
||||
</default>
|
||||
</profiles>
|
||||
<users>
|
||||
<default>
|
||||
<password></password>
|
||||
<profile>default</profile>
|
||||
<quota>default</quota>
|
||||
</default>
|
||||
</users>
|
||||
</yandex>
|
@ -0,0 +1,14 @@
|
||||
<yandex>
|
||||
<profiles>
|
||||
<default>
|
||||
<max_query_size from_zk="/setting/max_query_size" />
|
||||
</default>
|
||||
</profiles>
|
||||
<users>
|
||||
<default>
|
||||
<password></password>
|
||||
<profile>default</profile>
|
||||
<quota>default</quota>
|
||||
</default>
|
||||
</users>
|
||||
</yandex>
|
@ -0,0 +1,3 @@
|
||||
<yandex>
|
||||
<mqs>99999</mqs>
|
||||
</yandex>
|
28
dbms/tests/integration/test_config_substitutions/test.py
Normal file
28
dbms/tests/integration/test_config_substitutions/test.py
Normal file
@ -0,0 +1,28 @@
|
||||
import time
|
||||
import pytest
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node1 = cluster.add_instance('node1', main_configs=['configs/config_no_substs.xml']) # hardcoded value 33333
|
||||
node2 = cluster.add_instance('node2', main_configs=['configs/config_env.xml'], env_variables={"MAX_QUERY_SIZE": "55555"})
|
||||
node3 = cluster.add_instance('node3', main_configs=['configs/config_zk.xml'], with_zookeeper=True)
|
||||
node4 = cluster.add_instance('node4', main_configs=['configs/config_incl.xml', 'configs/max_query_size.xml']) # include value 77777
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def start_cluster():
|
||||
try:
|
||||
def create_zk_roots(zk):
|
||||
zk.create(path="/setting/max_query_size", value="77777", makepath=True)
|
||||
cluster.add_zookeeper_startup_command(create_zk_roots)
|
||||
|
||||
cluster.start()
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
def test_config(start_cluster):
|
||||
assert node1.query("select value from system.settings where name = 'max_query_size'") == "33333\n"
|
||||
assert node2.query("select value from system.settings where name = 'max_query_size'") == "55555\n"
|
||||
assert node3.query("select value from system.settings where name = 'max_query_size'") == "77777\n"
|
||||
assert node4.query("select value from system.settings where name = 'max_query_size'") == "99999\n"
|
@ -14,7 +14,7 @@ def _fill_nodes(nodes, shard):
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}', date, id, 8192);
|
||||
'''.format(shard=shard, replica=node.name))
|
||||
|
||||
cluster = ClickHouseCluster(__file__, server_bin_path="/home/alesap/ClickHouse/dbms/programs/clickhouse-server")
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml', 'configs/credentials1.xml'], with_zookeeper=True)
|
||||
node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml', 'configs/credentials1.xml'], with_zookeeper=True)
|
||||
|
||||
|
@ -0,0 +1,7 @@
|
||||
1 1
|
||||
0 1
|
||||
1 1
|
||||
0 1
|
||||
1 1
|
||||
0 8
|
||||
1 2
|
12
dbms/tests/queries/0_stateless/00676_group_by_in.sql
Normal file
12
dbms/tests/queries/0_stateless/00676_group_by_in.sql
Normal file
@ -0,0 +1,12 @@
|
||||
SELECT dummy IN (0) AS x, count() GROUP BY x;
|
||||
|
||||
SELECT 1 IN (0) AS x, count() GROUP BY x;
|
||||
SELECT 0 IN (0) AS x, count() GROUP BY x;
|
||||
SELECT materialize(1) IN (0) AS x, count() GROUP BY x;
|
||||
SELECT materialize(0) IN (0) AS x, count() GROUP BY x;
|
||||
|
||||
SELECT
|
||||
number IN (1, 2) AS x,
|
||||
count()
|
||||
FROM numbers(10)
|
||||
GROUP BY x;
|
@ -0,0 +1 @@
|
||||
4 ['hello','world'] hello
|
@ -0,0 +1 @@
|
||||
WITH arrayJoin(['hello', 'world']) AS s SELECT count(), arraySort(groupUniqArray(s)), anyHeavy(s) FROM remote('127.0.0.{2,3}', system.one);
|
4
debian/changelog
vendored
4
debian/changelog
vendored
@ -1,5 +1,5 @@
|
||||
clickhouse (18.2.0) unstable; urgency=low
|
||||
clickhouse (18.4.0) unstable; urgency=low
|
||||
|
||||
* Modified source code
|
||||
|
||||
-- <robot-metrika-test@yandex-team.ru> Mon, 23 Jul 2018 22:38:09 +0300
|
||||
-- <robot-metrika-test@yandex-team.ru> Sat, 28 Jul 2018 00:35:05 +0300
|
||||
|
@ -32,8 +32,6 @@ See `dbms/tests/integration/README.md` on how to run these tests.
|
||||
|
||||
Note that integration of ClickHouse with third-party drivers is not tested. Also we currently don't have integration tests with our JDBC and ODBC drivers.
|
||||
|
||||
We don't have integration tests for `Kafka` table engine that is developed by community - this is one of the most anticipated tests (otherwise there is almost no way to be confident with `Kafka` tables).
|
||||
|
||||
|
||||
## Unit Tests
|
||||
|
||||
|
@ -64,7 +64,11 @@ Comma Separated Values format ([RFC](https://tools.ietf.org/html/rfc4180)).
|
||||
|
||||
When formatting, rows are enclosed in double quotes. A double quote inside a string is output as two double quotes in a row. There are no other rules for escaping characters. Date and date-time are enclosed in double quotes. Numbers are output without quotes. Values are separated by a delimiter*. Rows are separated using the Unix line feed (LF). Arrays are serialized in CSV as follows: first the array is serialized to a string as in TabSeparated format, and then the resulting string is output to CSV in double quotes. Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost).
|
||||
|
||||
*By default — `,`. See a [format_csv_delimiter](/docs/en/operations/settings/settings/#format_csv_delimiter) setting for additional info.
|
||||
```
|
||||
clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv
|
||||
```
|
||||
|
||||
*By default — `,`. See a [format_csv_delimiter](/operations/settings/settings/#format_csv_delimiter) setting for additional info.
|
||||
|
||||
When parsing, all values can be parsed either with or without quotes. Both double and single quotes are supported. Rows can also be arranged without quotes. In this case, they are parsed up to a delimiter or line feed (CR or LF). In violation of the RFC, when parsing rows without quotes, the leading and trailing spaces and tabs are ignored. For the line feed, Unix (LF), Windows (CR LF) and Mac OS Classic (CR LF) are all supported.
|
||||
|
||||
|
@ -2,27 +2,27 @@
|
||||
|
||||
## True column-oriented DBMS
|
||||
|
||||
In a true column-oriented DBMS, there isn't any "garbage" stored with the values. Among other things, this means that constant-length values must be supported, to avoid storing their length "number" next to the values. As an example, a billion UInt8-type values should actually consume around 1 GB uncompressed, or this will strongly affect the CPU use. It is very important to store data compactly (without any "garbage") even when uncompressed, since the speed of decompression (CPU usage) depends mainly on the volume of uncompressed data.
|
||||
In a true column-oriented DBMS, there is no excessive data stored with the values. For example, this means that constant-length values must be supported, to avoid storing their length as additional integer next to the values. In this case, a billion UInt8 values should actually consume around 1 GB uncompressed, or this will strongly affect the CPU use. It is very important to store data compactly even when uncompressed, since the speed of decompression (CPU usage) depends mainly on the volume of uncompressed data.
|
||||
|
||||
This is worth noting because there are systems that can store values of separate columns separately, but that can't effectively process analytical queries due to their optimization for other scenarios. Examples are HBase, BigTable, Cassandra, and HyperTable. In these systems, you will get throughput around a hundred thousand rows per second, but not hundreds of millions of rows per second.
|
||||
This is worth noting because there are systems that can store values of different columns separately, but that can't effectively process analytical queries due to their optimization for other scenarios. Examples are HBase, BigTable, Cassandra, and HyperTable. In these systems, you will get throughput around a hundred thousand rows per second, but not hundreds of millions of rows per second.
|
||||
|
||||
Also note that ClickHouse is a DBMS, not a single database. ClickHouse allows creating tables and databases in runtime, loading data, and running queries without reconfiguring and restarting the server.
|
||||
Also note that ClickHouse is a database management system, not a single database. ClickHouse allows creating tables and databases in runtime, loading data, and running queries without reconfiguring and restarting the server.
|
||||
|
||||
## Data compression
|
||||
|
||||
Some column-oriented DBMSs (InfiniDB CE and MonetDB) do not use data compression. However, data compression really improves performance.
|
||||
Some column-oriented DBMSs (InfiniDB CE and MonetDB) do not use data compression. However, data compression is crucial to achieve excellent performance.
|
||||
|
||||
## Disk storage of data
|
||||
|
||||
Many column-oriented DBMSs (such as SAP HANA and Google PowerDrill) can only work in RAM. But even on thousands of servers, the RAM is too small for storing all the pageviews and sessions in Yandex.Metrica.
|
||||
Many column-oriented DBMSs (such as SAP HANA and Google PowerDrill) can only work in RAM. This approach stimulates the allocation of a larger hardware budget than is actually necessary for real-time analysis. ClickHouse is designed to work on regular hard drives, which ensures low cost of ownership per gigabyte of data, but SSD and additional RAM are also utilized fully if available.
|
||||
|
||||
## Parallel processing on multiple cores
|
||||
|
||||
Large queries are parallelized in a natural way.
|
||||
Large queries are parallelized in a natural way, utilizing all necessary resources that are available on the current server.
|
||||
|
||||
## Distributed processing on multiple servers
|
||||
|
||||
Almost none of the columnar DBMSs listed above have support for distributed processing.
|
||||
Almost none of the columnar DBMSs mentioned above have support for distributed query processing.
|
||||
In ClickHouse, data can reside on different shards. Each shard can be a group of replicas that are used for fault tolerance. The query is processed on all the shards in parallel. This is transparent for the user.
|
||||
|
||||
## SQL support
|
||||
@ -33,30 +33,37 @@ However, this is a declarative query language based on SQL that can't be differe
|
||||
JOINs are supported. Subqueries are supported in FROM, IN, and JOIN clauses, as well as scalar subqueries.
|
||||
Dependent subqueries are not supported.
|
||||
|
||||
ClickHouse supports declarative query language that is based on SQL and complies to SQL standard in many cases.
|
||||
GROUP BY, ORDER BY, scalar subqueries and subqueries in FROM, IN and JOIN clauses are supported.
|
||||
Correlated subqueries and window functions are not supported.
|
||||
|
||||
## Vector engine
|
||||
|
||||
Data is not only stored by columns, but is processed by vectors (parts of columns). This allows us to achieve high CPU performance.
|
||||
Data is not only stored by columns, but is also processed by vectors (parts of columns). This allows to achieve high CPU efficiency.
|
||||
|
||||
## Real-time data updates
|
||||
|
||||
ClickHouse supports primary key tables. In order to quickly perform queries on the range of the primary key, the data is sorted incrementally using the merge tree. Due to this, data can continually be added to the table. There is no locking when adding data.
|
||||
ClickHouse supports tables with a primary key. In order to quickly perform queries on the range of the primary key, the data is sorted incrementally using the merge tree. Due to this, data can continually be added to the table. No locks are taken when new data is ingested.
|
||||
|
||||
## Indexes
|
||||
## Index
|
||||
|
||||
Having a primary key makes it possible to extract data for specific clients (for instance, Yandex.Metrica tracking tags) for a specific time range, with low latency less than several dozen milliseconds.
|
||||
Having a data physically sorted by primary key makes it possible to extract data for it's specific values or value ranges with low latency, less than few dozen milliseconds.
|
||||
|
||||
## Suitable for online queries
|
||||
|
||||
This lets us use the system as the back-end for a web interface. Low latency means queries can be processed without delay, while the Yandex.Metrica interface page is loading. In other words, in online mode.
|
||||
Low latency means that queries can be processed without delay and without trying to prepare answer in advance, right at the same moment while user interface page is loading. In other words, online.
|
||||
|
||||
## Support for approximated calculations
|
||||
|
||||
1. The system contains aggregate functions for approximated calculation of the number of various values, medians, and quantiles.
|
||||
2. Supports running a query based on a part (sample) of data and getting an approximated result. In this case, proportionally less data is retrieved from the disk.
|
||||
3. Supports running an aggregation for a limited number of random keys, instead of for all keys. Under certain conditions for key distribution in the data, this provides a reasonably accurate result while using fewer resources.
|
||||
ClickHouse provides various ways to trade accuracy for performance:
|
||||
|
||||
## Data replication and support for data integrity on replicas
|
||||
1. Aggregate functions for approximated calculation of the number of distinct values, medians, and quantiles.
|
||||
2. Running a query based on a part (sample) of data and getting an approximated result. In this case, proportionally less data is retrieved from the disk.
|
||||
3. Running an aggregation for a limited number of random keys, instead of for all keys. Under certain conditions for key distribution in the data, this provides a reasonably accurate result while using fewer resources.
|
||||
|
||||
Uses asynchronous multimaster replication. After being written to any available replica, data is distributed to all the remaining replicas. The system maintains identical data on different replicas. Data is restored automatically after a failure, or using a "button" for complex cases.
|
||||
For more information, see the section [Data replication](../operations/table_engines/replication.md#table_engines-replication).
|
||||
## Data replication and integrity
|
||||
|
||||
ClickHouse uses asynchronous multimaster replication. After being written to any available replica, data is distributed to all the other replicas in background. The system maintains identical data on different replicas. Data is restored automatically after most failures, or semiautomatically in complicated cases.
|
||||
|
||||
For more information, see the [Data replication](../operations/table_engines/replication.md#table_engines-replication) section.
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
# ClickHouse features that can be considered disadvantages
|
||||
|
||||
1. No transactions.
|
||||
2. For aggregation, query results must fit in the RAM on a single server. However, the volume of source data for a query may be indefinitely large.
|
||||
3. Lack of full-fledged UPDATE/DELETE implementation.
|
||||
1. No full-fledged transactions.
|
||||
2. Lack of ability to modify or delete already inserted data with high rate and low latency. There are batch deletes available to clean up data that is not needed anymore or to comply with [GDPR](https://gdpr-info.eu). Batch updates are in development as of July 2018.
|
||||
3. Sparse index makes ClickHouse not really suitable for point queries retrieving single rows by their keys.
|
||||
|
||||
|
77
docs/en/operations/table_engines/url.md
Normal file
77
docs/en/operations/table_engines/url.md
Normal file
@ -0,0 +1,77 @@
|
||||
<a name="table_engines-url"></a>
|
||||
|
||||
# URL(URL, Format)
|
||||
|
||||
This data source operates with data on remote HTTP/HTTPS server. The engine is
|
||||
similar to [`File`](./file.md#).
|
||||
|
||||
## Usage in ClickHouse server
|
||||
|
||||
```
|
||||
URL(URL, Format)
|
||||
```
|
||||
|
||||
`Format` should be supported for `SELECT` and/or `INSERT`. For the full list of
|
||||
supported formats see [Formats](../../interfaces/formats.md#formats).
|
||||
|
||||
`URL` must match the format of Uniform Resource Locator. The specified
|
||||
URL must address a server working with HTTP or HTTPS. The server shouldn't
|
||||
require any additional HTTP-headers.
|
||||
|
||||
`INSERT` and `SELECT` queries are transformed into `POST` and `GET` requests
|
||||
respectively. For correct `POST`-requests handling the remote server should support
|
||||
[Chunked transfer encoding](https://ru.wikipedia.org/wiki/Chunked_transfer_encoding).
|
||||
|
||||
**Example:**
|
||||
|
||||
**1.** Create the `url_engine_table` table:
|
||||
|
||||
```sql
|
||||
CREATE TABLE url_engine_table (word String, value UInt64)
|
||||
ENGINE=URL('http://127.0.0.1:12345/', CSV)
|
||||
```
|
||||
|
||||
**2.** Implement simple http-server using python3:
|
||||
|
||||
```python3
|
||||
from http.server import BaseHTTPRequestHandler, HTTPServer
|
||||
|
||||
class CSVHTTPServer(BaseHTTPRequestHandler):
|
||||
def do_GET(self):
|
||||
self.send_response(200)
|
||||
self.send_header('Content-type', 'text/csv')
|
||||
self.end_headers()
|
||||
|
||||
self.wfile.write(bytes('Hello,1\nWorld,2\n', "utf-8"))
|
||||
|
||||
if __name__ == "__main__":
|
||||
server_address = ('127.0.0.1', 12345)
|
||||
HTTPServer(server_address, CSVHTTPServer).serve_forever()
|
||||
```
|
||||
|
||||
```bash
|
||||
python3 server.py
|
||||
```
|
||||
|
||||
**3.** Query the data:
|
||||
|
||||
```sql
|
||||
SELECT * FROM url_engine_table
|
||||
```
|
||||
|
||||
```text
|
||||
┌─word──┬─value─┐
|
||||
│ Hello │ 1 │
|
||||
│ World │ 2 │
|
||||
└───────┴───────┘
|
||||
```
|
||||
|
||||
|
||||
## Details of implementation
|
||||
|
||||
- Reads and writes can be parallel
|
||||
- Not supported:
|
||||
- `ALTER`
|
||||
- `SELECT ... SAMPLE`
|
||||
- Indices
|
||||
- Replication
|
19
docs/en/query_language/table_functions/url.md
Normal file
19
docs/en/query_language/table_functions/url.md
Normal file
@ -0,0 +1,19 @@
|
||||
<a name="table_functions-url"></a>
|
||||
|
||||
# url
|
||||
|
||||
`url(URL, format, structure)` - returns a table created from the `URL` with given
|
||||
`format` and `structure`.
|
||||
|
||||
URL - HTTP or HTTPS server address, which can accept `GET` and/or `POST` requests.
|
||||
|
||||
format - [format](../../interfaces/formats.md#formats) of the data.
|
||||
|
||||
structure - table structure in `'UserID UInt64, Name String'` format. Determines column names and types.
|
||||
|
||||
**Example**
|
||||
|
||||
```sql
|
||||
-- getting the first 3 lines of a table that contains columns of String and UInt32 type from HTTP-server which answers in CSV format.
|
||||
SELECT * FROM url('http://127.0.0.1:12345/', CSV, 'column1 String, column2 UInt32') LIMIT 3
|
||||
```
|
@ -66,7 +66,11 @@ struct Message {
|
||||
|
||||
При форматировании, строки выводятся в двойных кавычках. Двойная кавычка внутри строки выводится как две двойные кавычки подряд. Других правил экранирования нет. Даты и даты-с-временем выводятся в двойных кавычках. Числа выводятся без кавычек. Значения разделяются символом-разделителем*. Строки разделяются unix переводом строки (LF). Массивы сериализуются в CSV следующим образом: сначала массив сериализуется в строку, как в формате TabSeparated, а затем полученная строка выводится в CSV в двойных кавычках. Кортежи в формате CSV сериализуются, как отдельные столбцы (то есть, теряется их вложенность в кортеж).
|
||||
|
||||
*По умолчанию — `,`. См. настройку [format_csv_delimiter](/docs/ru/operations/settings/settings/#format_csv_delimiter) для дополнительной информации.
|
||||
```
|
||||
clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv
|
||||
```
|
||||
|
||||
*По умолчанию — `,`. См. настройку [format_csv_delimiter](/operations/settings/settings/#format_csv_delimiter) для дополнительной информации.
|
||||
|
||||
При парсинге, все значения могут парситься как в кавычках, так и без кавычек. Поддерживаются как двойные, так и одинарные кавычки. В том числе, строки могут быть расположены без кавычек - тогда они парсятся до символа-разделителя или перевода строки (CR или LF). В нарушение RFC, в случае парсинга строк не в кавычках, начальные и конечные пробелы и табы игнорируются. В качестве перевода строки, поддерживаются как Unix (LF), так и Windows (CR LF) и Mac OS Classic (LF CR) варианты.
|
||||
|
||||
|
@ -2,23 +2,23 @@
|
||||
|
||||
## По-настоящему столбцовая СУБД
|
||||
|
||||
В по-настоящему столбцовой СУБД рядом со значениями не хранится никакого "мусора". Например, должны поддерживаться значения постоянной длины, чтобы не хранить рядом со значениями типа "число" их длины. Для примера, миллиард значений типа UInt8 должен действительно занимать в несжатом виде около 1GB, иначе это сильно ударит по эффективности использования CPU. Очень важно хранить данные компактно (без "мусора") в том числе в несжатом виде, так как скорость разжатия (использование CPU) зависит, в основном, от объёма несжатых данных.
|
||||
В по-настоящему столбцовой СУБД рядом со значениями не хранится никаких лишних данных. Например, должны поддерживаться значения постоянной длины, чтобы не хранить рядом со значениями типа "число" их длины. Для примера, миллиард значений типа UInt8 должен действительно занимать в несжатом виде около 1GB, иначе это сильно ударит по эффективности использования CPU. Очень важно хранить данные компактно (без "мусора") в том числе в несжатом виде, так как скорость разжатия (использование CPU) зависит, в основном, от объёма несжатых данных.
|
||||
|
||||
Этот пункт пришлось выделить, так как существуют системы, которые могут хранить значения отдельных столбцов по отдельности, но не могут эффективно выполнять аналитические запросы в силу оптимизации под другой сценарий работы. Примеры: HBase, BigTable, Cassandra, HyperTable. В этих системах вы получите throughput в районе сотен тысяч строк в секунду, но не сотен миллионов строк в секунду.
|
||||
Этот пункт пришлось выделить, так как существуют системы, которые могут хранить значения отдельных столбцов по отдельности, но не могут эффективно выполнять аналитические запросы в силу оптимизации под другой сценарий работы. Примеры: HBase, BigTable, Cassandra, HyperTable. В этих системах вы получите пропускную способность в районе сотен тысяч строк в секунду, но не сотен миллионов строк в секунду.
|
||||
|
||||
Также стоит заметить, что ClickHouse является СУБД, а не одной базой данных. То есть, ClickHouse позволяет создавать таблицы и базы данных в runtime, загружать данные и выполнять запросы без переконфигурирования и перезапуска сервера.
|
||||
Также стоит заметить, что ClickHouse является системой управления базами данных, а не одной базой данных. То есть, ClickHouse позволяет создавать таблицы и базы данных в runtime, загружать данные и выполнять запросы без переконфигурирования и перезапуска сервера.
|
||||
|
||||
## Сжатие данных
|
||||
|
||||
Некоторые столбцовые СУБД (InfiniDB CE, MonetDB) не используют сжатие данных. Но сжатие данных действительно серьёзно увеличивает производительность.
|
||||
Некоторые столбцовые СУБД (InfiniDB CE, MonetDB) не используют сжатие данных. Однако сжатие данных действительно играет одну из ключевых ролей в демонстрации отличной производительности.
|
||||
|
||||
## Хранение данных на диске
|
||||
|
||||
Многие столбцовые СУБД (SAP HANA, Google PowerDrill) могут работать только в оперативке. Но оперативки (даже на тысячах серверах) слишком мало для хранения всех хитов и визитов в Яндекс.Метрике.
|
||||
Многие столбцовые СУБД (SAP HANA, Google PowerDrill) могут работать только в оперативной памяти. Такой подход стимулирует выделять больший бюджет на оборудование, чем фактически требуется для анализа в реальном времени. ClickHouse спроектирован для работы на обычных жестких дисках, что обеспечивает низкую стоимость хранения на гигабайт данных, но SSD b дополнительная оперативная память тоже полноценно используются, если доступны.
|
||||
|
||||
## Параллельная обработка запроса на многих процессорных ядрах
|
||||
|
||||
Большие запросы естественным образом распараллеливаются.
|
||||
Большие запросы естественным образом распараллеливаются, используя все необходимые ресурсы из доступных на сервере.
|
||||
|
||||
## Распределённая обработка запроса на многих серверах
|
||||
|
||||
@ -27,11 +27,9 @@
|
||||
|
||||
## Поддержка SQL
|
||||
|
||||
Если вы знаете, что такое стандартный SQL, то говорить о поддержке SQL всё-таки нельзя.
|
||||
Все функции названы по-другому.
|
||||
Тем не менее, это - декларативный язык запросов на основе SQL и во многих случаях не отличимый от SQL.
|
||||
Поддерживаются JOIN-ы. Поддерживаются подзапросы в секциях FROM, IN, JOIN, а также скалярные подзапросы.
|
||||
Зависимые подзапросы не поддерживаются.
|
||||
ClickHouse поддерживает декларативный язык запросов на основе SQL и во многих случаях совпадающий с SQL стандартом.
|
||||
Поддерживаются GROUP BY, ORDER BY, подзапросы в секциях FROM, IN, JOIN, а также скалярные подзапросы.
|
||||
Зависимые подзапросы и оконные функции не поддерживаются.
|
||||
|
||||
## Векторный движок
|
||||
|
||||
@ -41,21 +39,24 @@
|
||||
|
||||
ClickHouse поддерживает таблицы с первичным ключом. Для того, чтобы можно было быстро выполнять запросы по диапазону первичного ключа, данные инкрементально сортируются с помощью merge дерева. За счёт этого, поддерживается постоянное добавление данных в таблицу. Блокировки при добавлении данных отсутствуют.
|
||||
|
||||
## Наличие индексов
|
||||
## Наличие индекса
|
||||
|
||||
Наличие первичного ключа позволяет, например, вынимать данные для конкретных клиентов (счётчиков Метрики), для заданного диапазона времени, с низкими задержками - менее десятков миллисекунд.
|
||||
Физическая сортировка данных по первичному ключу позволяет получать данные для конкретных его значений или их диапазонов с низкими задержками - менее десятков миллисекунд.
|
||||
|
||||
## Подходит для онлайн запросов
|
||||
|
||||
Это позволяет использовать систему в качестве бэкенда для веб-интерфейса. Низкие задержки позволяют не откладывать выполнение запроса, а выполнять его в момент загрузки страницы интерфейса Яндекс.Метрики. То есть, в режиме онлайн.
|
||||
Низкие задержки позволяют не откладывать выполнение запроса и не подготавливать ответ заранее, а выполнять его именно в момент загрузки страницы пользовательского интерфейса. То есть, в режиме онлайн.
|
||||
|
||||
## Поддержка приближённых вычислений
|
||||
|
||||
ClickHouse предоставляет различные способы разменять точность вычислений на производительность:
|
||||
|
||||
1. Система содержит агрегатные функции для приближённого вычисления количества различных значений, медианы и квантилей.
|
||||
2. Поддерживается возможность выполнить запрос на основе части (выборки) данных и получить приближённый результат. При этом, с диска будет считано пропорционально меньше данных.
|
||||
3. Поддерживается возможность выполнить агрегацию не для всех ключей, а для ограниченного количества первых попавшихся ключей. При выполнении некоторых условий на распределение ключей в данных, это позволяет получить достаточно точный результат с использованием меньшего количества ресурсов.
|
||||
|
||||
## Репликация данных, поддержка целостности данных на репликах
|
||||
## Репликация данных и поддержка целостности
|
||||
|
||||
Используется асинхронная multimaster репликация. После записи на любую доступную реплику, данные распространяются на все остальные реплики в фоне. Система поддерживает полную идентичность данных на разных репликах. Восстановление после большинства сбоев осуществляется автоматически, а в сложных случаях — полуавтоматически.
|
||||
|
||||
Используется асинхронная multimaster репликация. После записи на любую доступную реплику, данные распространяются на все остальные реплики. Система поддерживает полную идентичность данных на разных репликах. Восстановление после сбоя осуществляется автоматически, а в сложных случаях - "по кнопке".
|
||||
Подробнее смотрите раздел [Репликация данных](../operations/table_engines/replication.md#table_engines-replication).
|
||||
|
@ -1,6 +1,6 @@
|
||||
# Особенности ClickHouse, которые могут считаться недостатками
|
||||
|
||||
1. Отсутствие транзакций.
|
||||
2. Необходимо, чтобы результат выполнения запроса, в случае агрегации, помещался в оперативку на одном сервере. Объём исходных данных для запроса, при этом, может быть сколь угодно большим.
|
||||
3. Отсутствие полноценной реализации UPDATE/DELETE.
|
||||
|
||||
1. Отсутствие полноценных транзакций.
|
||||
2. Возможность изменять или удалять ранее записанные данные с низкими задержками и высокой частотой запросов не предоставляется. Есть массовое удаление данных для очистки более не нужного или соответствия [GDPR](https://gdpr-info.eu). Массовое изменение данных находится в разработке (на момент июля 2018).
|
||||
3. Разреженный индекс делает ClickHouse плохо пригодным для точечных чтений одиночных строк по своим
|
||||
ключам.
|
||||
|
74
docs/ru/operations/table_engines/url.md
Normal file
74
docs/ru/operations/table_engines/url.md
Normal file
@ -0,0 +1,74 @@
|
||||
<a name="table_engines-url"></a>
|
||||
|
||||
# URL(URL, Format)
|
||||
|
||||
Управляет данными на удаленном HTTP/HTTPS сервере. Данный движок похож
|
||||
на движок [`File`](./file.md#).
|
||||
|
||||
## Использование движка в сервере ClickHouse
|
||||
|
||||
`Format` должен быть таким, который ClickHouse может использовать в запросах
|
||||
`SELECT` и, если есть необходимость, `INSERT`. Полный список поддерживаемых форматов смотрите в
|
||||
разделе [Форматы](../../interfaces/formats.md#formats).
|
||||
|
||||
`URL` должен соответствовать структуре Uniform Resource Locator. По указанному URL должен находится сервер
|
||||
работающий по протоколу HTTP или HTTPS. При этом не должно требоваться никаких
|
||||
дополнительных заголовков для получения ответа от сервера.
|
||||
|
||||
Запросы `INSERT` и `SELECT` транслируются в `POST` и `GET` запросы
|
||||
соответственно. Для обработки `POST`-запросов удаленный сервер должен поддерживать
|
||||
[Chunked transfer encoding](https://ru.wikipedia.org/wiki/Chunked_transfer_encoding).
|
||||
|
||||
**Пример:**
|
||||
|
||||
**1.** Создадим на сервере таблицу `url_engine_table`:
|
||||
|
||||
```sql
|
||||
CREATE TABLE url_engine_table (word String, value UInt64)
|
||||
ENGINE=URL('http://127.0.0.1:12345/', CSV)
|
||||
```
|
||||
|
||||
**2.** Создадим простейший http-сервер стандартными средствами языка python3 и
|
||||
запустим его:
|
||||
|
||||
```python3
|
||||
from http.server import BaseHTTPRequestHandler, HTTPServer
|
||||
|
||||
class CSVHTTPServer(BaseHTTPRequestHandler):
|
||||
def do_GET(self):
|
||||
self.send_response(200)
|
||||
self.send_header('Content-type', 'text/csv')
|
||||
self.end_headers()
|
||||
|
||||
self.wfile.write(bytes('Hello,1\nWorld,2\n', "utf-8"))
|
||||
|
||||
if __name__ == "__main__":
|
||||
server_address = ('127.0.0.1', 12345)
|
||||
HTTPServer(server_address, CSVHTTPServer).serve_forever()
|
||||
```
|
||||
|
||||
```bash
|
||||
python3 server.py
|
||||
```
|
||||
|
||||
**3.** Запросим данные:
|
||||
|
||||
```sql
|
||||
SELECT * FROM url_engine_table
|
||||
```
|
||||
|
||||
```text
|
||||
┌─word──┬─value─┐
|
||||
│ Hello │ 1 │
|
||||
│ World │ 2 │
|
||||
└───────┴───────┘
|
||||
```
|
||||
|
||||
## Особенности использования
|
||||
|
||||
- Поддерживается многопоточное чтение и запись.
|
||||
- Не поддерживается:
|
||||
- использование операций `ALTER` и `SELECT...SAMPLE`;
|
||||
- индексы;
|
||||
- репликация.
|
||||
|
20
docs/ru/query_language/table_functions/url.md
Normal file
20
docs/ru/query_language/table_functions/url.md
Normal file
@ -0,0 +1,20 @@
|
||||
<a name="table_functions-url"></a>
|
||||
|
||||
# url
|
||||
|
||||
`url(URL, format, structure)` - возвращает таблицу со столбцами, указанными в
|
||||
`structure`, созданную из данных находящихся по `URL` в формате `format`.
|
||||
|
||||
URL - адрес, по которому сервер принимает `GET` и/или `POST` запросы по
|
||||
протоколу HTTP или HTTPS.
|
||||
|
||||
format - [формат](../../interfaces/formats.md#formats) данных.
|
||||
|
||||
structure - структура таблицы в форме `'UserID UInt64, Name String'`. Определяет имена и типы столбцов.
|
||||
|
||||
**Пример**
|
||||
|
||||
```sql
|
||||
-- получение 3-х строк таблицы, состоящей из двух колонк типа String и UInt32 от сервера, отдающего данные в формате CSV
|
||||
SELECT * FROM url('http://127.0.0.1:12345/', CSV, 'column1 String, column2 UInt32') LIMIT 3
|
||||
```
|
@ -94,6 +94,7 @@ pages:
|
||||
- 'merge': 'query_language/table_functions/merge.md'
|
||||
- 'numbers': 'query_language/table_functions/numbers.md'
|
||||
- 'remote': 'query_language/table_functions/remote.md'
|
||||
- 'url': 'query_language/table_functions/url.md'
|
||||
- 'Dictionaries':
|
||||
- 'Introduction': 'query_language/dicts/index.md'
|
||||
- 'External dictionaries':
|
||||
@ -134,6 +135,7 @@ pages:
|
||||
- 'Null': 'operations/table_engines/null.md'
|
||||
- 'Set': 'operations/table_engines/set.md'
|
||||
- 'Join': 'operations/table_engines/join.md'
|
||||
- 'URL': 'operations/table_engines/url.md'
|
||||
- 'View': 'operations/table_engines/view.md'
|
||||
- 'MaterializedView': 'operations/table_engines/materializedview.md'
|
||||
- 'Integrations':
|
||||
|
@ -97,6 +97,7 @@ pages:
|
||||
- 'merge': 'query_language/table_functions/merge.md'
|
||||
- 'numbers': 'query_language/table_functions/numbers.md'
|
||||
- 'remote': 'query_language/table_functions/remote.md'
|
||||
- 'url': 'query_language/table_functions/url.md'
|
||||
- 'Словари':
|
||||
- 'Введение': 'query_language/dicts/index.md'
|
||||
- 'Внешние словари':
|
||||
@ -138,6 +139,7 @@ pages:
|
||||
- 'Null': 'operations/table_engines/null.md'
|
||||
- 'Set': 'operations/table_engines/set.md'
|
||||
- 'Join': 'operations/table_engines/join.md'
|
||||
- 'URL': 'operations/table_engines/url.md'
|
||||
- 'View': 'operations/table_engines/view.md'
|
||||
- 'MaterializedView': 'operations/table_engines/materializedview.md'
|
||||
- 'Интеграции':
|
||||
|
Loading…
Reference in New Issue
Block a user