mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge branch 'master' of github.com:ClickHouse/ClickHouse into clickhouse-copier
This commit is contained in:
commit
ce16a6e536
@ -5,7 +5,8 @@
|
||||
namespace DB
|
||||
{
|
||||
/// Contains info about all shards that contain a partition
|
||||
struct ClusterPartition {
|
||||
struct ClusterPartition
|
||||
{
|
||||
double elapsed_time_seconds = 0;
|
||||
UInt64 bytes_copied = 0;
|
||||
UInt64 rows_copied = 0;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include "Internals.h"
|
||||
|
||||
namespace DB {
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
|
||||
|
||||
|
@ -108,17 +108,20 @@ struct TaskStateWithOwner
|
||||
TaskState state{TaskState::Unknown};
|
||||
String owner;
|
||||
|
||||
static String getData(TaskState state, const String &owner) {
|
||||
static String getData(TaskState state, const String &owner)
|
||||
{
|
||||
return TaskStateWithOwner(state, owner).toString();
|
||||
}
|
||||
|
||||
String toString() {
|
||||
String toString()
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
wb << static_cast<UInt32>(state) << "\n" << escape << owner;
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
static TaskStateWithOwner fromString(const String &data) {
|
||||
static TaskStateWithOwner fromString(const String & data)
|
||||
{
|
||||
ReadBufferFromString rb(data);
|
||||
TaskStateWithOwner res;
|
||||
UInt32 state;
|
||||
@ -141,7 +144,8 @@ struct ShardPriority
|
||||
size_t hostname_difference = 0;
|
||||
UInt8 random = 0;
|
||||
|
||||
static bool greaterPriority(const ShardPriority & current, const ShardPriority & other) {
|
||||
static bool greaterPriority(const ShardPriority & current, const ShardPriority & other)
|
||||
{
|
||||
return std::forward_as_tuple(current.is_remote, current.hostname_difference, current.random)
|
||||
< std::forward_as_tuple(other.is_remote, other.hostname_difference, other.random);
|
||||
}
|
||||
|
@ -147,10 +147,11 @@ struct TaskTable {
|
||||
};
|
||||
|
||||
|
||||
struct TaskShard {
|
||||
struct TaskShard
|
||||
{
|
||||
TaskShard(TaskTable &parent, const ShardInfo &info_) : task_table(parent), info(info_) {}
|
||||
|
||||
TaskTable &task_table;
|
||||
TaskTable & task_table;
|
||||
|
||||
ShardInfo info;
|
||||
|
||||
@ -297,12 +298,14 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf
|
||||
}
|
||||
|
||||
template<typename RandomEngine>
|
||||
inline void TaskTable::initShards(RandomEngine && random_engine) {
|
||||
inline void TaskTable::initShards(RandomEngine && random_engine)
|
||||
{
|
||||
const String & fqdn_name = getFQDNOrHostName();
|
||||
std::uniform_int_distribution<UInt8> get_urand(0, std::numeric_limits<UInt8>::max());
|
||||
|
||||
// Compute the priority
|
||||
for (auto & shard_info : cluster_pull->getShardsInfo()) {
|
||||
for (auto & shard_info : cluster_pull->getShardsInfo())
|
||||
{
|
||||
TaskShardPtr task_shard = std::make_shared<TaskShard>(*this, shard_info);
|
||||
const auto & replicas = cluster_pull->getShardsAddresses().at(task_shard->indexInCluster());
|
||||
task_shard->priority = getReplicasPriority(replicas, fqdn_name, get_urand(random_engine));
|
||||
@ -312,13 +315,15 @@ inline void TaskTable::initShards(RandomEngine && random_engine) {
|
||||
|
||||
// Sort by priority
|
||||
std::sort(all_shards.begin(), all_shards.end(),
|
||||
[](const TaskShardPtr & lhs, const TaskShardPtr & rhs) {
|
||||
[](const TaskShardPtr & lhs, const TaskShardPtr & rhs)
|
||||
{
|
||||
return ShardPriority::greaterPriority(lhs->priority, rhs->priority);
|
||||
});
|
||||
|
||||
// Cut local shards
|
||||
auto it_first_remote = std::lower_bound(all_shards.begin(), all_shards.end(), 1,
|
||||
[](const TaskShardPtr & lhs, UInt8 is_remote) {
|
||||
[](const TaskShardPtr & lhs, UInt8 is_remote)
|
||||
{
|
||||
return lhs->priority.is_remote < is_remote;
|
||||
});
|
||||
|
||||
|
@ -403,15 +403,13 @@
|
||||
</text_log>
|
||||
-->
|
||||
|
||||
<!-- Uncomment to write metric log into table.
|
||||
Metric log contains rows with current values of ProfileEvents, CurrentMetrics collected with "collect_interval_milliseconds" interval.
|
||||
<!-- Metric log contains rows with current values of ProfileEvents, CurrentMetrics collected with "collect_interval_milliseconds" interval. -->
|
||||
<metric_log>
|
||||
<database>system</database>
|
||||
<table>metric_log</table>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
|
||||
</metric_log>
|
||||
-->
|
||||
|
||||
<!-- Parameters for embedded dictionaries, used in Yandex.Metrica.
|
||||
See https://clickhouse.yandex/docs/en/dicts/internal_dicts/
|
||||
|
@ -193,7 +193,7 @@ std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded
|
||||
{
|
||||
stream << "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
|
||||
<< ", e.displayText() = " << e.displayText()
|
||||
<< (with_stacktrace ? getExceptionStackTraceString(e) : "")
|
||||
<< (with_stacktrace ? ", Stack trace (when copying this message, always include the lines below):\n\n" + getExceptionStackTraceString(e) : "")
|
||||
<< (with_extra_info ? getExtraExceptionInfo(e) : "")
|
||||
<< " (version " << VERSION_STRING << VERSION_OFFICIAL << ")";
|
||||
}
|
||||
@ -210,9 +210,9 @@ std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded
|
||||
name += " (demangling status: " + toString(status) + ")";
|
||||
|
||||
stream << "std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", type: " << name << ", e.what() = " << e.what()
|
||||
<< (with_stacktrace ? getExceptionStackTraceString(e) : "")
|
||||
<< (with_stacktrace ? ", Stack trace (when copying this message, always include the lines below):\n\n" + getExceptionStackTraceString(e) : "")
|
||||
<< (with_extra_info ? getExtraExceptionInfo(e) : "")
|
||||
<< ", version = " << VERSION_STRING << VERSION_OFFICIAL;
|
||||
<< " (version " << VERSION_STRING << VERSION_OFFICIAL << ")";
|
||||
}
|
||||
catch (...) {}
|
||||
}
|
||||
|
@ -9,23 +9,12 @@
|
||||
#include <Compression/CompressionFactory.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event ReadCompressedBytes;
|
||||
extern const Event CompressedReadBufferBlocks;
|
||||
extern const Event CompressedReadBufferBytes;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CHECKSUM_DOESNT_MATCH;
|
||||
extern const int TOO_LARGE_SIZE_COMPRESSED;
|
||||
extern const int UNKNOWN_COMPRESSION_METHOD;
|
||||
extern const int CANNOT_DECOMPRESS;
|
||||
extern const int SEEK_POSITION_OUT_OF_BOUND;
|
||||
extern const int CORRUPTED_DATA;
|
||||
}
|
||||
|
||||
|
@ -124,6 +124,10 @@ public:
|
||||
{
|
||||
/// leave other comparisons as is
|
||||
}
|
||||
else if (functionIsLikeOperator(node.name)) /// LIKE, NOT LIKE
|
||||
{
|
||||
/// leave as is
|
||||
}
|
||||
else if (functionIsInOperator(node.name)) /// IN, NOT IN
|
||||
{
|
||||
if (auto ident = node.arguments->children.at(0)->as<ASTIdentifier>())
|
||||
|
@ -13,4 +13,9 @@ inline bool functionIsInOrGlobalInOperator(const std::string & name)
|
||||
return functionIsInOperator(name) || name == "globalIn" || name == "globalNotIn";
|
||||
}
|
||||
|
||||
inline bool functionIsLikeOperator(const std::string & name)
|
||||
{
|
||||
return name == "like" || name == "notLike";
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -264,6 +264,78 @@ const KeyCondition::AtomMap KeyCondition::atom_map
|
||||
};
|
||||
|
||||
|
||||
static const std::map<std::string, std::string> inverse_relations = {
|
||||
{"equals", "notEquals"},
|
||||
{"notEquals", "equals"},
|
||||
{"less", "greaterOrEquals"},
|
||||
{"greaterOrEquals", "less"},
|
||||
{"greater", "lessOrEquals"},
|
||||
{"lessOrEquals", "greater"},
|
||||
{"in", "notIn"},
|
||||
{"notIn", "in"},
|
||||
{"like", "notLike"},
|
||||
{"notLike", "like"},
|
||||
{"empty", "notEmpty"},
|
||||
{"notEmpty", "empty"},
|
||||
};
|
||||
|
||||
|
||||
bool isLogicalOperator(const String & func_name)
|
||||
{
|
||||
return (func_name == "and" || func_name == "or" || func_name == "not" || func_name == "indexHint");
|
||||
}
|
||||
|
||||
/// The node can be one of:
|
||||
/// - Logical operator (AND, OR, NOT and indexHint() - logical NOOP)
|
||||
/// - An "atom" (relational operator, constant, expression)
|
||||
/// - A logical constant expression
|
||||
/// - Any other function
|
||||
ASTPtr cloneASTWithInversionPushDown(const ASTPtr node, const bool need_inversion = false)
|
||||
{
|
||||
const ASTFunction * func = node->as<ASTFunction>();
|
||||
|
||||
if (func && isLogicalOperator(func->name))
|
||||
{
|
||||
if (func->name == "not")
|
||||
{
|
||||
return cloneASTWithInversionPushDown(func->arguments->children.front(), !need_inversion);
|
||||
}
|
||||
|
||||
const auto result_node = makeASTFunction(func->name);
|
||||
|
||||
/// indexHint() is a special case - logical NOOP function
|
||||
if (result_node->name != "indexHint" && need_inversion)
|
||||
{
|
||||
result_node->name = (result_node->name == "and") ? "or" : "and";
|
||||
}
|
||||
|
||||
if (func->arguments)
|
||||
{
|
||||
for (const auto & child : func->arguments->children)
|
||||
{
|
||||
result_node->arguments->children.push_back(cloneASTWithInversionPushDown(child, need_inversion));
|
||||
}
|
||||
}
|
||||
|
||||
return result_node;
|
||||
}
|
||||
|
||||
const auto cloned_node = node->clone();
|
||||
|
||||
if (func && inverse_relations.find(func->name) != inverse_relations.cend())
|
||||
{
|
||||
if (need_inversion)
|
||||
{
|
||||
cloned_node->as<ASTFunction>()->name = inverse_relations.at(func->name);
|
||||
}
|
||||
|
||||
return cloned_node;
|
||||
}
|
||||
|
||||
return need_inversion ? makeASTFunction("not", cloned_node) : cloned_node;
|
||||
}
|
||||
|
||||
|
||||
inline bool Range::equals(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateEquals(), lhs, rhs); }
|
||||
inline bool Range::less(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateLess(), lhs, rhs); }
|
||||
|
||||
@ -345,21 +417,23 @@ KeyCondition::KeyCondition(
|
||||
*/
|
||||
Block block_with_constants = getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context);
|
||||
|
||||
/// Trasform WHERE section to Reverse Polish notation
|
||||
const auto & select = query_info.query->as<ASTSelectQuery &>();
|
||||
if (select.where())
|
||||
const ASTSelectQuery & select = query_info.query->as<ASTSelectQuery &>();
|
||||
if (select.where() || select.prewhere())
|
||||
{
|
||||
traverseAST(select.where(), context, block_with_constants);
|
||||
ASTPtr filter_query;
|
||||
if (select.where() && select.prewhere())
|
||||
filter_query = makeASTFunction("and", select.where(), select.prewhere());
|
||||
else
|
||||
filter_query = select.where() ? select.where() : select.prewhere();
|
||||
|
||||
if (select.prewhere())
|
||||
{
|
||||
traverseAST(select.prewhere(), context, block_with_constants);
|
||||
rpn.emplace_back(RPNElement::FUNCTION_AND);
|
||||
}
|
||||
}
|
||||
else if (select.prewhere())
|
||||
{
|
||||
traverseAST(select.prewhere(), context, block_with_constants);
|
||||
/** When non-strictly monotonic functions are employed in functional index (e.g. ORDER BY toStartOfHour(dateTime)),
|
||||
* the use of NOT operator in predicate will result in the indexing algorithm leave out some data.
|
||||
* This is caused by rewriting in KeyCondition::tryParseAtomFromAST of relational operators to less strict
|
||||
* when parsing the AST into internal RPN representation.
|
||||
* To overcome the problem, before parsing the AST we transform it to its semantically equivalent form where all NOT's
|
||||
* are pushed down and applied (when possible) to leaf nodes.
|
||||
*/
|
||||
traverseAST(cloneASTWithInversionPushDown(filter_query), context, block_with_constants);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -432,9 +506,9 @@ void KeyCondition::traverseAST(const ASTPtr & node, const Context & context, Blo
|
||||
{
|
||||
RPNElement element;
|
||||
|
||||
if (auto * func = node->as<ASTFunction>())
|
||||
if (const auto * func = node->as<ASTFunction>())
|
||||
{
|
||||
if (operatorFromAST(func, element))
|
||||
if (tryParseLogicalOperatorFromAST(func, element))
|
||||
{
|
||||
auto & args = func->arguments->children;
|
||||
for (size_t i = 0, size = args.size(); i < size; ++i)
|
||||
@ -452,7 +526,7 @@ void KeyCondition::traverseAST(const ASTPtr & node, const Context & context, Blo
|
||||
}
|
||||
}
|
||||
|
||||
if (!atomFromAST(node, context, block_with_constants, element))
|
||||
if (!tryParseAtomFromAST(node, context, block_with_constants, element))
|
||||
{
|
||||
element.function = RPNElement::FUNCTION_UNKNOWN;
|
||||
}
|
||||
@ -680,7 +754,7 @@ static void castValueToType(const DataTypePtr & desired_type, Field & src_value,
|
||||
}
|
||||
|
||||
|
||||
bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out)
|
||||
bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out)
|
||||
{
|
||||
/** Functions < > = != <= >= in `notIn`, where one argument is a constant, and the other is one of columns of key,
|
||||
* or itself, wrapped in a chain of possibly-monotonic functions,
|
||||
@ -768,7 +842,9 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo
|
||||
func_name = "lessOrEquals";
|
||||
else if (func_name == "lessOrEquals")
|
||||
func_name = "greaterOrEquals";
|
||||
else if (func_name == "in" || func_name == "notIn" || func_name == "like")
|
||||
else if (func_name == "in" || func_name == "notIn" ||
|
||||
func_name == "like" || func_name == "notLike" ||
|
||||
func_name == "startsWith")
|
||||
{
|
||||
/// "const IN data_column" doesn't make sense (unlike "data_column IN const")
|
||||
return false;
|
||||
@ -809,7 +885,7 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo
|
||||
return false;
|
||||
}
|
||||
|
||||
bool KeyCondition::operatorFromAST(const ASTFunction * func, RPNElement & out)
|
||||
bool KeyCondition::tryParseLogicalOperatorFromAST(const ASTFunction * func, RPNElement & out)
|
||||
{
|
||||
/// Functions AND, OR, NOT.
|
||||
/** Also a special function `indexHint` - works as if instead of calling a function there are just parentheses
|
||||
|
@ -369,8 +369,8 @@ private:
|
||||
BoolMask initial_mask) const;
|
||||
|
||||
void traverseAST(const ASTPtr & node, const Context & context, Block & block_with_constants);
|
||||
bool atomFromAST(const ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out);
|
||||
bool operatorFromAST(const ASTFunction * func, RPNElement & out);
|
||||
bool tryParseAtomFromAST(const ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out);
|
||||
bool tryParseLogicalOperatorFromAST(const ASTFunction * func, RPNElement & out);
|
||||
|
||||
/** Is node the key column
|
||||
* or expression in which column of key is wrapped by chain of functions,
|
||||
|
@ -0,0 +1,3 @@
|
||||
SELECT \n k, \n r.k, \n name\nFROM n\nALL INNER JOIN r ON k = r.k\nWHERE (k = r.k) AND (name = \'A\')
|
||||
SELECT \n k, \n r.k, \n name\nFROM n\nALL INNER JOIN r ON k = r.k\nWHERE (k = r.k) AND (name LIKE \'A%\')
|
||||
SELECT \n k, \n r.k, \n name\nFROM n\nALL INNER JOIN r ON k = r.k\nWHERE (k = r.k) AND (name NOT LIKE \'A%\')
|
@ -0,0 +1,15 @@
|
||||
DROP TABLE IF EXISTS n;
|
||||
DROP TABLE IF EXISTS r;
|
||||
|
||||
CREATE TABLE n (k UInt32) ENGINE = Memory;
|
||||
CREATE TABLE r (k UInt32, name String) ENGINE = Memory;
|
||||
|
||||
SET enable_debug_queries = 1;
|
||||
SET enable_optimize_predicate_expression = 0;
|
||||
|
||||
ANALYZE SELECT * FROM n, r WHERE n.k = r.k AND r.name = 'A';
|
||||
ANALYZE SELECT * FROM n, r WHERE n.k = r.k AND r.name LIKE 'A%';
|
||||
ANALYZE SELECT * FROM n, r WHERE n.k = r.k AND r.name NOT LIKE 'A%';
|
||||
|
||||
DROP TABLE n;
|
||||
DROP TABLE r;
|
@ -0,0 +1,33 @@
|
||||
TP1
|
||||
7.51
|
||||
7.42
|
||||
7.41
|
||||
7.42
|
||||
7.41
|
||||
7.42
|
||||
7.41
|
||||
7.42
|
||||
7.41
|
||||
7.51
|
||||
TP2
|
||||
7.42
|
||||
7.41
|
||||
7.42
|
||||
7.51
|
||||
7.42
|
||||
7.41
|
||||
7.51
|
||||
7.51
|
||||
TP3
|
||||
7.42
|
||||
7.41
|
||||
7.51
|
||||
TP4
|
||||
7.42
|
||||
7.41
|
||||
7.42
|
||||
7.42
|
||||
7.41
|
||||
TP5
|
||||
7.41
|
||||
7.51
|
@ -0,0 +1,33 @@
|
||||
SET max_threads = 1;
|
||||
|
||||
CREATE TABLE IF NOT EXISTS functional_index_mergetree (x Float64) ENGINE = MergeTree ORDER BY round(x);
|
||||
INSERT INTO functional_index_mergetree VALUES (7.42)(7.41)(7.51);
|
||||
|
||||
SELECT 'TP1';
|
||||
SELECT * FROM functional_index_mergetree WHERE x > 7.42;
|
||||
SELECT * FROM functional_index_mergetree WHERE x < 7.49;
|
||||
SELECT * FROM functional_index_mergetree WHERE x < 7.5;
|
||||
|
||||
SELECT * FROM functional_index_mergetree WHERE NOT (NOT x < 7.49);
|
||||
SELECT * FROM functional_index_mergetree WHERE NOT (NOT x < 7.5);
|
||||
SELECT * FROM functional_index_mergetree WHERE NOT (NOT x > 7.42);
|
||||
|
||||
SELECT 'TP2';
|
||||
SELECT * FROM functional_index_mergetree WHERE NOT x > 7.49;
|
||||
SELECT * FROM functional_index_mergetree WHERE NOT x < 7.42;
|
||||
SELECT * FROM functional_index_mergetree WHERE NOT x < 7.41;
|
||||
SELECT * FROM functional_index_mergetree WHERE NOT x < 7.5;
|
||||
|
||||
SELECT 'TP3';
|
||||
SELECT * FROM functional_index_mergetree WHERE x > 7.41 AND x < 7.51;
|
||||
SELECT * FROM functional_index_mergetree WHERE NOT (x > 7.41 AND x < 7.51);
|
||||
|
||||
SELECT 'TP4';
|
||||
SELECT * FROM functional_index_mergetree WHERE NOT x < 7.41 AND NOT x > 7.49;
|
||||
SELECT * FROM functional_index_mergetree WHERE NOT x < 7.42 AND NOT x > 7.42;
|
||||
SELECT * FROM functional_index_mergetree WHERE (NOT x < 7.4) AND (NOT x > 7.49);
|
||||
|
||||
SELECT 'TP5';
|
||||
SELECT * FROM functional_index_mergetree WHERE NOT or(NOT x, toUInt64(x) AND NOT floor(x) > 6, x >= 7.42 AND round(x) <= 7);
|
||||
|
||||
DROP TABLE functional_index_mergetree;
|
16
docs/en/interfaces/third-party/gui.md
vendored
16
docs/en/interfaces/third-party/gui.md
vendored
@ -101,6 +101,22 @@ Features:
|
||||
- Refactorings.
|
||||
- Search and Navigation.
|
||||
|
||||
### Yandex DataLens {#yandex-datalens}
|
||||
|
||||
[Yandex DataLens](https://cloud.yandex.ru/services/datalens) is a service of data visualization and analytics.
|
||||
|
||||
Features:
|
||||
|
||||
- Wide range of available visualizations, from simple bar charts to complex dashboards.
|
||||
- Dashboards could be made publicly available.
|
||||
- Support for multiple data sources including ClickHouse.
|
||||
- Storage for materialized data based on ClickHouse.
|
||||
|
||||
DataLens is [available for free](https://cloud.yandex.com/docs/datalens/pricing) for low-load projects, even for commercial use.
|
||||
|
||||
- [DataLens documentation](https://cloud.yandex.com/docs/datalens/).
|
||||
- [Tutorial](https://cloud.yandex.com/docs/datalens/solutions/data-from-db-visualization) on visualizing data from a ClickHouse database.
|
||||
|
||||
### Holistics Software
|
||||
|
||||
[Holistics](https://www.holistics.io/) is a full-stack data platform and business intelligence tool.
|
||||
|
@ -275,8 +275,6 @@ Views look the same as normal tables. For example, they are listed in the result
|
||||
|
||||
There isn't a separate query for deleting views. To delete a view, use `DROP TABLE`.
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/query_language/create/) <!--hide-->
|
||||
|
||||
## CREATE DICTIONARY {#create-dictionary-query}
|
||||
|
||||
```sql
|
||||
@ -300,3 +298,5 @@ External dictionary structure consists of attributes. Dictionary attributes are
|
||||
Depending on dictionary [layout](dicts/external_dicts_dict_layout.md) one or more attributes can be specified as dictionary keys.
|
||||
|
||||
For more information, see [External Dictionaries](dicts/external_dicts.md) section.
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/query_language/create/) <!--hide-->
|
||||
|
@ -6,7 +6,7 @@ ClickHouse:
|
||||
|
||||
- Fully or partially stores dictionaries in RAM.
|
||||
- Periodically updates dictionaries and dynamically loads missing values. In other words, dictionaries can be loaded dynamically.
|
||||
- Allows to create external dictionaries with xml-files or [DDL queries](../create.md#create-dictionary-query).
|
||||
- Allows to create external dictionaries with xml files or [DDL queries](../create.md#create-dictionary-query).
|
||||
|
||||
The configuration of external dictionaries can be located in one or more xml-files. The path to the configuration is specified in the [dictionaries_config](../../operations/server_settings/settings.md#server_settings-dictionaries_config) parameter.
|
||||
|
||||
@ -34,12 +34,16 @@ You can [configure](external_dicts_dict.md) any number of dictionaries in the sa
|
||||
|
||||
[DDL queries for dictionaries](../create.md#create-dictionary-query) doesn't require any additional records in server configuration. They allow to work with dictionaries as first-class entities, like tables or views.
|
||||
|
||||
!!! attention
|
||||
!!! attention "Attention"
|
||||
You can convert values for a small dictionary by describing it in a `SELECT` query (see the [transform](../functions/other_functions.md) function). This functionality is not related to external dictionaries.
|
||||
|
||||
**See also**
|
||||
## See also {#ext-dicts-see-also}
|
||||
|
||||
- [Configuring an External Dictionary](external_dicts_dict.md)
|
||||
- [Storing Dictionaries in Memory](external_dicts_dict_layout.md)
|
||||
- [Dictionary Updates](external_dicts_dict_lifetime.md)
|
||||
- [Sources of External Dictionaries](external_dicts_dict_sources.md)
|
||||
- [Dictionary Key and Fields](external_dicts_dict_structure.md)
|
||||
- [Functions for Working with External Dictionaries](../functions/ext_dict_functions.md)
|
||||
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/query_language/dicts/external_dicts/) <!--hide-->
|
||||
|
@ -1,6 +1,6 @@
|
||||
# Configuring an External Dictionary {#dicts-external_dicts_dict}
|
||||
|
||||
If dictionary is configured using xml-file, than dictionary configuration has the following structure:
|
||||
If dictionary is configured using xml file, than dictionary configuration has the following structure:
|
||||
|
||||
```xml
|
||||
<dictionary>
|
||||
@ -37,7 +37,7 @@ LAYOUT(...) -- Memory layout configuration
|
||||
LIFETIME(...) -- Lifetime of dictionary in memory
|
||||
```
|
||||
|
||||
- name – The identifier that can be used to access the dictionary. Use the characters `[a-zA-Z0-9_\-]`.
|
||||
- `name` – The identifier that can be used to access the dictionary. Use the characters `[a-zA-Z0-9_\-]`.
|
||||
- [source](external_dicts_dict_sources.md) — Source of the dictionary.
|
||||
- [layout](external_dicts_dict_layout.md) — Dictionary layout in memory.
|
||||
- [structure](external_dicts_dict_structure.md) — Structure of the dictionary . A key and attributes that can be retrieved by this key.
|
||||
|
@ -34,7 +34,7 @@ The configuration looks like this:
|
||||
</yandex>
|
||||
```
|
||||
|
||||
in case of [DDL-query](../create.md#create-dictionary-query), equal configuration will looks like
|
||||
Corresponding [DDL-query](../create.md#create-dictionary-query):
|
||||
|
||||
```sql
|
||||
CREATE DICTIONARY (...)
|
||||
|
@ -47,10 +47,14 @@ Attributes are described in the query body:
|
||||
|
||||
ClickHouse supports the following types of keys:
|
||||
|
||||
- Numeric key. UInt64. Defined in the `<id>` tag or using `PRIMARY KEY` keyword.
|
||||
- Numeric key. `UInt64`. Defined in the `<id>` tag or using `PRIMARY KEY` keyword.
|
||||
- Composite key. Set of values of different types. Defined in the tag `<key>` or `PRIMARY KEY` keyword.
|
||||
|
||||
A xml structure can contain either `<id>` or `<key>`. DDL-query must contain single `PRIMARY KEY`.
|
||||
An xml structure can contain either `<id>` or `<key>`. DDL-query must contain single `PRIMARY KEY`.
|
||||
|
||||
!!! warning "Warning"
|
||||
You must not describe key as an attribute.
|
||||
|
||||
|
||||
### Numeric Key {#ext_dict-numeric-key}
|
||||
|
||||
|
@ -81,7 +81,7 @@ SELECT name FROM system.dictionaries WHERE database = <db> [AND name LIKE <patte
|
||||
|
||||
**Example**
|
||||
|
||||
The following query selects the first two rows from the list of tables in the `system` database, whose names contain `co`.
|
||||
The following query selects the first two rows from the list of tables in the `system` database, whose names contain `reg`.
|
||||
|
||||
```sql
|
||||
SHOW DICTIONARIES FROM db LIKE '%reg%' LIMIT 2
|
||||
@ -92,3 +92,5 @@ SHOW DICTIONARIES FROM db LIKE '%reg%' LIMIT 2
|
||||
│ region_names │
|
||||
└──────────────┘
|
||||
```
|
||||
|
||||
[Original article](https://clickhouse.tech/docs/en/query_language/show/) <!--hide-->
|
||||
|
@ -1,25 +0,0 @@
|
||||
# パフォーマンス
|
||||
|
||||
Yandexの内部テスト結果によると、ClickHouseは、テスト可能なクラスのシステム間で同等の動作シナリオで最高のパフォーマンス(長時間のクエリで最も高いスループットと、短時間のクエリで最小のレイテンシの両方)を示します。 [別のページで](https://clickhouse.yandex/benchmark.html)テスト結果を表示できます 。
|
||||
|
||||
これは、多数の独立したベンチマークでも確認されています。インターネット検索で見つけることは難しくありませんし、 [ 私達がまとめた関連リンク集 ](https://clickhouse.yandex/#independent-benchmarks) から見つけることもできます。
|
||||
|
||||
## 単一の巨大なクエリのスループット
|
||||
|
||||
スループットは、1秒あたりの行数または1秒あたりのメガバイト数で測定できます。データがページキャッシュに配置される場合、モダンなハードウェアで実行される、それほど複雑でないクエリは、単一サーバ上の非圧縮データに対し 約2〜10GB/秒 の速度で処理されます (最も単純な場合、速度は30GB/秒)。データがページキャッシュに配置されない場合、速度はディスクサブシステムとデータ圧縮率に依存します。たとえば、ディスクサブシステムが400 MB /秒でのデータの読み取りを許可し、データ圧縮率が3の場合、速度は約1.2 GB /秒になります。 1秒あたりの行数で速度を計算するには、1秒あたりのバイト数での速度をクエリで使用される列の合計サイズで除算します。たとえば、10バイトの列が抽出される場合、速度は1秒あたり約1億から2億行になります。
|
||||
|
||||
分散処理の場合、処理速度はほぼ線形に向上しますが、これは集約または並べ替えの結果として生じる行の数があまり大きくない場合のみです。
|
||||
|
||||
## 短いクエリを処理するときのレイテンシ
|
||||
|
||||
クエリが主キーを使用し、処理する行数がそれほど多くなく(数十万)、列数も多くない場合、データがページキャッシュにあれば50ミリ秒未満のレイテンシ(最良の場合は1桁のミリ秒)が期待できます。それ以外の場合、レイテンシはシーク数から計算されます。ディスクドライブを使用する場合、過負荷になっていないシステムの場合、レイテンシは次の式で計算されます: シーク時間(10ミリ秒) * クエリされる列の数 * データ部分の数
|
||||
|
||||
## 大量の短いクエリを処理するときのスループット
|
||||
|
||||
同じ条件下で、ClickHouseは1台のサーバーで毎秒数百のクエリを処理できます(最良の場合は数千まで)。このシナリオは分析DBMSでは一般的ではないため、1秒あたり最大100クエリを想定することをお勧めします。
|
||||
|
||||
## データ挿入時のパフォーマンス
|
||||
|
||||
少なくとも1000行のパケットにデータを挿入することをお勧めします。または、1秒あたり1回のリクエストを超えないでください。タブ区切りのダンプデータをMergeTreeテーブルに挿入する場合、挿入速度は50〜200MB/sになります。挿入された行のサイズが約1Kbの場合、速度は毎秒50,000〜200,000行になります。行が小さい場合、パフォーマンスは1秒あたりの行数で高くなります(Banner System データ- `>` 500,000行/秒、Graphite データ- `>` 1,000,000行/秒)。パフォーマンスを向上させるために、複数のINSERTクエリを並行して作成することで、パフォーマンスを線形に向上できます。
|
||||
|
||||
[Original article](https://clickhouse.yandex/docs/en/introduction/performance/) <!--hide-->
|
25
docs/ja/introduction/performance.md
Normal file
25
docs/ja/introduction/performance.md
Normal file
@ -0,0 +1,25 @@
|
||||
# パフォーマンス
|
||||
|
||||
Yandexの内部テスト結果によると、ClickHouseは、テスト可能なクラスのシステム間で同等の動作シナリオで最高のパフォーマンス(長時間のクエリで最も高いスループットと、短時間のクエリで最小のレイテンシの両方)を示します。 [別のページで](https://clickhouse.yandex/benchmark.html)テスト結果を表示できます 。
|
||||
|
||||
これは、多数の独立したベンチマークでも確認されています。インターネット検索で見つけることは難しくありませんし、 [ 私達がまとめた関連リンク集 ](https://clickhouse.yandex/#independent-benchmarks) から見つけることもできます。
|
||||
|
||||
## 単一の巨大なクエリのスループット
|
||||
|
||||
スループットは、1秒あたりの行数または1秒あたりのメガバイト数で測定できます。データがページキャッシュに配置される場合、モダンなハードウェアで実行される、それほど複雑でないクエリは、単一サーバ上の非圧縮データに対し 約2〜10GB/秒 の速度で処理されます (最も単純な場合、速度は30GB/秒)。データがページキャッシュに配置されない場合、速度はディスクサブシステムとデータ圧縮率に依存します。たとえば、ディスクサブシステムが400 MB /秒でのデータの読み取りを許可し、データ圧縮率が3の場合、速度は約1.2 GB /秒になります。 1秒あたりの行数で速度を計算するには、1秒あたりのバイト数での速度をクエリで使用される列の合計サイズで除算します。たとえば、10バイトの列が抽出される場合、速度は1秒あたり約1億から2億行になります。
|
||||
|
||||
分散処理の場合、処理速度はほぼ線形に向上しますが、これは集約または並べ替えの結果として生じる行の数があまり大きくない場合のみです。
|
||||
|
||||
## 短いクエリを処理するときのレイテンシ
|
||||
|
||||
クエリが主キーを使用し、処理する行数がそれほど多くなく(数十万)、列数も多くない場合、データがページキャッシュにあれば50ミリ秒未満のレイテンシ(最良の場合は1桁のミリ秒)が期待できます。それ以外の場合、レイテンシはシーク数から計算されます。ディスクドライブを使用する場合、過負荷になっていないシステムの場合、レイテンシは次の式で計算されます: シーク時間(10ミリ秒) * クエリされる列の数 * データ部分の数
|
||||
|
||||
## 大量の短いクエリを処理するときのスループット
|
||||
|
||||
同じ条件下で、ClickHouseは1台のサーバーで毎秒数百のクエリを処理できます(最良の場合は数千まで)。このシナリオは分析DBMSでは一般的ではないため、1秒あたり最大100クエリを想定することをお勧めします。
|
||||
|
||||
## データ挿入時のパフォーマンス
|
||||
|
||||
少なくとも1000行のパケットにデータを挿入することをお勧めします。または、1秒あたり1回のリクエストを超えないでください。タブ区切りのダンプデータをMergeTreeテーブルに挿入する場合、挿入速度は50〜200MB/sになります。挿入された行のサイズが約1Kbの場合、速度は毎秒50,000〜200,000行になります。行が小さい場合、パフォーマンスは1秒あたりの行数で高くなります(Banner System データ- `>` 500,000行/秒、Graphite データ- `>` 1,000,000行/秒)。パフォーマンスを向上させるために、複数のINSERTクエリを並行して作成することで、パフォーマンスを線形に向上できます。
|
||||
|
||||
[Original article](https://clickhouse.yandex/docs/ja/introduction/performance/) <!--hide-->
|
16
docs/ru/interfaces/third-party/gui.md
vendored
16
docs/ru/interfaces/third-party/gui.md
vendored
@ -104,6 +104,22 @@
|
||||
- Рефакторинги.
|
||||
- Поиск и навигация.
|
||||
|
||||
### Yandex DataLens {#yandex-datalens}
|
||||
|
||||
[Yandex DataLens](https://cloud.yandex.ru/services/datalens) — cервис визуализации и анализа данных.
|
||||
|
||||
Основные возможности:
|
||||
|
||||
- Широкий выбор инструментов визуализации, от простых столбчатых диаграмм до сложных дашбордов.
|
||||
- Возможность опубликовать дашборды на широкую аудиторию.
|
||||
- Поддержка множества источников данных, включая ClickHouse.
|
||||
- Хранение материализованных данных в кластере ClickHouse DataLens.
|
||||
|
||||
Для небольших проектов DataLens [доступен бесплатно](https://cloud.yandex.ru/docs/datalens/pricing), в том числе и для коммерческого использования.
|
||||
|
||||
- [Документация DataLens](https://cloud.yandex.ru/docs/datalens/).
|
||||
- [Пособие по визуализации данных из ClickHouse](https://cloud.yandex.ru/docs/solutions/datalens/data-from-ch-visualization).
|
||||
|
||||
### Holistics Software
|
||||
|
||||
[Holistics](https://www.holistics.io/) — full-stack платформа для обработки данных и бизнес-аналитики.
|
||||
|
@ -274,4 +274,28 @@ SELECT a, b, c FROM (SELECT ...)
|
||||
|
||||
Отсутствует отдельный запрос для удаления представлений. Чтобы удалить представление, следует использовать `DROP TABLE`.
|
||||
|
||||
## CREATE DICTIONARY {#create-dictionary-query}
|
||||
|
||||
```sql
|
||||
CREATE DICTIONARY [IF NOT EXISTS] [db.]dictionary_name
|
||||
(
|
||||
key1 type1 [DEFAULT|EXPRESSION expr1] [HIERARCHICAL|INJECTIVE|IS_OBJECT_ID],
|
||||
key2 type2 [DEFAULT|EXPRESSION expr2] [HIERARCHICAL|INJECTIVE|IS_OBJECT_ID],
|
||||
attr1 type2 [DEFAULT|EXPRESSION expr3],
|
||||
attr2 type2 [DEFAULT|EXPRESSION expr4]
|
||||
)
|
||||
PRIMARY KEY key1, key2
|
||||
SOURCE(SOURCE_NAME([param1 value1 ... paramN valueN]))
|
||||
LAYOUT(LAYOUT_NAME([param_name param_value]))
|
||||
LIFETIME([MIN val1] MAX val2)
|
||||
```
|
||||
|
||||
Создаёт [внешний словарь](dicts/external_dicts.md) с заданной [структурой](dicts/external_dicts_dict_structure.md), [источником](dicts/external_dicts_dict_sources.md), [способом размещения в памяти](dicts/external_dicts_dict_layout.md) и [периодом обновления](dicts/external_dicts_dict_lifetime.md).
|
||||
|
||||
Структура внешнего словаря состоит из атрибутов. Атрибуты словаря задаются как столбцы таблицы. Единственным обязательным свойством атрибута является его тип, все остальные свойства могут иметь значения по умолчанию.
|
||||
|
||||
В зависимости от [способа размещения словаря в памяти](dicts/external_dicts_dict_layout.md), ключами словаря могут быть один и более атрибутов.
|
||||
|
||||
Смотрите [Внешние словари](dicts/external_dicts.md).
|
||||
|
||||
[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/create/) <!--hide-->
|
||||
|
@ -2,9 +2,12 @@
|
||||
|
||||
Существует возможность подключать собственные словари из различных источников данных. Источником данных для словаря может быть локальный текстовый/исполняемый файл, HTTP(s) ресурс или другая СУБД. Подробнее смотрите в разделе "[Источники внешних словарей](external_dicts_dict_sources.md)".
|
||||
|
||||
ClickHouse полностью или частично хранит словари в оперативной памяти. Словари можно подгружать динамически, ClickHouse периодически обновляет их и динамически подгружает отсутствующие значения.
|
||||
ClickHouse:
|
||||
- Полностью или частично хранит словари в оперативной памяти.
|
||||
- Периодически обновляет их и динамически подгружает отсутствующие значения.
|
||||
- Позволяет создавать внешние словари с помощью xml-файлов или [DDL-запросов](../create.md#create-dictionary-query).
|
||||
|
||||
Конфигурация внешних словарей находится в одном или нескольких файлах. Путь к конфигурации указывается в параметре [dictionaries_config](../../operations/server_settings/settings.md).
|
||||
Конфигурация внешних словарей может находится в одном или нескольких xml-файлах. Путь к конфигурации указывается в параметре [dictionaries_config](../../operations/server_settings/settings.md).
|
||||
|
||||
Словари могут загружаться при старте сервера или при первом использовании, в зависимости от настройки [dictionaries_lazy_load](../../operations/server_settings/settings.md).
|
||||
|
||||
@ -30,12 +33,15 @@ ClickHouse полностью или частично хранит словар
|
||||
</yandex>
|
||||
```
|
||||
|
||||
В одном файле можно [сконфигурировать](external_dicts_dict.md) произвольное количество словарей. Формат файла сохраняется даже если словарь один (т.е. `<yandex><dictionary> <!--configuration--> </dictionary></yandex>`).
|
||||
В одном файле можно [сконфигурировать](external_dicts_dict.md) произвольное количество словарей.
|
||||
|
||||
>можете преобразовывать значения по небольшому словарю, описав его в запросе `SELECT` (см. функцию [transform](../functions/other_functions.md)). Эта функциональность не связана с внешними словарями.
|
||||
Если вы создаёте внешние словари [DDL-запросами](../create.md#create-dictionary-query), то не задавайте конфигурацию словаря в конфигурации сервера.
|
||||
|
||||
!!! attention "Внимание"
|
||||
Можно преобразовывать значения по небольшому словарю, описав его в запросе `SELECT` (см. функцию [transform](../functions/other_functions.md)). Эта функциональность не связана с внешними словарями.
|
||||
|
||||
|
||||
Смотрите также:
|
||||
## Смотрите также {#ext-dicts-see-also}
|
||||
|
||||
- [Настройка внешнего словаря](external_dicts_dict.md)
|
||||
- [Хранение словарей в памяти](external_dicts_dict_layout.md)
|
||||
|
@ -1,11 +1,15 @@
|
||||
# Настройка внешнего словаря {#dicts-external_dicts_dict}
|
||||
|
||||
Конфигурация словаря имеет следующую структуру:
|
||||
XML-конфигурация словаря имеет следующую структуру:
|
||||
|
||||
```xml
|
||||
<dictionary>
|
||||
<name>dict_name</name>
|
||||
|
||||
<structure>
|
||||
<!-- Complex key configuration -->
|
||||
</structure>
|
||||
|
||||
<source>
|
||||
<!-- Source configuration -->
|
||||
</source>
|
||||
@ -14,20 +18,29 @@
|
||||
<!-- Memory layout configuration -->
|
||||
</layout>
|
||||
|
||||
<structure>
|
||||
<!-- Complex key configuration -->
|
||||
</structure>
|
||||
|
||||
<lifetime>
|
||||
<!-- Lifetime of dictionary in memory -->
|
||||
</lifetime>
|
||||
</dictionary>
|
||||
```
|
||||
|
||||
- name - Идентификатор, под которым словарь будет доступен для использования. Используйте символы `[a-zA-Z0-9_\-]`.
|
||||
- [source](external_dicts_dict_sources.md) - Источник словаря.
|
||||
- [layout](external_dicts_dict_layout.md) - Размещение словаря в памяти.
|
||||
- [structure](external_dicts_dict_structure.md) - Структура словаря. Ключ и атрибуты, которые можно получить по ключу.
|
||||
- [lifetime](external_dicts_dict_lifetime.md) - Периодичность обновления словарей.
|
||||
Соответствующий [DDL-запрос](../create.md#create-dictionary-query) имеет следующий вид:
|
||||
|
||||
```sql
|
||||
CREATE DICTIONARY dict_name
|
||||
(
|
||||
... -- attributes
|
||||
)
|
||||
PRIMARY KEY ... -- complex or single key configuration
|
||||
SOURCE(...) -- Source configuration
|
||||
LAYOUT(...) -- Memory layout configuration
|
||||
LIFETIME(...) -- Lifetime of dictionary in memory
|
||||
```
|
||||
|
||||
- `name` — Идентификатор, под которым словарь будет доступен для использования. Используйте символы `[a-zA-Z0-9_\-]`.
|
||||
- [source](external_dicts_dict_sources.md) — Источник словаря.
|
||||
- [layout](external_dicts_dict_layout.md) — Размещение словаря в памяти.
|
||||
- [structure](external_dicts_dict_structure.md) — Структура словаря. Ключ и атрибуты, которые можно получить по ключу.
|
||||
- [lifetime](external_dicts_dict_lifetime.md) — Периодичность обновления словарей.
|
||||
|
||||
[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/dicts/external_dicts_dict/) <!--hide-->
|
||||
|
@ -35,15 +35,25 @@
|
||||
```
|
||||
|
||||
|
||||
Соответствущий [DDL-запрос](../create.md#create-dictionary-query):
|
||||
|
||||
```sql
|
||||
CREATE DICTIONARY (...)
|
||||
...
|
||||
LAYOUT(LAYOUT_TYPE(param value)) -- layout settings
|
||||
...
|
||||
```
|
||||
|
||||
## Способы размещения словарей в памяти
|
||||
|
||||
- [flat](#flat)
|
||||
- [hashed](#hashed)
|
||||
- [cache](#cache)
|
||||
- [range_hashed](#range-hashed)
|
||||
- [complex_key_hashed](#complex-key-hashed)
|
||||
- [complex_key_cache](#complex-key-cache)
|
||||
- [ip_trie](#ip-trie)
|
||||
- [flat](#flat)
|
||||
- [hashed](#hashed)
|
||||
- [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed)
|
||||
- [cache](#cache)
|
||||
- [range_hashed](#range-hashed)
|
||||
- [complex_key_hashed](#complex-key-hashed)
|
||||
- [complex_key_cache](#complex-key-cache)
|
||||
- [ip_trie](#ip-trie)
|
||||
|
||||
### flat
|
||||
|
||||
@ -63,6 +73,12 @@
|
||||
</layout>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
LAYOUT(FLAT())
|
||||
```
|
||||
|
||||
### hashed
|
||||
|
||||
Словарь полностью хранится в оперативной памяти в виде хэш-таблиц. Словарь может содержать произвольное количество элементов с произвольными идентификаторами. На практике, количество ключей может достигать десятков миллионов элементов.
|
||||
@ -77,6 +93,29 @@
|
||||
</layout>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
LAYOUT(HASHED())
|
||||
```
|
||||
|
||||
### sparse_hashed {#dicts-external_dicts_dict_layout-sparse_hashed}
|
||||
|
||||
Аналогичен `hashed`, но при этом занимает меньше места в памяти и генерирует более высокую загрузку CPU.
|
||||
|
||||
Пример конфигурации:
|
||||
|
||||
```xml
|
||||
<layout>
|
||||
<sparse_hashed />
|
||||
</layout>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
LAYOUT(SPARSE_HASHED())
|
||||
```
|
||||
|
||||
### complex_key_hashed
|
||||
|
||||
@ -90,6 +129,12 @@
|
||||
</layout>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
LAYOUT(COMPLEX_KEY_HASHED())
|
||||
```
|
||||
|
||||
|
||||
### range_hashed
|
||||
|
||||
@ -131,6 +176,19 @@
|
||||
...
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
CREATE DICTIONARY somedict (
|
||||
id UInt64,
|
||||
first Date,
|
||||
last Date
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(RANGE_HASHED())
|
||||
RANGE(MIN first MAX last)
|
||||
```
|
||||
|
||||
Для работы с такими словарями в функцию `dictGetT` необходимо передавать дополнительный аргумент, для которого подбирается диапазон:
|
||||
|
||||
dictGetT('dict_name', 'attr_name', id, date)
|
||||
@ -178,6 +236,18 @@
|
||||
</yandex>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
CREATE DICTIONARY somedict(
|
||||
Abcdef UInt64,
|
||||
StartTimeStamp UInt64,
|
||||
EndTimeStamp UInt64,
|
||||
XXXType String DEFAULT ''
|
||||
)
|
||||
PRIMARY KEY Abcdef
|
||||
RANGE(MIN StartTimeStamp MAX EndTimeStamp)
|
||||
```
|
||||
|
||||
### cache
|
||||
|
||||
@ -204,6 +274,12 @@
|
||||
</layout>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
LAYOUT(CACHE(SIZE_IN_CELLS 1000000000))
|
||||
```
|
||||
|
||||
Укажите достаточно большой размер кэша. Количество ячеек следует подобрать экспериментальным путём:
|
||||
|
||||
1. Выставить некоторое значение.
|
||||
@ -265,6 +341,17 @@
|
||||
...
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
CREATE DICTIONARY somedict (
|
||||
prefix String,
|
||||
asn UInt32,
|
||||
cca2 String DEFAULT '??'
|
||||
)
|
||||
PRIMARY KEY prefix
|
||||
```
|
||||
|
||||
Этот ключ должен иметь только один атрибут типа `String`, содержащий допустимый префикс IP. Другие типы еще не поддерживаются.
|
||||
|
||||
Для запросов необходимо использовать те же функции (`dictGetT` с кортежем), что и для словарей с составными ключами:
|
||||
|
@ -14,6 +14,15 @@ ClickHouse периодически обновляет словари. Инте
|
||||
...
|
||||
</dictionary>
|
||||
```
|
||||
или
|
||||
|
||||
```sql
|
||||
CREATE DICTIONARY (...)
|
||||
...
|
||||
LIFETIME(300)
|
||||
...
|
||||
```
|
||||
|
||||
|
||||
Настройка `<lifetime>0</lifetime>` запрещает обновление словарей.
|
||||
|
||||
@ -32,6 +41,12 @@ ClickHouse периодически обновляет словари. Инте
|
||||
</dictionary>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
LIFETIME(MIN 300 MAX 360)
|
||||
```
|
||||
|
||||
При обновлении словарей сервер ClickHouse применяет различную логику в зависимости от типа [источника](external_dicts_dict_sources.md):
|
||||
|
||||
> - У текстового файла проверяется время модификации. Если время изменилось по отношению к запомненному ранее, то словарь обновляется.
|
||||
@ -56,4 +71,12 @@ ClickHouse периодически обновляет словари. Инте
|
||||
</dictionary>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
...
|
||||
SOURCE(ODBC(... invalidate_query 'SELECT update_time FROM dictionary_source where id = 1'))
|
||||
...
|
||||
```
|
||||
|
||||
[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/dicts/external_dicts_dict_lifetime/) <!--hide-->
|
||||
|
@ -3,7 +3,7 @@
|
||||
|
||||
Внешний словарь можно подключить из множества источников.
|
||||
|
||||
Общий вид конфигурации:
|
||||
Общий вид XML-конфигурации:
|
||||
|
||||
```xml
|
||||
<yandex>
|
||||
@ -20,6 +20,16 @@
|
||||
</yandex>
|
||||
```
|
||||
|
||||
Аналогичный [DDL-запрос](../create.md#create-dictionary-query):
|
||||
|
||||
```sql
|
||||
CREATE DICTIONARY dict_name (...)
|
||||
...
|
||||
SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration
|
||||
...
|
||||
```
|
||||
|
||||
|
||||
Источник настраивается в разделе `source`.
|
||||
|
||||
Типы источников (`source_type`):
|
||||
@ -48,10 +58,16 @@
|
||||
</source>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
|
||||
```
|
||||
|
||||
Поля настройки:
|
||||
|
||||
- `path` - Абсолютный путь к файлу.
|
||||
- `format` - Формат файла. Поддерживаются все форматы, описанные в разделе "[Форматы](../../interfaces/formats.md#formats)".
|
||||
- `path` — Абсолютный путь к файлу.
|
||||
- `format` — Формат файла. Поддерживаются все форматы, описанные в разделе "[Форматы](../../interfaces/formats.md#formats)".
|
||||
|
||||
|
||||
## Исполняемый файл {#dicts-external_dicts_dict_sources-executable}
|
||||
@ -69,10 +85,16 @@
|
||||
</source>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
SOURCE(EXECUTABLE(command 'cat /opt/dictionaries/os.tsv' format 'TabSeparated'))
|
||||
```
|
||||
|
||||
Поля настройки:
|
||||
|
||||
- `command` - Абсолютный путь к исполняемому файлу или имя файла (если каталог программы прописан в `PATH`).
|
||||
- `format` - Формат файла. Поддерживаются все форматы, описанные в разделе "[Форматы](../../interfaces/formats.md#formats)".
|
||||
- `command` — Абсолютный путь к исполняемому файлу или имя файла (если каталог программы прописан в `PATH`).
|
||||
- `format` — Формат файла. Поддерживаются все форматы, описанные в разделе "[Форматы](../../interfaces/formats.md#formats)".
|
||||
|
||||
|
||||
## HTTP(s) {#dicts-external_dicts_dict_sources-http}
|
||||
@ -86,16 +108,37 @@
|
||||
<http>
|
||||
<url>http://[::1]/os.tsv</url>
|
||||
<format>TabSeparated</format>
|
||||
<credentials>
|
||||
<user>user</user>
|
||||
<password>password</password>
|
||||
</credentials>
|
||||
<headers>
|
||||
<header>
|
||||
<name>API-KEY</name>
|
||||
<value>key</value>
|
||||
</header>
|
||||
</headers>
|
||||
</http>
|
||||
</source>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
SOURCE(HTTP(
|
||||
url 'http://[::1]/os.tsv'
|
||||
format 'TabSeparated'
|
||||
credentials(user 'user' password 'password')
|
||||
headers(header(name 'API-KEY' value 'key'))
|
||||
))
|
||||
```
|
||||
|
||||
Чтобы ClickHouse смог обратиться к HTTPS-ресурсу, необходимо [настроить openSSL](../../operations/server_settings/settings.md) в конфигурации сервера.
|
||||
|
||||
Поля настройки:
|
||||
|
||||
- `url` - URL источника.
|
||||
- `format` - Формат файла. Поддерживаются все форматы, описанные в разделе "[Форматы](../../interfaces/formats.md#formats)".
|
||||
- `url` — URL источника.
|
||||
- `format` — Формат файла. Поддерживаются все форматы, описанные в разделе "[Форматы](../../interfaces/formats.md#formats)".
|
||||
|
||||
|
||||
## ODBC {#dicts-external_dicts_dict_sources-odbc}
|
||||
@ -105,20 +148,33 @@
|
||||
Пример настройки:
|
||||
|
||||
```xml
|
||||
<odbc>
|
||||
<db>DatabaseName</db>
|
||||
<table>ShemaName.TableName</table>
|
||||
<connection_string>DSN=some_parameters</connection_string>
|
||||
<invalidate_query>SQL_QUERY</invalidate_query>
|
||||
</odbc>
|
||||
<source>
|
||||
<odbc>
|
||||
<db>DatabaseName</db>
|
||||
<table>ShemaName.TableName</table>
|
||||
<connection_string>DSN=some_parameters</connection_string>
|
||||
<invalidate_query>SQL_QUERY</invalidate_query>
|
||||
</odbc>
|
||||
</source>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
SOURCE(ODBC(
|
||||
db 'DatabaseName'
|
||||
table 'SchemaName.TableName'
|
||||
connection_string 'DSN=some_parameters'
|
||||
invalidate_query 'SQL_QUERY'
|
||||
))
|
||||
```
|
||||
|
||||
Поля настройки:
|
||||
|
||||
- `db` - имя базы данных. Не указывать, если имя базы задано в параметрах. `<connection_string>`.
|
||||
- `table` - имя таблицы и схемы, если она есть.
|
||||
- `connection_string` - строка соединения.
|
||||
- `invalidate_query` - запрос для проверки статуса словаря. Необязательный параметр. Читайте подробнее в разделе [Обновление словарей](external_dicts_dict_lifetime.md).
|
||||
- `db` — имя базы данных. Не указывать, если имя базы задано в параметрах. `<connection_string>`.
|
||||
- `table` — имя таблицы и схемы, если она есть.
|
||||
- `connection_string` — строка соединения.
|
||||
- `invalidate_query` — запрос для проверки статуса словаря. Необязательный параметр. Читайте подробнее в разделе [Обновление словарей](external_dicts_dict_lifetime.md).
|
||||
|
||||
ClickHouse получает от ODBC-драйвера информацию о квотировании и квотирует настройки в запросах к драйверу, поэтому имя таблицы нужно указывать в соответствии с регистром имени таблицы в базе данных.
|
||||
|
||||
@ -216,6 +272,18 @@ $ sudo apt-get install -y unixodbc odbcinst odbc-postgresql
|
||||
</yandex>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
CREATE DICTIONARY table_name (
|
||||
id UInt64,
|
||||
some_column UInt64 DEFAULT 0
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(ODBC(connection_string 'DSN=myconnection' table 'postgresql_table'))
|
||||
LAYOUT(HASHED())
|
||||
LIFETIME(MIN 300 MAX 360)
|
||||
|
||||
Может понадобиться в `odbc.ini` указать полный путь до библиотеки с драйвером `DRIVER=/usr/local/lib/psqlodbcw.so`.
|
||||
|
||||
### Пример подключения MS SQL Server
|
||||
@ -299,6 +367,20 @@ $ sudo apt-get install tdsodbc freetds-bin sqsh
|
||||
</yandex>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
CREATE DICTIONARY test (
|
||||
k UInt64,
|
||||
s String DEFAULT ''
|
||||
)
|
||||
PRIMARY KEY k
|
||||
SOURCE(ODBC(table 'dict' connection_string 'DSN=MSSQL;UID=test;PWD=test'))
|
||||
LAYOUT(FLAT())
|
||||
LIFETIME(MIN 300 MAX 360)
|
||||
```
|
||||
|
||||
|
||||
## СУБД
|
||||
|
||||
|
||||
@ -328,6 +410,22 @@ $ sudo apt-get install tdsodbc freetds-bin sqsh
|
||||
</source>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
SOURCE(MYSQL(
|
||||
port 3306
|
||||
user 'clickhouse'
|
||||
password 'qwerty'
|
||||
replica(host 'example01-1' priority 1)
|
||||
replica(host 'example01-2' priority 1)
|
||||
db 'db_name'
|
||||
table 'table_name'
|
||||
where 'id=10'
|
||||
invalidate_query 'SQL_QUERY'
|
||||
))
|
||||
```
|
||||
|
||||
Поля настройки:
|
||||
|
||||
- `port` — порт сервера MySQL. Можно указать для всех реплик или для каждой в отдельности (внутри `<replica>`).
|
||||
@ -362,6 +460,21 @@ MySQL можно подключить на локальном хосте чер
|
||||
</source>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
SOURCE(MYSQL(
|
||||
host 'localhost'
|
||||
socket '/path/to/socket/file.sock'
|
||||
user 'clickhouse'
|
||||
password 'qwerty'
|
||||
db 'db_name'
|
||||
table 'table_name'
|
||||
where 'id=10'
|
||||
invalidate_query 'SQL_QUERY'
|
||||
))
|
||||
```
|
||||
|
||||
|
||||
### ClickHouse {#dicts-external_dicts_dict_sources-clickhouse}
|
||||
|
||||
@ -381,16 +494,30 @@ MySQL можно подключить на локальном хосте чер
|
||||
</source>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
SOURCE(CLICKHOUSE(
|
||||
host 'example01-01-1'
|
||||
port 9000
|
||||
user 'default'
|
||||
password ''
|
||||
db 'default'
|
||||
table 'ids'
|
||||
where 'id=10'
|
||||
))
|
||||
```
|
||||
|
||||
Поля настройки:
|
||||
|
||||
- `host` - хост ClickHouse. Если host локальный, то запрос выполняется без сетевого взаимодействия. Чтобы повысить отказоустойчивость решения, можно создать таблицу типа [Distributed](../../operations/table_engines/distributed.md) и прописать её в дальнейших настройках.
|
||||
- `port` - порт сервера ClickHouse.
|
||||
- `user` - имя пользователя ClickHouse.
|
||||
- `password` - пароль пользователя ClickHouse.
|
||||
- `db` - имя базы данных.
|
||||
- `table` - имя таблицы.
|
||||
- `where` - условие выбора. Может отсутствовать.
|
||||
- `invalidate_query` - запрос для проверки статуса словаря. Необязательный параметр. Читайте подробнее в разделе [Обновление словарей](external_dicts_dict_lifetime.md).
|
||||
- `host` — хост ClickHouse. Если host локальный, то запрос выполняется без сетевого взаимодействия. Чтобы повысить отказоустойчивость решения, можно создать таблицу типа [Distributed](../../operations/table_engines/distributed.md) и прописать её в дальнейших настройках.
|
||||
- `port` — порт сервера ClickHouse.
|
||||
- `user` — имя пользователя ClickHouse.
|
||||
- `password` — пароль пользователя ClickHouse.
|
||||
- `db` — имя базы данных.
|
||||
- `table` — имя таблицы.
|
||||
- `where` — условие выбора. Может отсутствовать.
|
||||
- `invalidate_query` — запрос для проверки статуса словаря. Необязательный параметр. Читайте подробнее в разделе [Обновление словарей](external_dicts_dict_lifetime.md).
|
||||
|
||||
|
||||
### MongoDB {#dicts-external_dicts_dict_sources-mongodb}
|
||||
@ -410,14 +537,27 @@ MySQL можно подключить на локальном хосте чер
|
||||
</source>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
SOURCE(MONGO(
|
||||
host 'localhost'
|
||||
port 27017
|
||||
user ''
|
||||
password ''
|
||||
db 'test'
|
||||
collection 'dictionary_source'
|
||||
))
|
||||
```
|
||||
|
||||
Поля настройки:
|
||||
|
||||
- `host` - хост MongoDB.
|
||||
- `port` - порт сервера MongoDB.
|
||||
- `user` - имя пользователя MongoDB.
|
||||
- `password` - пароль пользователя MongoDB.
|
||||
- `db` - имя базы данных.
|
||||
- `collection` - имя коллекции.
|
||||
- `host` — хост MongoDB.
|
||||
- `port` — порт сервера MongoDB.
|
||||
- `user` — имя пользователя MongoDB.
|
||||
- `password` — пароль пользователя MongoDB.
|
||||
- `db` — имя базы данных.
|
||||
- `collection` — имя коллекции.
|
||||
|
||||
### Redis {#dicts-external_dicts_dict_sources-redis}
|
||||
|
||||
@ -434,6 +574,17 @@ MySQL можно подключить на локальном хосте чер
|
||||
</source>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
SOURCE(REDIS(
|
||||
host 'localhost'
|
||||
port 6379
|
||||
storage_type 'simple'
|
||||
db_index 0
|
||||
))
|
||||
```
|
||||
|
||||
Поля настройки:
|
||||
|
||||
- `host` – хост Redis.
|
||||
|
@ -24,10 +24,10 @@
|
||||
Атрибуты описываются элементами:
|
||||
|
||||
- `<id>` — [столбец с ключом](external_dicts_dict_structure.md#ext_dict_structure-key).
|
||||
- `<attribute>` — [столбец данных](external_dicts_dict_structure.md#ext_dict_structure-attributes). Можно задать несколько столбцов.
|
||||
- `<attribute>` — [столбец данных](external_dicts_dict_structure.md#ext_dict_structure-attributes). Можно задать несколько атрибутов.
|
||||
|
||||
|
||||
Запрос создания словаря:
|
||||
Создание словаря запросом:
|
||||
|
||||
```sql
|
||||
CREATE DICTIONARY dict_name (
|
||||
@ -48,10 +48,10 @@ PRIMARY KEY Id
|
||||
|
||||
ClickHouse поддерживает следующие виды ключей:
|
||||
|
||||
- Числовой ключ. `UInt64`. Описывается в теге `<id>`.
|
||||
- Составной ключ. Набор значений разного типа. Описывается в теге `<key>`.
|
||||
- Числовой ключ. `UInt64`. Описывается в теге `<id>` или ключевым словом `PRIMARY KEY`.
|
||||
- Составной ключ. Набор значений разного типа. Описывается в теге `<key>` или ключевым словом `PRIMARY KEY`.
|
||||
|
||||
Структура может содержать либо `<id>` либо `<key>`.
|
||||
Структура может содержать либо `<id>` либо `<key>`. DDL-запрос может содержать только `PRIMARY KEY`.
|
||||
|
||||
!!! warning "Обратите внимание"
|
||||
Ключ не надо дополнительно описывать в атрибутах.
|
||||
@ -72,6 +72,20 @@ ClickHouse поддерживает следующие виды ключей:
|
||||
|
||||
- `name` — имя столбца с ключами.
|
||||
|
||||
Для DDL-запроса:
|
||||
|
||||
```sql
|
||||
CREATE DICTIONARY (
|
||||
Id UInt64,
|
||||
...
|
||||
)
|
||||
PRIMARY KEY Id
|
||||
...
|
||||
```
|
||||
|
||||
- `PRIMARY KEY` – имя столбца с ключами.
|
||||
|
||||
|
||||
### Составной ключ
|
||||
|
||||
Ключом может быть кортеж (`tuple`) из полей произвольных типов. В этом случае [layout](external_dicts_dict_layout.md) должен быть `complex_key_hashed` или `complex_key_cache`.
|
||||
@ -97,6 +111,18 @@ ClickHouse поддерживает следующие виды ключей:
|
||||
...
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
CREATE DICTIONARY (
|
||||
field1 String,
|
||||
field2 String
|
||||
...
|
||||
)
|
||||
PRIMARY KEY field1, field2
|
||||
...
|
||||
```
|
||||
|
||||
При запросе в функции `dictGet*` в качестве ключа передаётся кортеж. Пример: `dictGetString('dict_name', 'attr_name', tuple('string for field1', num_for_field2))`.
|
||||
|
||||
|
||||
@ -119,6 +145,15 @@ ClickHouse поддерживает следующие виды ключей:
|
||||
</structure>
|
||||
```
|
||||
|
||||
или
|
||||
|
||||
```sql
|
||||
CREATE DICTIONARY somename (
|
||||
Name ClickHouseDataType DEFAULT '' EXPRESSION rand64() HIERARCHICAL INJECTIVE IS_OBJECT_ID
|
||||
)
|
||||
```
|
||||
|
||||
|
||||
Поля конфигурации:
|
||||
|
||||
| Тег | Описание | Обязательный |
|
||||
|
@ -3,10 +3,10 @@
|
||||
## SHOW CREATE TABLE
|
||||
|
||||
```sql
|
||||
SHOW CREATE [TEMPORARY] TABLE [db.]table [INTO OUTFILE filename] [FORMAT format]
|
||||
SHOW CREATE [TEMPORARY] [TABLE|DICTIONARY] [db.]table [INTO OUTFILE filename] [FORMAT format]
|
||||
```
|
||||
|
||||
Возвращает один столбец типа `String` с именем statement, содержащий одно значение — запрос `CREATE TABLE`, с помощью которого была создана указанная таблица.
|
||||
Возвращает один столбец типа `String` с именем statement, содержащий одно значение — запрос `CREATE TABLE`, с помощью которого был создан указанный объект.
|
||||
|
||||
## SHOW DATABASES {#show-databases}
|
||||
|
||||
@ -62,3 +62,35 @@ SHOW TABLES FROM system LIKE '%co%' LIMIT 2
|
||||
│ collations │
|
||||
└────────────────────────────────┘
|
||||
```
|
||||
|
||||
## SHOW DICTIONARIES
|
||||
|
||||
Выводит список [внешних словарей](dicts/external_dicts.md).
|
||||
|
||||
```sql
|
||||
SHOW DICTIONARIES [FROM <db>] [LIKE '<pattern>'] [LIMIT <N>] [INTO OUTFILE <filename>] [FORMAT <format>]
|
||||
```
|
||||
|
||||
Если секция `FROM` не указана, запрос возвращает список словарей из текущей базы данных.
|
||||
|
||||
Аналогичный результат можно получить следующим запросом:
|
||||
|
||||
```sql
|
||||
SELECT name FROM system.dictionaries WHERE database = <db> [AND name LIKE <pattern>] [LIMIT <N>] [INTO OUTFILE <filename>] [FORMAT <format>]
|
||||
```
|
||||
|
||||
**Example**
|
||||
|
||||
Запрос выводит первые две стоки из списка таблиц в базе данных `system`, имена которых содержат `reg`.
|
||||
|
||||
```sql
|
||||
SHOW DICTIONARIES FROM db LIKE '%reg%' LIMIT 2
|
||||
```
|
||||
```text
|
||||
┌─name─────────┐
|
||||
│ regions │
|
||||
│ region_names │
|
||||
└──────────────┘
|
||||
```
|
||||
|
||||
[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/show/) <!--hide-->
|
||||
|
@ -2,35 +2,35 @@ alabaster==0.7.12
|
||||
Babel==2.5.1
|
||||
backports-abc==0.5
|
||||
beautifulsoup4==4.8.2
|
||||
certifi==2017.11.5
|
||||
certifi==2019.11.28
|
||||
chardet==3.0.4
|
||||
click==6.7
|
||||
CommonMark==0.5.4
|
||||
CommonMark==0.9.1
|
||||
cssmin==0.2.0
|
||||
docutils==0.16
|
||||
futures==3.1.1
|
||||
htmlmin==0.1.12
|
||||
idna==2.6
|
||||
idna==2.9
|
||||
imagesize==1.2.0
|
||||
Jinja2==2.11.1
|
||||
jsmin==2.2.2
|
||||
livereload==2.5.1
|
||||
Markdown==2.6.11
|
||||
MarkupSafe==1.0
|
||||
MarkupSafe==1.1.1
|
||||
mkdocs==1.0.4
|
||||
Pygments==2.5.2
|
||||
python-slugify==1.2.6
|
||||
pytz==2017.3
|
||||
PyYAML==5.3
|
||||
recommonmark==0.4.0
|
||||
requests==2.21.0
|
||||
requests==2.23.0
|
||||
singledispatch==3.4.0.3
|
||||
six==1.11.0
|
||||
six==1.14.0
|
||||
snowballstemmer==1.2.1
|
||||
Sphinx==1.6.5
|
||||
sphinxcontrib-websupport==1.0.1
|
||||
tornado==5.1
|
||||
typing==3.7.4.1
|
||||
Unidecode==1.0.23
|
||||
Unidecode==1.1.1
|
||||
urllib3==1.25.8
|
||||
gitpython==2.1.14
|
||||
|
Loading…
Reference in New Issue
Block a user