Merge branch 'master' of github.com:ClickHouse/ClickHouse

This commit is contained in:
Sergei Shtykov 2019-11-26 13:14:17 +03:00
commit b2efc2d904
56 changed files with 2149 additions and 1274 deletions

View File

@ -1,3 +1,84 @@
## ClickHouse release v19.17.4.11, 2019-11-22
### Backward Incompatible Change
* Using column instead of AST to store scalar subquery results for better performance. Setting `enable_scalar_subquery_optimization` was added in 19.17 and it was enabled by default. It leads to errors like [this](https://github.com/ClickHouse/ClickHouse/issues/7851) during upgrade to 19.17.2 or 19.17.3 from previous versions. This setting was disabled by default in 19.17.4, to make possible upgrading from 19.16 and older versions without errors. [#7392](https://github.com/ClickHouse/ClickHouse/pull/7392) ([Amos Bird](https://github.com/amosbird))
### New Feature
* Add the ability to create dictionaries with DDL queries. [#7360](https://github.com/ClickHouse/ClickHouse/pull/7360) ([alesapin](https://github.com/alesapin))
* Authentication in S3 table function and storage. Now we have complete support for S3 import/export. [#7623](https://github.com/ClickHouse/ClickHouse/pull/7623) ([Vladimir Chebotarev](https://github.com/excitoon))
* Make `bloom_filter` type of index supporting `LowCardinality` and `Nullable` [#7363](https://github.com/ClickHouse/ClickHouse/issues/7363) [#7561](https://github.com/ClickHouse/ClickHouse/pull/7561) ([Nikolai Kochetov](https://github.com/KochetovNicolai))
* Add function `isValidJSON` to check that passed string is a valid json. [#5910](https://github.com/ClickHouse/ClickHouse/issues/5910) [#7293](https://github.com/ClickHouse/ClickHouse/pull/7293) ([Vdimir](https://github.com/Vdimir))
* Implement `arrayCompact` function [#7328](https://github.com/ClickHouse/ClickHouse/pull/7328) ([Memo](https://github.com/Joeywzr))
* Created function `hex` for Decimal numbers. It works like `hex(reinterpretAsString())`, but doesn't delete last zero bytes. [#7355](https://github.com/ClickHouse/ClickHouse/pull/7355) ([Mikhail Korotov](https://github.com/millb))
* Add `arrayFill` and `arrayReverseFill` functions, which replace elements by other elements in front/back of them in the array. [#7380](https://github.com/ClickHouse/ClickHouse/pull/7380) ([hcz](https://github.com/hczhcz))
* Up precision of `avg` aggregate function result to max of `Decimal` type [#7446](https://github.com/ClickHouse/ClickHouse/pull/7446) ([Andrey Konyaev](https://github.com/akonyaev90))
* Add `CRC32IEEE()`/`CRC64()` support [#7480](https://github.com/ClickHouse/ClickHouse/pull/7480) ([Azat Khuzhin](https://github.com/azat))
* Implement `char` function similar to one in [mysql](https://dev.mysql.com/doc/refman/8.0/en/string-functions.html#function_char) [#7486](https://github.com/ClickHouse/ClickHouse/pull/7486) ([sundyli](https://github.com/sundy-li))
* Add `bitmapTransform` function. It transforms an array of values in a bitmap to another array of values, the result is a new bitmap [#7598](https://github.com/ClickHouse/ClickHouse/pull/7598) ([Zhichang Yu](https://github.com/yuzhichang))
* Implemented `javaHashUTF16LE()` function [#7651](https://github.com/ClickHouse/ClickHouse/pull/7651) ([achimbab](https://github.com/achimbab))
* Add `_shard_num` virtual column for the Distributed engine [#7624](https://github.com/ClickHouse/ClickHouse/pull/7624) ([Azat Khuzhin](https://github.com/azat))
### Experimental Feature
* Support for processors (new query execution pipeline) in `MergeTree`. [#7181](https://github.com/ClickHouse/ClickHouse/pull/7181) ([Nikolai Kochetov](https://github.com/KochetovNicolai))
### Bug Fix
* Fix incorrect float parsing in `Values` [#7817](https://github.com/ClickHouse/ClickHouse/issues/7817) [#7870](https://github.com/ClickHouse/ClickHouse/pull/7870) ([tavplubix](https://github.com/tavplubix))
* Fix rare deadlock which can happen when trace_log is enabled. [#7838](https://github.com/ClickHouse/ClickHouse/pull/7838) ([filimonov](https://github.com/filimonov))
* Prevent message duplication when producing Kafka table has any MVs selecting from it [#7265](https://github.com/ClickHouse/ClickHouse/pull/7265) ([Ivan](https://github.com/abyss7))
* Support for `Array(LowCardinality(Nullable(String)))` in `IN`. Resolves [#7364](https://github.com/ClickHouse/ClickHouse/issues/7364) [#7366](https://github.com/ClickHouse/ClickHouse/pull/7366) ([achimbab](https://github.com/achimbab))
* Add handling of `SQL_TINYINT` and `SQL_BIGINT`, and fix handling of `SQL_FLOAT` data source types in ODBC Bridge. [#7491](https://github.com/ClickHouse/ClickHouse/pull/7491) ([Denis Glazachev](https://github.com/traceon))
* Fix aggregation (`avg` and quantiles) over empty decimal columns [#7431](https://github.com/ClickHouse/ClickHouse/pull/7431) ([Andrey Konyaev](https://github.com/akonyaev90))
* Fix `INSERT` into Distributed with `MATERIALIZED` columns [#7377](https://github.com/ClickHouse/ClickHouse/pull/7377) ([Azat Khuzhin](https://github.com/azat))
* Make `MOVE PARTITION` work if some parts of partition are already on destination disk or volume [#7434](https://github.com/ClickHouse/ClickHouse/pull/7434) ([Vladimir Chebotarev](https://github.com/excitoon))
* Fixed bug with hardlinks failing to be created during mutations in `ReplicatedMergeTree` in multi-disk configurations. [#7558](https://github.com/ClickHouse/ClickHouse/pull/7558) ([Vladimir Chebotarev](https://github.com/excitoon))
* Fixed a bug with a mutation on a MergeTree when whole part remains unchanged and best space is being found on another disk [#7602](https://github.com/ClickHouse/ClickHouse/pull/7602) ([Vladimir Chebotarev](https://github.com/excitoon))
* Fixed bug with `keep_free_space_ratio` not being read from disks configuration [#7645](https://github.com/ClickHouse/ClickHouse/pull/7645) ([Vladimir Chebotarev](https://github.com/excitoon))
* Fix bug with table contains only `Tuple` columns or columns with complex paths. Fixes [7541](https://github.com/ClickHouse/ClickHouse/issues/7541). [#7545](https://github.com/ClickHouse/ClickHouse/pull/7545) ([alesapin](https://github.com/alesapin))
* Do not account memory for Buffer engine in max_memory_usage limit [#7552](https://github.com/ClickHouse/ClickHouse/pull/7552) ([Azat Khuzhin](https://github.com/azat))
* Fix final mark usage in `MergeTree` tables ordered by `tuple()`. In rare cases it could lead to `Can't adjust last granule` error while select. [#7639](https://github.com/ClickHouse/ClickHouse/pull/7639) ([Anton Popov](https://github.com/CurtizJ))
* Fix bug in mutations that have predicate with actions that require context (for example functions for json), which may lead to crashes or strange exceptions. [#7664](https://github.com/ClickHouse/ClickHouse/pull/7664) ([alesapin](https://github.com/alesapin))
* Fix mismatch of database and table names escaping in `data/` and `shadow/` directories [#7575](https://github.com/ClickHouse/ClickHouse/pull/7575) ([Alexander Burmak](https://github.com/Alex-Burmak))
* Support duplicated keys in RIGHT|FULL JOINs, e.g. ```ON t.x = u.x AND t.x = u.y```. Fix crash in this case. [#7586](https://github.com/ClickHouse/ClickHouse/pull/7586) ([Artem Zuikov](https://github.com/4ertus2))
* Fix `Not found column <expression> in block` when joining on expression with RIGHT or FULL JOIN. [#7641](https://github.com/ClickHouse/ClickHouse/pull/7641) ([Artem Zuikov](https://github.com/4ertus2))
* One more attempt to fix infinite loop in `PrettySpace` format [#7591](https://github.com/ClickHouse/ClickHouse/pull/7591) ([Olga Khvostikova](https://github.com/stavrolia))
* Fix bug in `concat` function when all arguments were `FixedString` of the same size. [#7635](https://github.com/ClickHouse/ClickHouse/pull/7635) ([alesapin](https://github.com/alesapin))
* Fixed exception in case of using 1 argument while defining S3, URL and HDFS storages. [#7618](https://github.com/ClickHouse/ClickHouse/pull/7618) ([Vladimir Chebotarev](https://github.com/excitoon))
* Fix scope of the InterpreterSelectQuery for views with query [#7601](https://github.com/ClickHouse/ClickHouse/pull/7601) ([Azat Khuzhin](https://github.com/azat))
### Improvement
* `Nullable` columns recognized and NULL-values handled correctly by ODBC-bridge [#7402](https://github.com/ClickHouse/ClickHouse/pull/7402) ([Vasily Nemkov](https://github.com/Enmk))
* Write current batch for distributed send atomically [#7600](https://github.com/ClickHouse/ClickHouse/pull/7600) ([Azat Khuzhin](https://github.com/azat))
* Throw an exception if we cannot detect table for column name in query. [#7358](https://github.com/ClickHouse/ClickHouse/pull/7358) ([Artem Zuikov](https://github.com/4ertus2))
* Add `merge_max_block_size` setting to `MergeTreeSettings` [#7412](https://github.com/ClickHouse/ClickHouse/pull/7412) ([Artem Zuikov](https://github.com/4ertus2))
* Queries with `HAVING` and without `GROUP BY` assume group by constant. So, `SELECT 1 HAVING 1` now returns a result. [#7496](https://github.com/ClickHouse/ClickHouse/pull/7496) ([Amos Bird](https://github.com/amosbird))
* Support parsing `(X,)` as tuple similar to python. [#7501](https://github.com/ClickHouse/ClickHouse/pull/7501), [#7562](https://github.com/ClickHouse/ClickHouse/pull/7562) ([Amos Bird](https://github.com/amosbird))
* Make `range` function behaviors almost like pythonic one. [#7518](https://github.com/ClickHouse/ClickHouse/pull/7518) ([sundyli](https://github.com/sundy-li))
* Add `constraints` columns to table `system.settings` [#7553](https://github.com/ClickHouse/ClickHouse/pull/7553) ([Vitaly Baranov](https://github.com/vitlibar))
* Better Null format for tcp handler, so that it's possible to use `select ignore(<expression>) from table format Null` for perf measure via clickhouse-client [#7606](https://github.com/ClickHouse/ClickHouse/pull/7606) ([Amos Bird](https://github.com/amosbird))
* Queries like `CREATE TABLE ... AS (SELECT (1, 2))` are parsed correctly [#7542](https://github.com/ClickHouse/ClickHouse/pull/7542) ([hcz](https://github.com/hczhcz))
### Performance Improvement
* The performance of aggregation over short string keys is improved. [#6243](https://github.com/ClickHouse/ClickHouse/pull/6243) ([Alexander Kuzmenkov](https://github.com/akuzm), [Amos Bird](https://github.com/amosbird))
* Run another pass of syntax/expression analysis to get potential optimizations after constant predicates are folded. [#7497](https://github.com/ClickHouse/ClickHouse/pull/7497) ([Amos Bird](https://github.com/amosbird))
* Use storage meta info to evaluate trivial `SELECT count() FROM table;` [#7510](https://github.com/ClickHouse/ClickHouse/pull/7510) ([Amos Bird](https://github.com/amosbird), [alexey-milovidov](https://github.com/alexey-milovidov))
* Vectorize processing `arrayReduce` similar to Aggregator `addBatch`. [#7608](https://github.com/ClickHouse/ClickHouse/pull/7608) ([Amos Bird](https://github.com/amosbird))
* Minor improvements in performance of `Kafka` consumption [#7475](https://github.com/ClickHouse/ClickHouse/pull/7475) ([Ivan](https://github.com/abyss7))
### Build/Testing/Packaging Improvement
* Add support for cross-compiling to the CPU architecture AARCH64. Refactor packager script. [#7370](https://github.com/ClickHouse/ClickHouse/pull/7370) [#7539](https://github.com/ClickHouse/ClickHouse/pull/7539) ([Ivan](https://github.com/abyss7))
* Unpack darwin-x86_64 and linux-aarch64 toolchains into mounted Docker volume when building packages [#7534](https://github.com/ClickHouse/ClickHouse/pull/7534) ([Ivan](https://github.com/abyss7))
* Update Docker Image for Binary Packager [#7474](https://github.com/ClickHouse/ClickHouse/pull/7474) ([Ivan](https://github.com/abyss7))
* Fixed compile errors on MacOS Catalina [#7585](https://github.com/ClickHouse/ClickHouse/pull/7585) ([Ernest Poletaev](https://github.com/ernestp))
* Some refactoring in query analysis logic: split complex class into several simple ones. [#7454](https://github.com/ClickHouse/ClickHouse/pull/7454) ([Artem Zuikov](https://github.com/4ertus2))
* Fix build without submodules [#7295](https://github.com/ClickHouse/ClickHouse/pull/7295) ([proller](https://github.com/proller))
* Better `add_globs` in CMake files [#7418](https://github.com/ClickHouse/ClickHouse/pull/7418) ([Amos Bird](https://github.com/amosbird))
* Remove hardcoded paths in `unwind` target [#7460](https://github.com/ClickHouse/ClickHouse/pull/7460) ([Konstantin Podshumok](https://github.com/podshumok))
* Allow to use mysql format without ssl [#7524](https://github.com/ClickHouse/ClickHouse/pull/7524) ([proller](https://github.com/proller))
### Other
* Added ANTLR4 grammar for ClickHouse SQL dialect [#7595](https://github.com/ClickHouse/ClickHouse/issues/7595) [#7596](https://github.com/ClickHouse/ClickHouse/pull/7596) ([alexey-milovidov](https://github.com/alexey-milovidov))
## ClickHouse release v19.16.2.2, 2019-10-30 ## ClickHouse release v19.16.2.2, 2019-10-30
### Backward Incompatible Change ### Backward Incompatible Change

View File

@ -14,6 +14,5 @@ ClickHouse is an open-source column-oriented database management system that all
## Upcoming Events ## Upcoming Events
* [ClickHouse Meetup in Singapore](https://www.meetup.com/Singapore-Clickhouse-Meetup-Group/events/265085331/) on November 23.
* [ClickHouse Meetup in San Francisco](https://www.eventbrite.com/e/clickhouse-december-meetup-registration-78642047481) on December 3. * [ClickHouse Meetup in San Francisco](https://www.eventbrite.com/e/clickhouse-december-meetup-registration-78642047481) on December 3.

View File

@ -1,6 +1,7 @@
#include "ClusterCopier.h" #include "ClusterCopier.h"
#include <chrono> #include <chrono>
#include <optional>
#include <Poco/Util/XMLConfiguration.h> #include <Poco/Util/XMLConfiguration.h>
#include <Poco/Logger.h> #include <Poco/Logger.h>
#include <Poco/ConsoleChannel.h> #include <Poco/ConsoleChannel.h>
@ -178,7 +179,9 @@ struct ShardPartition
ShardPartition(TaskShard & parent, const String & name_quoted_) : task_shard(parent), name(name_quoted_) {} ShardPartition(TaskShard & parent, const String & name_quoted_) : task_shard(parent), name(name_quoted_) {}
String getPartitionPath() const; String getPartitionPath() const;
String getPartitionCleanStartPath() const;
String getCommonPartitionIsDirtyPath() const; String getCommonPartitionIsDirtyPath() const;
String getCommonPartitionIsCleanedPath() const;
String getPartitionActiveWorkersPath() const; String getPartitionActiveWorkersPath() const;
String getActiveWorkerPath() const; String getActiveWorkerPath() const;
String getPartitionShardsPath() const; String getPartitionShardsPath() const;
@ -259,6 +262,8 @@ struct TaskTable
String getPartitionPath(const String & partition_name) const; String getPartitionPath(const String & partition_name) const;
String getPartitionIsDirtyPath(const String & partition_name) const; String getPartitionIsDirtyPath(const String & partition_name) const;
String getPartitionIsCleanedPath(const String & partition_name) const;
String getPartitionTaskStatusPath(const String & partition_name) const;
String name_in_config; String name_in_config;
@ -369,23 +374,6 @@ struct MultiTransactionInfo
Coordination::Responses responses; Coordination::Responses responses;
}; };
/// Atomically checks that is_dirty node is not exists, and made the remaining op
/// Returns relative number of failed operation in the second field (the passed op has 0 index)
static MultiTransactionInfo checkNoNodeAndCommit(
const zkutil::ZooKeeperPtr & zookeeper,
const String & checking_node_path,
Coordination::RequestPtr && op)
{
MultiTransactionInfo info;
info.requests.emplace_back(zkutil::makeCreateRequest(checking_node_path, "", zkutil::CreateMode::Persistent));
info.requests.emplace_back(zkutil::makeRemoveRequest(checking_node_path, -1));
info.requests.emplace_back(std::move(op));
info.code = zookeeper->tryMulti(info.requests, info.responses);
return info;
}
// Creates AST representing 'ENGINE = Distributed(cluster, db, table, [sharding_key]) // Creates AST representing 'ENGINE = Distributed(cluster, db, table, [sharding_key])
std::shared_ptr<ASTStorage> createASTStorageDistributed( std::shared_ptr<ASTStorage> createASTStorageDistributed(
const String & cluster_name, const String & database, const String & table, const ASTPtr & sharding_key_ast = nullptr) const String & cluster_name, const String & database, const String & table, const ASTPtr & sharding_key_ast = nullptr)
@ -431,6 +419,11 @@ String TaskTable::getPartitionPath(const String & partition_name) const
+ "/" + escapeForFileName(partition_name); // 201701 + "/" + escapeForFileName(partition_name); // 201701
} }
String ShardPartition::getPartitionCleanStartPath() const
{
return getPartitionPath() + "/clean_start";
}
String ShardPartition::getPartitionPath() const String ShardPartition::getPartitionPath() const
{ {
return task_shard.task_table.getPartitionPath(name); return task_shard.task_table.getPartitionPath(name);
@ -438,8 +431,9 @@ String ShardPartition::getPartitionPath() const
String ShardPartition::getShardStatusPath() const String ShardPartition::getShardStatusPath() const
{ {
// /root/table_test.hits/201701/1 // schema: /<root...>/tables/<table>/<partition>/shards/<shard>
return getPartitionPath() + "/shards/" + toString(task_shard.numberInCluster()); // e.g. /root/table_test.hits/201701/shards/1
return getPartitionShardsPath() + "/" + toString(task_shard.numberInCluster());
} }
String ShardPartition::getPartitionShardsPath() const String ShardPartition::getPartitionShardsPath() const
@ -462,11 +456,26 @@ String ShardPartition::getCommonPartitionIsDirtyPath() const
return getPartitionPath() + "/is_dirty"; return getPartitionPath() + "/is_dirty";
} }
String ShardPartition::getCommonPartitionIsCleanedPath() const
{
return getCommonPartitionIsDirtyPath() + "/cleaned";
}
String TaskTable::getPartitionIsDirtyPath(const String & partition_name) const String TaskTable::getPartitionIsDirtyPath(const String & partition_name) const
{ {
return getPartitionPath(partition_name) + "/is_dirty"; return getPartitionPath(partition_name) + "/is_dirty";
} }
String TaskTable::getPartitionIsCleanedPath(const String & partition_name) const
{
return getPartitionIsDirtyPath(partition_name) + "/cleaned";
}
String TaskTable::getPartitionTaskStatusPath(const String & partition_name) const
{
return getPartitionPath(partition_name) + "/shards";
}
String DB::TaskShard::getDescription() const String DB::TaskShard::getDescription() const
{ {
std::stringstream ss; std::stringstream ss;
@ -1129,9 +1138,9 @@ protected:
} }
/** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock. /** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock.
* State of some task could be changed during the processing. * State of some task could change during the processing.
* We have to ensure that all shards have the finished state and there are no dirty flag. * We have to ensure that all shards have the finished state and there is no dirty flag.
* Moreover, we have to check status twice and check zxid, because state could be changed during the checking. * Moreover, we have to check status twice and check zxid, because state can change during the checking.
*/ */
bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition)
{ {
@ -1170,11 +1179,23 @@ protected:
} }
// Check that partition is not dirty // Check that partition is not dirty
if (zookeeper->exists(task_table.getPartitionIsDirtyPath(partition_name))) {
CleanStateClock clean_state_clock (
zookeeper,
task_table.getPartitionIsDirtyPath(partition_name),
task_table.getPartitionIsCleanedPath(partition_name)
);
Coordination::Stat stat;
LogicalClock task_start_clock;
if (zookeeper->exists(task_table.getPartitionTaskStatusPath(partition_name), &stat))
task_start_clock = LogicalClock(stat.mzxid);
zookeeper->get(task_table.getPartitionTaskStatusPath(partition_name), &stat);
if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid)
{ {
LOG_INFO(log, "Partition " << partition_name << " become dirty"); LOG_INFO(log, "Partition " << partition_name << " become dirty");
return false; return false;
} }
}
get_futures.clear(); get_futures.clear();
for (const String & path : status_paths) for (const String & path : status_paths)
@ -1260,17 +1281,135 @@ protected:
return res; return res;
} }
bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper) class LogicalClock
{
public:
std::optional<UInt64> zxid;
LogicalClock() = default;
LogicalClock(UInt64 _zxid)
: zxid(_zxid)
{}
bool hasHappened() const
{
return bool(zxid);
}
// happens-before relation with a reasonable time bound
bool happensBefore(const LogicalClock & other) const
{
const UInt64 HALF = 1ull << 63;
return
!zxid ||
(other.zxid && *zxid <= *other.zxid && *other.zxid - *zxid < HALF) ||
(other.zxid && *zxid >= *other.zxid && *zxid - *other.zxid > HALF);
}
bool operator<=(const LogicalClock & other) const
{
return happensBefore(other);
}
// strict equality check
bool operator==(const LogicalClock & other) const
{
return zxid == other.zxid;
}
};
class CleanStateClock
{
public:
LogicalClock discovery_zxid;
std::optional<UInt32> discovery_version;
LogicalClock clean_state_zxid;
std::optional<UInt32> clean_state_version;
std::shared_ptr<std::atomic_bool> stale;
bool is_clean() const
{
return
!is_stale()
&& (
!discovery_zxid.hasHappened()
|| (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid));
}
bool is_stale() const
{
return stale->load();
}
CleanStateClock(
const zkutil::ZooKeeperPtr & zookeeper,
const String & discovery_path,
const String & clean_state_path)
: stale(std::make_shared<std::atomic_bool>(false))
{
Coordination::Stat stat;
String _some_data;
auto watch_callback =
[stale = stale] (const Coordination::WatchResponse & rsp)
{
auto logger = &Poco::Logger::get("ClusterCopier");
if (rsp.error == Coordination::ZOK)
{
switch (rsp.type)
{
case Coordination::CREATED:
LOG_DEBUG(logger, "CleanStateClock change: CREATED, at " << rsp.path);
stale->store(true);
break;
case Coordination::CHANGED:
LOG_DEBUG(logger, "CleanStateClock change: CHANGED, at" << rsp.path);
stale->store(true);
}
}
};
if (zookeeper->tryGetWatch(discovery_path, _some_data, &stat, watch_callback))
{
discovery_zxid = LogicalClock(stat.mzxid);
discovery_version = stat.version;
}
if (zookeeper->tryGetWatch(clean_state_path, _some_data, &stat, watch_callback))
{
clean_state_zxid = LogicalClock(stat.mzxid);
clean_state_version = stat.version;
}
}
bool operator==(const CleanStateClock & other) const
{
return !is_stale()
&& !other.is_stale()
&& discovery_zxid == other.discovery_zxid
&& discovery_version == other.discovery_version
&& clean_state_zxid == other.clean_state_zxid
&& clean_state_version == other.clean_state_version;
}
bool operator!=(const CleanStateClock & other) const
{
return !(*this == other);
}
};
bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock)
{ {
if (is_safe_mode) if (is_safe_mode)
throw Exception("DROP PARTITION is prohibited in safe mode", ErrorCodes::NOT_IMPLEMENTED); throw Exception("DROP PARTITION is prohibited in safe mode", ErrorCodes::NOT_IMPLEMENTED);
TaskTable & task_table = task_partition.task_shard.task_table; TaskTable & task_table = task_partition.task_shard.task_table;
String current_shards_path = task_partition.getPartitionShardsPath(); const String current_shards_path = task_partition.getPartitionShardsPath();
String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath();
String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath();
String dirt_cleaner_path = is_dirty_flag_path + "/cleaner"; const String dirt_cleaner_path = is_dirty_flag_path + "/cleaner";
const String is_dirt_cleaned_path = task_partition.getCommonPartitionIsCleanedPath();
zkutil::EphemeralNodeHolder::Ptr cleaner_holder; zkutil::EphemeralNodeHolder::Ptr cleaner_holder;
try try
@ -1294,14 +1433,47 @@ protected:
{ {
if (stat.numChildren != 0) if (stat.numChildren != 0)
{ {
LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren << " active workers, sleep"); LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren << " active workers while trying to drop it. Going to sleep.");
std::this_thread::sleep_for(default_sleep_time); std::this_thread::sleep_for(default_sleep_time);
return false; return false;
} }
else
{
zookeeper->remove(current_partition_active_workers_dir);
}
} }
{
zkutil::EphemeralNodeHolder::Ptr active_workers_lock;
try
{
active_workers_lock = zkutil::EphemeralNodeHolder::create(current_partition_active_workers_dir, *zookeeper, host_id);
}
catch (const Coordination::Exception & e)
{
if (e.code == Coordination::ZNODEEXISTS)
{
LOG_DEBUG(log, "Partition " << task_partition.name << " is being filled now by somebody, sleep");
return false;
}
throw;
}
// Lock the dirty flag
zookeeper->set(is_dirty_flag_path, host_id, clean_state_clock.discovery_version.value());
zookeeper->tryRemove(task_partition.getPartitionCleanStartPath());
CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirt_cleaned_path);
/// Remove all status nodes /// Remove all status nodes
zookeeper->tryRemoveRecursive(current_shards_path); {
Strings children;
if (zookeeper->tryGetChildren(current_shards_path, children) == Coordination::ZOK)
for (const auto & child : children)
{
zookeeper->removeRecursive(current_shards_path + "/" + child);
}
}
String query = "ALTER TABLE " + getQuotedTable(task_table.table_push); String query = "ALTER TABLE " + getQuotedTable(task_table.table_push);
query += " DROP PARTITION " + task_partition.name + ""; query += " DROP PARTITION " + task_partition.name + "";
@ -1325,13 +1497,28 @@ protected:
return false; return false;
} }
/// Remove the locking node /// Update the locking node
Coordination::Requests requests; if (!my_clock.is_stale())
requests.emplace_back(zkutil::makeRemoveRequest(dirt_cleaner_path, -1)); {
requests.emplace_back(zkutil::makeRemoveRequest(is_dirty_flag_path, -1)); zookeeper->set(is_dirty_flag_path, host_id, my_clock.discovery_version.value());
zookeeper->multi(requests); if (my_clock.clean_state_version)
zookeeper->set(is_dirt_cleaned_path, host_id, my_clock.clean_state_version.value());
else
zookeeper->create(is_dirt_cleaned_path, host_id, zkutil::CreateMode::Persistent);
}
else
{
LOG_DEBUG(log, "Clean state is altered when dropping the partition, cowardly bailing");
/// clean state is stale
return false;
}
LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name); LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name);
if (zookeeper->tryCreate(current_shards_path, host_id, zkutil::CreateMode::Persistent) == Coordination::ZNODEEXISTS)
zookeeper->set(current_shards_path, host_id);
}
LOG_INFO(log, "Partition " << task_partition.name << " is safe for work now.");
return true; return true;
} }
@ -1362,6 +1549,7 @@ protected:
/// Process each source shard having current partition and copy current partition /// Process each source shard having current partition and copy current partition
/// NOTE: shards are sorted by "distance" to current host /// NOTE: shards are sorted by "distance" to current host
bool has_shard_to_process = false;
for (const TaskShardPtr & shard : task_table.all_shards) for (const TaskShardPtr & shard : task_table.all_shards)
{ {
/// Does shard have a node with current partition? /// Does shard have a node with current partition?
@ -1405,6 +1593,7 @@ protected:
bool is_unprioritized_task = !previous_shard_is_instantly_finished && shard->priority.is_remote; bool is_unprioritized_task = !previous_shard_is_instantly_finished && shard->priority.is_remote;
PartitionTaskStatus task_status = PartitionTaskStatus::Error; PartitionTaskStatus task_status = PartitionTaskStatus::Error;
bool was_error = false; bool was_error = false;
has_shard_to_process = true;
for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num)
{ {
task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task); task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task);
@ -1432,11 +1621,13 @@ protected:
cluster_partition.elapsed_time_seconds += watch.elapsedSeconds(); cluster_partition.elapsed_time_seconds += watch.elapsedSeconds();
/// Check that whole cluster partition is done /// Check that whole cluster partition is done
/// Firstly check number failed partition tasks, than look into ZooKeeper and ensure that each partition is done /// Firstly check the number of failed partition tasks, then look into ZooKeeper and ensure that each partition is done
bool partition_is_done = num_failed_shards == 0; bool partition_is_done = num_failed_shards == 0;
try try
{ {
partition_is_done = partition_is_done && checkPartitionIsDone(task_table, partition_name, expected_shards); partition_is_done =
!has_shard_to_process
|| (partition_is_done && checkPartitionIsDone(task_table, partition_name, expected_shards));
} }
catch (...) catch (...)
{ {
@ -1526,20 +1717,35 @@ protected:
TaskTable & task_table = task_shard.task_table; TaskTable & task_table = task_shard.task_table;
ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name);
/// We need to update table definitions for each partition, it could be changed after ALTER
createShardInternalTables(timeouts, task_shard);
auto zookeeper = context.getZooKeeper(); auto zookeeper = context.getZooKeeper();
String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath();
String current_task_is_active_path = task_partition.getActiveWorkerPath(); const String is_dirt_cleaned_path = task_partition.getCommonPartitionIsCleanedPath();
String current_task_status_path = task_partition.getShardStatusPath(); const String current_task_is_active_path = task_partition.getActiveWorkerPath();
const String current_task_status_path = task_partition.getShardStatusPath();
/// Auxiliary functions: /// Auxiliary functions:
/// Creates is_dirty node to initialize DROP PARTITION /// Creates is_dirty node to initialize DROP PARTITION
auto create_is_dirty_node = [&] () auto create_is_dirty_node = [&, this] (const CleanStateClock & clock)
{ {
auto code = zookeeper->tryCreate(is_dirty_flag_path, current_task_status_path, zkutil::CreateMode::Persistent); if (clock.is_stale())
if (code && code != Coordination::ZNODEEXISTS) LOG_DEBUG(log, "Clean state clock is stale while setting dirty flag, cowardly bailing");
throw Coordination::Exception(code, is_dirty_flag_path); else if (!clock.is_clean())
LOG_DEBUG(log, "Thank you, Captain Obvious");
else if (clock.discovery_version)
{
LOG_DEBUG(log, "Updating clean state clock");
zookeeper->set(is_dirty_flag_path, host_id, clock.discovery_version.value());
}
else
{
LOG_DEBUG(log, "Creating clean state clock");
zookeeper->create(is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent);
}
}; };
/// Returns SELECT query filtering current partition and applying user filter /// Returns SELECT query filtering current partition and applying user filter
@ -1563,14 +1769,29 @@ protected:
LOG_DEBUG(log, "Processing " << current_task_status_path); LOG_DEBUG(log, "Processing " << current_task_status_path);
CleanStateClock clean_state_clock (zookeeper, is_dirty_flag_path, is_dirt_cleaned_path);
LogicalClock task_start_clock;
{
Coordination::Stat stat;
if (zookeeper->exists(task_partition.getPartitionShardsPath(), &stat))
task_start_clock = LogicalClock(stat.mzxid);
}
/// Do not start if partition is dirty, try to clean it /// Do not start if partition is dirty, try to clean it
if (zookeeper->exists(is_dirty_flag_path)) if (clean_state_clock.is_clean()
&& (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock))
{
LOG_DEBUG(log, "Partition " << task_partition.name << " appears to be clean");
zookeeper->createAncestors(current_task_status_path);
}
else
{ {
LOG_DEBUG(log, "Partition " << task_partition.name << " is dirty, try to drop it"); LOG_DEBUG(log, "Partition " << task_partition.name << " is dirty, try to drop it");
try try
{ {
tryDropPartition(task_partition, zookeeper); tryDropPartition(task_partition, zookeeper, clean_state_clock);
} }
catch (...) catch (...)
{ {
@ -1598,7 +1819,8 @@ protected:
throw; throw;
} }
/// Exit if task has been already processed, create blocking node if it is abandoned /// Exit if task has been already processed;
/// create blocking node to signal cleaning up if it is abandoned
{ {
String status_data; String status_data;
if (zookeeper->tryGet(current_task_status_path, status_data)) if (zookeeper->tryGet(current_task_status_path, status_data))
@ -1611,21 +1833,21 @@ protected:
} }
// Task is abandoned, initialize DROP PARTITION // Task is abandoned, initialize DROP PARTITION
LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << status.owner); LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << status.owner << ". Partition will be dropped and refilled.");
create_is_dirty_node(); create_is_dirty_node(clean_state_clock);
return PartitionTaskStatus::Error; return PartitionTaskStatus::Error;
} }
} }
zookeeper->createAncestors(current_task_status_path);
/// We need to update table definitions for each partition, it could be changed after ALTER
createShardInternalTables(timeouts, task_shard);
/// Check that destination partition is empty if we are first worker /// Check that destination partition is empty if we are first worker
/// NOTE: this check is incorrect if pull and push tables have different partition key! /// NOTE: this check is incorrect if pull and push tables have different partition key!
String clean_start_status;
if (!zookeeper->tryGet(task_partition.getPartitionCleanStartPath(), clean_start_status) || clean_start_status != "ok")
{ {
zookeeper->createIfNotExists(task_partition.getPartitionCleanStartPath(), "");
auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", *zookeeper, host_id);
// Maybe we are the first worker
ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()"); ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()");
UInt64 count; UInt64 count;
{ {
@ -1643,36 +1865,38 @@ protected:
Coordination::Stat stat_shards; Coordination::Stat stat_shards;
zookeeper->get(task_partition.getPartitionShardsPath(), &stat_shards); zookeeper->get(task_partition.getPartitionShardsPath(), &stat_shards);
/// NOTE: partition is still fresh if dirt discovery happens before cleaning
if (stat_shards.numChildren == 0) if (stat_shards.numChildren == 0)
{ {
LOG_WARNING(log, "There are no any workers for partition " << task_partition.name LOG_WARNING(log, "There are no workers for partition " << task_partition.name
<< ", but destination table contains " << count << " rows" << ", but destination table contains " << count << " rows"
<< ". Partition will be dropped and refilled."); << ". Partition will be dropped and refilled.");
create_is_dirty_node(); create_is_dirty_node(clean_state_clock);
return PartitionTaskStatus::Error; return PartitionTaskStatus::Error;
} }
} }
zookeeper->set(task_partition.getPartitionCleanStartPath(), "ok");
} }
/// At this point, we need to sync that the destination table is clean
/// before any actual work
/// Try start processing, create node about it /// Try start processing, create node about it
{ {
String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id);
auto op_create = zkutil::makeCreateRequest(current_task_status_path, start_state, zkutil::CreateMode::Persistent); CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirt_cleaned_path);
MultiTransactionInfo info = checkNoNodeAndCommit(zookeeper, is_dirty_flag_path, std::move(op_create)); if (clean_state_clock != new_clean_state_clock)
if (info.code)
{ {
zkutil::KeeperMultiException exception(info.code, info.requests, info.responses); LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing");
if (exception.getPathForFirstFailedOp() == is_dirty_flag_path)
{
LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled");
return PartitionTaskStatus::Error; return PartitionTaskStatus::Error;
} }
else if (!new_clean_state_clock.is_clean())
throw exception; {
LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled");
create_is_dirty_node(new_clean_state_clock);
return PartitionTaskStatus::Error;
} }
zookeeper->create(current_task_status_path, start_state, zkutil::CreateMode::Persistent);
} }
/// Try create table (if not exists) on each shard /// Try create table (if not exists) on each shard
@ -1733,12 +1957,13 @@ protected:
output = io_insert.out; output = io_insert.out;
} }
/// Fail-fast optimization to abort copying when the current clean state expires
std::future<Coordination::ExistsResponse> future_is_dirty_checker; std::future<Coordination::ExistsResponse> future_is_dirty_checker;
Stopwatch watch(CLOCK_MONOTONIC_COARSE); Stopwatch watch(CLOCK_MONOTONIC_COARSE);
constexpr UInt64 check_period_milliseconds = 500; constexpr UInt64 check_period_milliseconds = 500;
/// Will asynchronously check that ZooKeeper connection and is_dirty flag appearing while copy data /// Will asynchronously check that ZooKeeper connection and is_dirty flag appearing while copying data
auto cancel_check = [&] () auto cancel_check = [&] ()
{ {
if (zookeeper->expired()) if (zookeeper->expired())
@ -1754,8 +1979,13 @@ protected:
Coordination::ExistsResponse status = future_is_dirty_checker.get(); Coordination::ExistsResponse status = future_is_dirty_checker.get();
if (status.error != Coordination::ZNONODE) if (status.error != Coordination::ZNONODE)
{
LogicalClock dirt_discovery_epoch (status.stat.mzxid);
if (dirt_discovery_epoch == clean_state_clock.discovery_zxid)
return false;
throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED); throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED);
} }
}
return false; return false;
}; };
@ -1789,20 +2019,19 @@ protected:
/// Finalize the processing, change state of current partition task (and also check is_dirty flag) /// Finalize the processing, change state of current partition task (and also check is_dirty flag)
{ {
String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id);
auto op_set = zkutil::makeSetRequest(current_task_status_path, state_finished, 0); CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirt_cleaned_path);
MultiTransactionInfo info = checkNoNodeAndCommit(zookeeper, is_dirty_flag_path, std::move(op_set)); if (clean_state_clock != new_clean_state_clock)
if (info.code)
{ {
zkutil::KeeperMultiException exception(info.code, info.requests, info.responses); LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing");
if (exception.getPathForFirstFailedOp() == is_dirty_flag_path)
LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled");
else
LOG_INFO(log, "Someone made the node abandoned. Will refill partition. " << zkutil::ZooKeeper::error2string(info.code));
return PartitionTaskStatus::Error; return PartitionTaskStatus::Error;
} }
else if (!new_clean_state_clock.is_clean())
{
LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled");
create_is_dirty_node(new_clean_state_clock);
return PartitionTaskStatus::Error;
}
zookeeper->set(current_task_status_path, state_finished, 0);
} }
LOG_INFO(log, "Partition " << task_partition.name << " copied"); LOG_INFO(log, "Partition " << task_partition.name << " copied");

View File

@ -438,6 +438,13 @@ int Server::main(const std::vector<std::string> & /*args*/)
buildLoggers(*config, logger()); buildLoggers(*config, logger());
global_context->setClustersConfig(config); global_context->setClustersConfig(config);
global_context->setMacros(std::make_unique<Macros>(*config, "macros")); global_context->setMacros(std::make_unique<Macros>(*config, "macros"));
/// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default)
if (config->has("max_table_size_to_drop"))
global_context->setMaxTableSizeToDrop(config->getUInt64("max_table_size_to_drop"));
if (config->has("max_partition_size_to_drop"))
global_context->setMaxPartitionSizeToDrop(config->getUInt64("max_partition_size_to_drop"));
}, },
/* already_loaded = */ true); /* already_loaded = */ true);
@ -469,13 +476,6 @@ int Server::main(const std::vector<std::string> & /*args*/)
/// Limit on total number of concurrently executed queries. /// Limit on total number of concurrently executed queries.
global_context->getProcessList().setMaxSize(config().getInt("max_concurrent_queries", 0)); global_context->getProcessList().setMaxSize(config().getInt("max_concurrent_queries", 0));
/// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default)
if (config().has("max_table_size_to_drop"))
global_context->setMaxTableSizeToDrop(config().getUInt64("max_table_size_to_drop"));
if (config().has("max_partition_size_to_drop"))
global_context->setMaxPartitionSizeToDrop(config().getUInt64("max_partition_size_to_drop"));
/// Set up caches. /// Set up caches.
/// Lower cache size on low-memory systems. /// Lower cache size on low-memory systems.

View File

@ -411,7 +411,7 @@
<!-- Protection from accidental DROP. <!-- Protection from accidental DROP.
If size of a MergeTree table is greater than max_table_size_to_drop (in bytes) than table could not be dropped with any DROP query. If size of a MergeTree table is greater than max_table_size_to_drop (in bytes) than table could not be dropped with any DROP query.
If you want do delete one table and don't want to restart clickhouse-server, you could create special file <clickhouse-path>/flags/force_drop_table and make DROP once. If you want do delete one table and don't want to change clickhouse-server config, you could create special file <clickhouse-path>/flags/force_drop_table and make DROP once.
By default max_table_size_to_drop is 50GB; max_table_size_to_drop=0 allows to DROP any tables. By default max_table_size_to_drop is 50GB; max_table_size_to_drop=0 allows to DROP any tables.
The same for max_partition_size_to_drop. The same for max_partition_size_to_drop.
Uncomment to disable protection. Uncomment to disable protection.

View File

@ -292,6 +292,8 @@ void registerOutputFormatProcessorTemplate(FormatFactory &factory);
/// File Segmentation Engines for parallel reading /// File Segmentation Engines for parallel reading
void registerFileSegmentationEngineTabSeparated(FormatFactory & factory); void registerFileSegmentationEngineTabSeparated(FormatFactory & factory);
void registerFileSegmentationEngineCSV(FormatFactory & factory);
void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory);
/// Output only (presentational) formats. /// Output only (presentational) formats.
@ -344,6 +346,8 @@ FormatFactory::FormatFactory()
registerOutputFormatProcessorTemplate(*this); registerOutputFormatProcessorTemplate(*this);
registerFileSegmentationEngineTabSeparated(*this); registerFileSegmentationEngineTabSeparated(*this);
registerFileSegmentationEngineCSV(*this);
registerFileSegmentationEngineJSONEachRow(*this);
registerOutputFormatNull(*this); registerOutputFormatNull(*this);

View File

@ -971,8 +971,16 @@ public:
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!isStringOrFixedString(arguments[0].type)) if (!isStringOrFixedString(arguments[0].type))
{
if (this->getName().find("OrZero") != std::string::npos ||
this->getName().find("OrNull") != std::string::npos)
throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() +
". Conversion functions with postfix 'OrZero' or 'OrNull' should take String argument",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
else
throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName(), throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
if (arguments.size() == 2) if (arguments.size() == 2)
{ {

View File

@ -1053,4 +1053,36 @@ void skipToUnescapedNextLineOrEOF(ReadBuffer & buf)
} }
} }
/// TODO (akuzm) - write comments for this and next function.
void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current)
{
assert(current >= in.position());
assert(current <= in.buffer().end());
const int old_bytes = memory.size();
const int additional_bytes = current - in.position();
const int new_bytes = old_bytes + additional_bytes;
/// There are no new bytes to add to memory.
/// No need to do extra stuff.
if (new_bytes == 0)
return;
memory.resize(new_bytes);
memcpy(memory.data() + old_bytes, in.position(), additional_bytes);
in.position() = current;
}
bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current)
{
assert(current <= in.buffer().end());
if (current < in.buffer().end())
return true;
saveUpToPosition(in, memory, current);
bool loaded_more = !in.eof();
assert(in.position() == in.buffer().begin());
current = in.position();
return loaded_more;
}
} }

View File

@ -924,4 +924,8 @@ if (method == DB::CompressionMethod::Gzip)
return std::make_unique<TReadBuffer>(args...); return std::make_unique<TReadBuffer>(args...);
} }
/// TODO (akuzm) - write comments for this and next function.
void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current);
bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current);
} }

View File

@ -151,8 +151,8 @@ struct ContextShared
mutable std::unique_ptr<DiskSpace::StoragePolicySelector> merge_tree_storage_policy_selector; mutable std::unique_ptr<DiskSpace::StoragePolicySelector> merge_tree_storage_policy_selector;
std::optional<MergeTreeSettings> merge_tree_settings; /// Settings of MergeTree* engines. std::optional<MergeTreeSettings> merge_tree_settings; /// Settings of MergeTree* engines.
size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default) std::atomic_size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default)
size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default) std::atomic_size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default)
String format_schema_path; /// Path to a directory that contains schema files used by input formats. String format_schema_path; /// Path to a directory that contains schema files used by input formats.
ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers
std::optional<SystemLogs> system_logs; /// Used to log queries and operations on parts std::optional<SystemLogs> system_logs; /// Used to log queries and operations on parts
@ -1891,14 +1891,14 @@ void Context::checkCanBeDropped(const String & database, const String & table, c
void Context::setMaxTableSizeToDrop(size_t max_size) void Context::setMaxTableSizeToDrop(size_t max_size)
{ {
// Is initialized at server startup // Is initialized at server startup and updated at config reload
shared->max_table_size_to_drop = max_size; shared->max_table_size_to_drop.store(max_size, std::memory_order_relaxed);
} }
void Context::checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const void Context::checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const
{ {
size_t max_table_size_to_drop = shared->max_table_size_to_drop; size_t max_table_size_to_drop = shared->max_table_size_to_drop.load(std::memory_order_relaxed);
checkCanBeDropped(database, table, table_size, max_table_size_to_drop); checkCanBeDropped(database, table, table_size, max_table_size_to_drop);
} }
@ -1906,14 +1906,14 @@ void Context::checkTableCanBeDropped(const String & database, const String & tab
void Context::setMaxPartitionSizeToDrop(size_t max_size) void Context::setMaxPartitionSizeToDrop(size_t max_size)
{ {
// Is initialized at server startup // Is initialized at server startup and updated at config reload
shared->max_partition_size_to_drop = max_size; shared->max_partition_size_to_drop.store(max_size, std::memory_order_relaxed);
} }
void Context::checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const void Context::checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const
{ {
size_t max_partition_size_to_drop = shared->max_partition_size_to_drop; size_t max_partition_size_to_drop = shared->max_partition_size_to_drop.load(std::memory_order_relaxed);
checkCanBeDropped(database, table, partition_size, max_partition_size_to_drop); checkCanBeDropped(database, table, partition_size, max_partition_size_to_drop);
} }

View File

@ -668,7 +668,7 @@ InterpreterSelectQuery::analyzeExpressions(
ExpressionActionsChain chain(context); ExpressionActionsChain chain(context);
Names additional_required_columns_after_prewhere; Names additional_required_columns_after_prewhere;
if (storage && query.sample_size()) if (storage && (query.sample_size() || context.getSettingsRef().parallel_replicas_count > 1))
{ {
Names columns_for_sampling = storage->getColumnsRequiredForSampling(); Names columns_for_sampling = storage->getColumnsRequiredForSampling();
additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(),
@ -1694,6 +1694,13 @@ void InterpreterSelectQuery::executeFetchColumns(
if (query_info.prewhere_info) if (query_info.prewhere_info)
{ {
if (query_info.prewhere_info->alias_actions)
{
streams.back() = std::make_shared<ExpressionBlockInputStream>(
streams.back(),
query_info.prewhere_info->alias_actions);
}
streams.back() = std::make_shared<FilterBlockInputStream>( streams.back() = std::make_shared<FilterBlockInputStream>(
streams.back(), streams.back(),
prewhere_info->prewhere_actions, prewhere_info->prewhere_actions,
@ -1718,6 +1725,10 @@ void InterpreterSelectQuery::executeFetchColumns(
if (query_info.prewhere_info) if (query_info.prewhere_info)
{ {
if (query_info.prewhere_info->alias_actions)
pipe.addSimpleTransform(std::make_shared<ExpressionTransform>(
pipe.getHeader(), query_info.prewhere_info->alias_actions));
pipe.addSimpleTransform(std::make_shared<FilterTransform>( pipe.addSimpleTransform(std::make_shared<FilterTransform>(
pipe.getHeader(), pipe.getHeader(),
prewhere_info->prewhere_actions, prewhere_info->prewhere_actions,

View File

@ -422,4 +422,64 @@ void registerInputFormatProcessorCSV(FormatFactory & factory)
} }
} }
bool fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
{
char * pos = in.position();
bool quotes = false;
bool need_more_data = true;
while (loadAtPosition(in, memory, pos) && need_more_data)
{
if (quotes)
{
pos = find_first_symbols<'"'>(pos, in.buffer().end());
if (pos == in.buffer().end())
continue;
if (*pos == '"')
{
++pos;
if (loadAtPosition(in, memory, pos) && *pos == '"')
++pos;
else
quotes = false;
}
}
else
{
pos = find_first_symbols<'"', '\r', '\n'>(pos, in.buffer().end());
if (pos == in.buffer().end())
continue;
if (*pos == '"')
{
quotes = true;
++pos;
}
else if (*pos == '\n')
{
if (memory.size() + static_cast<size_t>(pos - in.position()) >= min_chunk_size)
need_more_data = false;
++pos;
if (loadAtPosition(in, memory, pos) && *pos == '\r')
++pos;
}
else if (*pos == '\r')
{
if (memory.size() + static_cast<size_t>(pos - in.position()) >= min_chunk_size)
need_more_data = false;
++pos;
if (loadAtPosition(in, memory, pos) && *pos == '\n')
++pos;
}
}
}
saveUpToPosition(in, memory, pos);
return loadAtPosition(in, memory, pos);
}
void registerFileSegmentationEngineCSV(FormatFactory & factory)
{
factory.registerFileSegmentationEngine("CSV", &fileSegmentationEngineCSVImpl);
}
} }

View File

@ -270,4 +270,69 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory)
}); });
} }
bool fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
{
skipWhitespaceIfAny(in);
char * pos = in.position();
size_t balance = 0;
bool quotes = false;
while (loadAtPosition(in, memory, pos) && (balance || memory.size() + static_cast<size_t>(pos - in.position()) < min_chunk_size))
{
if (quotes)
{
pos = find_first_symbols<'\\', '"'>(pos, in.buffer().end());
if (pos == in.buffer().end())
continue;
if (*pos == '\\')
{
++pos;
if (loadAtPosition(in, memory, pos))
++pos;
}
else if (*pos == '"')
{
++pos;
quotes = false;
}
}
else
{
pos = find_first_symbols<'{', '}', '\\', '"'>(pos, in.buffer().end());
if (pos == in.buffer().end())
continue;
if (*pos == '{')
{
++balance;
++pos;
}
else if (*pos == '}')
{
--balance;
++pos;
}
else if (*pos == '\\')
{
++pos;
if (loadAtPosition(in, memory, pos))
++pos;
}
else if (*pos == '"')
{
quotes = true;
++pos;
}
}
}
saveUpToPosition(in, memory, pos);
return loadAtPosition(in, memory, pos);
}
void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory)
{
factory.registerFileSegmentationEngine("JSONEachRow", &fileSegmentationEngineJSONEachRowImpl);
}
} }

View File

@ -384,69 +384,32 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory)
} }
} }
void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current)
{
assert(current >= in.position());
assert(current <= in.buffer().end());
const int old_bytes = memory.size();
const int additional_bytes = current - in.position();
const int new_bytes = old_bytes + additional_bytes;
/// There are no new bytes to add to memory.
/// No need to do extra stuff.
if (new_bytes == 0)
return;
memory.resize(new_bytes);
memcpy(memory.data() + old_bytes, in.position(), additional_bytes);
in.position() = current;
}
bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current)
{
assert(current <= in.buffer().end());
if (current < in.buffer().end())
{
return true;
}
saveUpToPosition(in, memory, current);
bool loaded_more = !in.eof();
assert(in.position() == in.buffer().begin());
current = in.position();
return loaded_more;
}
bool fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) bool fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
{ {
bool need_more_data = true; bool need_more_data = true;
char * pos = in.position(); char * pos = in.position();
while (loadAtPosition(in, memory, pos) && need_more_data) while (loadAtPosition(in, memory, pos) && need_more_data)
{ {
pos = find_first_symbols<'\\', '\r', '\n'>(pos, in.buffer().end()); pos = find_first_symbols<'\\', '\r', '\n'>(pos, in.buffer().end());
if (pos == in.buffer().end()) if (pos == in.buffer().end())
{
continue; continue;
}
if (*pos == '\\') if (*pos == '\\')
{ {
++pos; ++pos;
if (loadAtPosition(in, memory, pos)) if (loadAtPosition(in, memory, pos))
{
++pos; ++pos;
} }
}
else if (*pos == '\n' || *pos == '\r') else if (*pos == '\n' || *pos == '\r')
{ {
if (memory.size() + static_cast<size_t>(pos - in.position()) >= min_chunk_size) if (memory.size() + static_cast<size_t>(pos - in.position()) >= min_chunk_size)
{
need_more_data = false; need_more_data = false;
}
++pos; ++pos;
} }
} }
saveUpToPosition(in, memory, pos); saveUpToPosition(in, memory, pos);
return loadAtPosition(in, memory, pos); return loadAtPosition(in, memory, pos);

View File

@ -2933,7 +2933,7 @@ MergeTreeData::getDetachedParts() const
{ {
std::vector<DetachedPartInfo> res; std::vector<DetachedPartInfo> res;
for (const String & path : getDataPaths()) for (const auto & [path, disk] : getDataPathsWithDisks())
{ {
for (Poco::DirectoryIterator it(path + "detached"); for (Poco::DirectoryIterator it(path + "detached");
it != Poco::DirectoryIterator(); ++it) it != Poco::DirectoryIterator(); ++it)
@ -2944,6 +2944,7 @@ MergeTreeData::getDetachedParts() const
auto & part = res.back(); auto & part = res.back();
DetachedPartInfo::tryParseDetachedPartName(dir_name, part, format_version); DetachedPartInfo::tryParseDetachedPartName(dir_name, part, format_version);
part.disk = disk->getName();
} }
} }
return res; return res;
@ -3332,6 +3333,15 @@ Strings MergeTreeData::getDataPaths() const
return res; return res;
} }
MergeTreeData::PathsWithDisks MergeTreeData::getDataPathsWithDisks() const
{
PathsWithDisks res;
auto disks = storage_policy->getDisks();
for (const auto & disk : disks)
res.emplace_back(getFullPathOnDisk(disk), disk);
return res;
}
void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & with_name, const Context & context) void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & with_name, const Context & context)
{ {
String clickhouse_path = Poco::Path(context.getPath()).makeAbsolute().toString(); String clickhouse_path = Poco::Path(context.getPath()).makeAbsolute().toString();

View File

@ -669,6 +669,10 @@ public:
Strings getDataPaths() const override; Strings getDataPaths() const override;
using PathWithDisk = std::pair<String, DiskSpace::DiskPtr>;
using PathsWithDisks = std::vector<PathWithDisk>;
PathsWithDisks getDataPathsWithDisks() const;
/// Reserves space at least 1MB /// Reserves space at least 1MB
DiskSpace::ReservationPtr reserveSpace(UInt64 expected_size); DiskSpace::ReservationPtr reserveSpace(UInt64 expected_size);

View File

@ -388,18 +388,18 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
used_sample_factor = 1.0 / boost::rational_cast<Float64>(relative_sample_size); used_sample_factor = 1.0 / boost::rational_cast<Float64>(relative_sample_size);
RelativeSize size_of_universum = 0; RelativeSize size_of_universum = 0;
DataTypePtr type = data.primary_key_sample.getByName(data.sampling_expr_column_name).type; DataTypePtr sampling_column_type = data.primary_key_sample.getByName(data.sampling_expr_column_name).type;
if (typeid_cast<const DataTypeUInt64 *>(type.get())) if (typeid_cast<const DataTypeUInt64 *>(sampling_column_type.get()))
size_of_universum = RelativeSize(std::numeric_limits<UInt64>::max()) + RelativeSize(1); size_of_universum = RelativeSize(std::numeric_limits<UInt64>::max()) + RelativeSize(1);
else if (typeid_cast<const DataTypeUInt32 *>(type.get())) else if (typeid_cast<const DataTypeUInt32 *>(sampling_column_type.get()))
size_of_universum = RelativeSize(std::numeric_limits<UInt32>::max()) + RelativeSize(1); size_of_universum = RelativeSize(std::numeric_limits<UInt32>::max()) + RelativeSize(1);
else if (typeid_cast<const DataTypeUInt16 *>(type.get())) else if (typeid_cast<const DataTypeUInt16 *>(sampling_column_type.get()))
size_of_universum = RelativeSize(std::numeric_limits<UInt16>::max()) + RelativeSize(1); size_of_universum = RelativeSize(std::numeric_limits<UInt16>::max()) + RelativeSize(1);
else if (typeid_cast<const DataTypeUInt8 *>(type.get())) else if (typeid_cast<const DataTypeUInt8 *>(sampling_column_type.get()))
size_of_universum = RelativeSize(std::numeric_limits<UInt8>::max()) + RelativeSize(1); size_of_universum = RelativeSize(std::numeric_limits<UInt8>::max()) + RelativeSize(1);
else else
throw Exception("Invalid sampling column type in storage parameters: " + type->getName() + ". Must be unsigned integer type.", throw Exception("Invalid sampling column type in storage parameters: " + sampling_column_type->getName() + ". Must be unsigned integer type.",
ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER); ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER);
if (settings.parallel_replicas_count > 1) if (settings.parallel_replicas_count > 1)
@ -453,13 +453,25 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
std::shared_ptr<ASTFunction> lower_function; std::shared_ptr<ASTFunction> lower_function;
std::shared_ptr<ASTFunction> upper_function; std::shared_ptr<ASTFunction> upper_function;
/// If sample and final are used together no need to calculate sampling expression twice.
/// The first time it was calculated for final, because sample key is a part of the PK.
/// So, assume that we already have calculated column.
ASTPtr sampling_key_ast = data.getSamplingKeyAST();
if (select.final())
{
sampling_key_ast = std::make_shared<ASTIdentifier>(data.sampling_expr_column_name);
/// We do spoil available_real_columns here, but it is not used later.
available_real_columns.emplace_back(data.sampling_expr_column_name, std::move(sampling_column_type));
}
if (has_lower_limit) if (has_lower_limit)
{ {
if (!key_condition.addCondition(data.sampling_expr_column_name, Range::createLeftBounded(lower, true))) if (!key_condition.addCondition(data.sampling_expr_column_name, Range::createLeftBounded(lower, true)))
throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN);
ASTPtr args = std::make_shared<ASTExpressionList>(); ASTPtr args = std::make_shared<ASTExpressionList>();
args->children.push_back(data.getSamplingKeyAST()); args->children.push_back(sampling_key_ast);
args->children.push_back(std::make_shared<ASTLiteral>(lower)); args->children.push_back(std::make_shared<ASTLiteral>(lower));
lower_function = std::make_shared<ASTFunction>(); lower_function = std::make_shared<ASTFunction>();
@ -476,7 +488,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN);
ASTPtr args = std::make_shared<ASTExpressionList>(); ASTPtr args = std::make_shared<ASTExpressionList>();
args->children.push_back(data.getSamplingKeyAST()); args->children.push_back(sampling_key_ast);
args->children.push_back(std::make_shared<ASTLiteral>(upper)); args->children.push_back(std::make_shared<ASTLiteral>(upper));
upper_function = std::make_shared<ASTFunction>(); upper_function = std::make_shared<ASTFunction>();
@ -503,11 +515,16 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
auto syntax_result = SyntaxAnalyzer(context).analyze(query, available_real_columns); auto syntax_result = SyntaxAnalyzer(context).analyze(query, available_real_columns);
filter_expression = ExpressionAnalyzer(filter_function, syntax_result, context).getActions(false); filter_expression = ExpressionAnalyzer(filter_function, syntax_result, context).getActions(false);
if (!select.final())
{
/// Add columns needed for `sample_by_ast` to `column_names_to_read`. /// Add columns needed for `sample_by_ast` to `column_names_to_read`.
/// Skip this if final was used, because such columns were already added from PK.
std::vector<String> add_columns = filter_expression->getRequiredColumns(); std::vector<String> add_columns = filter_expression->getRequiredColumns();
column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end());
std::sort(column_names_to_read.begin(), column_names_to_read.end()); std::sort(column_names_to_read.begin(), column_names_to_read.end());
column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), column_names_to_read.end()); column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()),
column_names_to_read.end());
}
} }
} }

View File

@ -95,6 +95,8 @@ struct DetachedPartInfo : public MergeTreePartInfo
String dir_name; String dir_name;
String prefix; String prefix;
String disk;
/// If false, MergeTreePartInfo is in invalid state (directory name was not successfully parsed). /// If false, MergeTreePartInfo is in invalid state (directory name was not successfully parsed).
bool valid_name; bool valid_name;

View File

@ -35,6 +35,7 @@ protected:
{"table", std::make_shared<DataTypeString>()}, {"table", std::make_shared<DataTypeString>()},
{"partition_id", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())}, {"partition_id", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
{"name", std::make_shared<DataTypeString>()}, {"name", std::make_shared<DataTypeString>()},
{"disk", std::make_shared<DataTypeString>()},
{"reason", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())}, {"reason", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
{"min_block_number", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeInt64>())}, {"min_block_number", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeInt64>())},
{"max_block_number", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeInt64>())}, {"max_block_number", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeInt64>())},
@ -66,6 +67,7 @@ protected:
new_columns[i++]->insert(info.table); new_columns[i++]->insert(info.table);
new_columns[i++]->insert(p.valid_name ? p.partition_id : Field()); new_columns[i++]->insert(p.valid_name ? p.partition_id : Field());
new_columns[i++]->insert(p.dir_name); new_columns[i++]->insert(p.dir_name);
new_columns[i++]->insert(p.disk);
new_columns[i++]->insert(p.valid_name ? p.prefix : Field()); new_columns[i++]->insert(p.valid_name ? p.prefix : Field());
new_columns[i++]->insert(p.valid_name ? p.min_block : Field()); new_columns[i++]->insert(p.valid_name ? p.min_block : Field());
new_columns[i++]->insert(p.valid_name ? p.max_block : Field()); new_columns[i++]->insert(p.valid_name ? p.max_block : Field());

View File

@ -1,268 +0,0 @@
Building of ClickHouse is supported on Linux, FreeBSD and Mac OS X.
# If you use Windows
If you use Windows, you need to create a virtual machine with Ubuntu. To start working with a virtual machine please install VirtualBox. You can download Ubuntu from the website: https://www.ubuntu.com/#download. Please create a virtual machine from the downloaded image (you should reserve at least 4GB of RAM for it). To run a command line terminal in Ubuntu, please locate a program containing the word "terminal" in its name (gnome-terminal, konsole etc.) or just press Ctrl+Alt+T.
# Creating a repository on GitHub
To start working with ClickHouse repository you will need a GitHub account.
You probably already have one, but if you don't, please register at https://github.com. In case you do not have SSH keys, you should generate them and then upload them on GitHub. It is required for sending over your patches. It is also possible to use the same SSH keys that you use with any other SSH servers - probably you already have those.
Create a fork of ClickHouse repository. To do that please click on the "fork" button in the upper right corner at https://github.com/ClickHouse/ClickHouse. It will fork your own copy of ClickHouse/ClickHouse to your account.
Development process consists of first committing the intended changes into your fork of ClickHouse and then creating a "pull request" for these changes to be accepted into the main repository (ClickHouse/ClickHouse).
To work with git repositories, please install `git`.
To do that in Ubuntu you would run in the command line terminal:
```
sudo apt update
sudo apt install git
```
A brief manual on using Git can be found here: https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf.
For a detailed manual on Git see: https://git-scm.com/book/ru/v2.
# Cloning a repository to your development machine
Next, you need to download the source files onto your working machine. This is called "to clone a repository" because it creates a local copy of the repository on your working machine.
In the command line terminal run:
```
git clone --recursive git@guthub.com:your_github_username/ClickHouse.git
cd ClickHouse
```
Note: please, substitute *your_github_username* with what is appropriate!
This command will create a directory `ClickHouse` containing the working copy of the project.
It is important that the path to the working directory contains no whitespaces as it may lead to problems with running the build system.
Please note that ClickHouse repository uses `submodules`. That is what the references to additional repositories are called (i.e. external libraries on which the project depends). It means that when cloning the repository you need to specify the `--recursive` flag as in the example above. If the repository has been cloned without submodules, to download them you need to run the following:
```
git submodule init
git submodule update
```
You can check status with command: `git submodule status`.
If you get the following error message:
```
Permission denied (publickey).
fatal: Could not read from remote repository.
Please make sure you have the correct access rights
and the repository exists.
```
It generally means that the SSH keys for connecting to GitHub are missing. These keys are normally located in `~/.ssh`. For SSH keys to be accepted you need to upload them in the settings section of GitHub UI.
You can also clone the repository via https protocol:
```
git clone https://github.com/ClickHouse/ClickHouse.git
```
This however will not let you send your changes to the server. You can still use it temporarily and add the SSH keys later replacing the remote address of the repository with `git remote` command.
You can also add original ClickHouse repo's address to your local repository to pull updates from there:
```
git remote add upstream git@github.com:ClickHouse/ClickHouse.git
```
After successfully running this command you will be able to pull updates from the main ClickHouse repo by running `git pull upstream master`.
# Build System
ClickHouse uses CMake and Ninja for building.
CMake - a meta-build system that can generate Ninja files (build tasks).
Ninja - a smaller build system with focus on speed used to execute those cmake generated tasks.
To install on Ubuntu, Debian or Mint run `sudo apt install cmake ninja-build`.
On CentOS, RedHat run `sudo yum install cmake ninja-build`.
If you use Arch or Gentoo, you probably know it yourself how to install CMake.
For installing CMake and Ninja on Mac OS X first install Homebrew and then install everything else via brew:
```
/usr/bin/ruby -e "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/master/install)"
brew install cmake ninja
```
Next, check the version of CMake: `cmake --version`. If it is below 3.3, you should install a newer version from the website: https://cmake.org/download/.
# Optional External Libraries
ClickHouse uses several external libraries for building. Most of them do not need to be installed separately as they are built together with ClickHouse from the sources located in the submodules. You can check the list in `contrib`.
There is a couple of libraries that are not built from sources but are supplied by the system: ICU and Readline, and thus are recommended to be installed.
Ubuntu: `sudo apt install libicu-dev libreadline-dev`
Mac OS X: `brew install icu4c readline`
However, these libraries are optional and ClickHouse can well be built without them. ICU is used for support of `COLLATE` in `ORDER BY` (i.e. for sorting in turkish alphabet). Readline is used for more convenient command input in clickhouse-client.
# C++ Compiler
Compilers GCC starting from version 9 and Clang version 8 or above are supported for building ClickHouse.
Official Yandex builds currently use GCC because it generates machine code of slightly better performance (yielding a difference of up to several percent according to our benchmarks). And Clang is more convenient for development usually. Though, our continuous integration (CI) platform runs checks for about a dozen of build combinations.
To install GCC on Ubuntu run: `sudo apt install gcc g++`
Check the version of gcc: `gcc --version`. If it is below 9, then follow the instruction here: https://clickhouse.yandex/docs/en/development/build/#install-gcc-9.
To install GCC on Mac OS X run: `brew install gcc`.
If you decide to use Clang, you can also install `libc++` and `lld`, if you know what it is. Using `ccache` is also recommended.
# The Building process
Now that you are ready to build ClickHouse we recommend you to create a separate directory `build` inside `ClickHouse` that will contain all of the build artefacts:
```
mkdir build
cd build
```
You can have several different directories (build_release, build_debug, etc.) for different types of build.
While inside the `build` directory, configure your build by running CMake. Before the first run you need to define environment variables that specify compiler (version 9 gcc compiler in this example).
```
export CC=gcc-9 CXX=g++-9
cmake ..
```
The `CC` variable specifies the compiler for C (short for C Compiler), and `CXX` variable instructs which C++ compiler is to be used for building.
For a faster build you can resort to the `debug` build type - a build with no optimizations. For that supply the following parameter `-D CMAKE_BUILD_TYPE=Debug`:
```
cmake -D CMAKE_BUILD_TYPE=Debug ..
```
You can change the type of build by running this command in the `build` directory.
Run ninja to build:
```
ninja clickhouse-server clickhouse-client
```
Only the required binaries are going to be built in this example.
If you require to build all the binaries (utilities and tests), you should run ninja with no parameters:
```
ninja
```
Full build requires about 30GB of free disk space or 15GB to build the main binaries.
When large amount of RAM is available on build machine you should limit the number of build tasks run in parallel with `-j` param:
```
ninja -j 1 clickhouse-server clickhouse-client
```
On machines with 4GB of RAM it is recommended to specify 1, for 8GB of RAM `-j 2` is recommended.
If you get the message: `ninja: error: loading 'build.ninja': No such file or directory`, it means that generating a build configuration has failed and you need to inspect the message above.
Upon successful start of the building process you'll see the build progress - the number of processed tasks and the total number of tasks.
While building messages about protobuf files in libhdfs2 library like `libprotobuf WARNING` may show up. They affect nothing and are safe to be ignored.
Upon successful build you get an executable file `ClickHouse/<build_dir>/dbms/programs/clickhouse`:
```
ls -l dbms/programs/clickhouse
```
# Running the built executable of ClickHouse
To run the server under the current user you need to navigate to `ClickHouse/dbms/programs/server/` (located outside of `build`) and run:
```
../../../build/dbms/programs/clickhouse server
```
In this case ClickHouse will use config files located in the current directory. You can run `clickhouse server` from any directory specifying the path to a config file as a command line parameter `--config-file`.
To connect to ClickHouse with clickhouse-client in another terminal navigate to `ClickHouse/build/dbms/programs/` and run `clickhouse client`.
If you get `Connection refused` message on Mac OS X or FreeBSD, try specifying host address 127.0.0.1:
```
clickhouse client --host 127.0.0.1
```
You can replace production version of ClickHouse binary installed in your system with your custom built ClickHouse binary. To do that install ClickHouse on your machine following the instructions from the official website. Next, run the following:
```
sudo service clickhouse-server stop
sudo cp ClickHouse/build/dbms/programs/clickhouse /usr/bin/
sudo service clickhouse-server start
```
Note that `clickhouse-client`, `clickhouse-server` and others are symlinks to the commonly shared `clickhouse` binary.
You can also run your custom built ClickHouse binary with the config file from the ClickHouse package installed on your system:
```
sudo service clickhouse-server stop
sudo -u clickhouse ClickHouse/build/dbms/programs/clickhouse server --config-file /etc/clickhouse-server/config.xml
```
# IDE (Integrated Development Environment)
If you do not know which IDE to use, we recommend that you use CLion. CLion is a commercial software, but it offers 30 day free trial period. It is also free of charge for students. CLion can be used both on Linux and on Mac OS X.
KDevelop and QTCreator are another great alternatives of an IDE for developing ClickHouse. KDevelop comes in as a very handy IDE although unstable. If KDevelop crashes after a while upon opening project, you should click "Stop All" button as soon as it has opened the list of project's files. After doing so KDevelop should be fine to work with.
As simple code editors you can use Sublime Text or Visual Studio Code, or Kate (all of which are available on Linux).
Just in case, it is worth mentioning that CLion creates by itself its own `build` path, it also selects by itself `debug` for build type, for configuration it uses a version of CMake that is defined in CLion and not the one installed by you, and finally CLion will use `make` to run build tasks instead of `ninja`. This is a normal behaviour, just keep that in mind to avoid confusion.
# Writing Code
The description of ClickHouse architecture can be found here: https://clickhouse.yandex/docs/en/development/architecture/
The Code Style Guide: https://clickhouse.yandex/docs/en/development/style/
Writing tests: https://clickhouse.yandex/docs/en/development/tests/
List of tasks: https://github.com/yandex/ClickHouse/blob/master/dbms/tests/instructions/easy_tasks_sorted_en.md
# Test Data
Developing ClickHouse often requires loading realistic datasets. It is particularly important for performance testing. We have a specially prepared set of anonymized data from Yandex.Metrica. It requires additionally some 3GB of free disk space. Note that this data is not required to accomplish most of development tasks.
```
sudo apt install wget xz-utils
wget https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_v1.tsv.xz
wget https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz
xz -v -d hits_v1.tsv.xz
xz -v -d visits_v1.tsv.xz
clickhouse-client
CREATE TABLE test.hits ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime);
CREATE TABLE test.visits ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), `Goals.ID` Array(UInt32), `Goals.Serial` Array(UInt32), `Goals.EventTime` Array(DateTime), `Goals.Price` Array(Int64), `Goals.OrderID` Array(String), `Goals.CurrencyID` Array(UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, `TraficSource.ID` Array(Int8), `TraficSource.SearchEngineID` Array(UInt16), `TraficSource.AdvEngineID` Array(UInt8), `TraficSource.PlaceID` Array(UInt16), `TraficSource.SocialSourceNetworkID` Array(UInt8), `TraficSource.Domain` Array(String), `TraficSource.SearchPhrase` Array(String), `TraficSource.SocialSourcePage` Array(String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `Market.Type` Array(UInt8), `Market.GoalID` Array(UInt32), `Market.OrderID` Array(String), `Market.OrderPrice` Array(Int64), `Market.PP` Array(UInt32), `Market.DirectPlaceID` Array(UInt32), `Market.DirectOrderID` Array(UInt32), `Market.DirectBannerID` Array(UInt32), `Market.GoodID` Array(String), `Market.GoodName` Array(String), `Market.GoodQuantity` Array(Int32), `Market.GoodPrice` Array(Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID);
clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.hits FORMAT TSV" < hits_v1.tsv
clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.visits FORMAT TSV" < visits_v1.tsv
```
# Creating Pull Request
Navigate to your fork repository in GitHub's UI. If you have been developing in a branch, you need to select that branch. There will be a "Pull request" button located on the screen. In essence this means "create a request for accepting my changes into the main repository".
A pull request can be created even if the work is not completed yet. In this case please put the word "WIP" (work in progress) at the beginning of the title, it can be changed later. This is useful for cooperative reviewing and discussion of changes as well as for running all of the available tests. It is important that you provide a brief description of your changes, it will later be used for generating realease changelogs.
Testing will commence as soon as Yandex employees label your PR with a tag "can be tested". The results of some first checks (e.g. code style) will come in within several minutes. Build check results will arrive within a half an hour. And the main set of tests will report itself within an hour.
The system will prepare ClickHouse binary builds for your pull request individually. To retrieve these builds click the "Details" link next to "ClickHouse build check" entry in the list of checks. There you will find direct links to the built .deb packages of ClickHouse which you can deploy even on your production servers (if you have no fear).
Most probably some of the builds will fail at first times. This is due to the fact that we check builds both with gcc as well as with clang, with almost all of existing warnings (always with the `-Werror` flag) enabled for clang. On that same page you can find all of the build logs so that you do not have to build ClickHouse in all of the possible ways.

View File

@ -0,0 +1 @@
../../../docs/en/development/developer_instruction.md

View File

@ -1,268 +0,0 @@
Сборка ClickHouse поддерживается на Linux, FreeBSD, Mac OS X.
# Если вы используете Windows
Если вы используете Windows, вам потребуется создать виртуальную машину с Ubuntu. Для работы с виртуальной машиной, установите VirtualBox. Скачать Ubuntu можно на сайте: https://www.ubuntu.com/#download Создайте виртуальную машину из полученного образа. Выделите для неё не менее 4 GB оперативной памяти. Для запуска терминала в Ubuntu, найдите в меню программу со словом terminal (gnome-terminal, konsole или что-то в этом роде) или нажмите Ctrl+Alt+T.
# Создание репозитория на GitHub
Для работы с репозиторием ClickHouse, вам потребуется аккаунт на GitHub. Наверное, он у вас уже есть.
Если аккаунта нет - зарегистрируйтесь на https://github.com/. Создайте ssh ключи, если их нет, и загрузите публичные ключи на GitHub. Это потребуется для отправки изменений. Для работы с GitHub можно использовать такие же ssh ключи, как и для работы с другими ssh серверами - скорее всего, они уже у вас есть.
Создайте fork репозитория ClickHouse. Для этого, на странице https://github.com/ClickHouse/ClickHouse нажмите на кнопку "fork" в правом верхнем углу. Вы получите полную копию репозитория ClickHouse на своём аккаунте, которая называется "форк". Процесс разработки состоит в том, чтобы внести нужные изменения в свой форк репозитория, а затем создать "pull request" для принятия изменений в основной репозиторий.
Для работы с git репозиториями, установите `git`.
В Ubuntu выполните в терминале:
```
sudo apt update
sudo apt install git
```
Краткое руководство по использованию Git: https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf
Подробное руководство по использованию Git: https://git-scm.com/book/ru/v2
# Клонирование репозитория на рабочую машину
Затем вам потребуется загрузить исходники для работы на свой компьютер. Это называется "клонирование репозитория", потому что создаёт на вашем компьютере локальную копию репозитория, с которой вы будете работать.
Выполните в терминале:
```
git clone --recursive git@github.com:yandex/ClickHouse.git
cd ClickHouse
```
Замените *yandex* на имя вашего аккаунта на GitHub.
Эта команда создаст директорию ClickHouse, содержащую рабочую копию проекта.
Необходимо, чтобы путь к рабочей копии не содержал пробелы в именах директорий. Это может привести к проблемам в работе системы сборки.
Обратите внимание, что репозиторий ClickHouse использует submodules. Так называются ссылки на дополнительные репозитории (например, внешние библиотеки, от которых зависит проект). Это значит, что при клонировании репозитория, следует указывать ключ `--recursive`, как в примере выше. Если репозиторий был клонирован без submodules, то для их скачивания, необходимо выполнить:
```
git submodule init
git submodule update
```
Проверить наличие submodules можно с помощью команды `git submodule status`.
Если вы получили сообщение об ошибке:
```
Permission denied (publickey).
fatal: Could not read from remote repository.
Please make sure you have the correct access rights
and the repository exists.
```
Как правило это означает, что отсутствуют ssh ключи для соединения с GitHub. Ключи расположены в директории `~/.ssh`. В интерфейсе GitHub, в настройках, необходимо загрузить публичные ключи, чтобы он их понимал.
Вы также можете клонировать репозиторий по протоколу https:
```
git clone https://github.com/ClickHouse/ClickHouse.git
```
Этот вариант не подходит для отправки изменений на сервер. Вы можете временно его использовать, а затем добавить ssh ключи и заменить адрес репозитория с помощью команды `git remote`.
Вы можете также добавить для своего локального репозитория адрес оригинального репозитория Яндекса, чтобы притягивать оттуда обновления:
```
git remote add upstream git@github.com:yandex/ClickHouse.git
```
После этого, вы сможете добавлять в свой репозиторий обновления из репозитория Яндекса с помощью команды `git pull upstream master`.
# Система сборки
ClickHouse использует систему сборки CMake и Ninja.
CMake - генератор задач сборки.
Ninja - система запуска сборочных задач.
Для установки на Ubuntu или Debian, Mint, выполните `sudo apt install cmake ninja-build`.
Для установки на CentOS, RedHat, выполните `sudo yum install cmake ninja-build`.
Если у вас Arch или Gentoo, то вы сами знаете, как установить CMake.
Для установки CMake и Ninja на Mac OS X, сначала установите Homebrew, а затем, с помощью него, установите всё остальное.
```
/usr/bin/ruby -e "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/master/install)"
brew install cmake ninja
```
Проверьте версию CMake: `cmake --version`. Если версия меньше 3.3, то установите новую версию с сайта https://cmake.org/download/
# Необязательные внешние библиотеки
ClickHouse использует для сборки некоторое количество внешних библиотек. Большинство из них не требуется отдельно устанавливать, так как они собираются вместе с ClickHouse, из исходников, которые расположены в submodules. Посмотреть набор этих библиотек можно в директории contrib.
Пара библиотек не собирается из исходников, а используется из системы: ICU и Readline, и их рекомендуется установить.
Ubuntu: `sudo apt install libicu-dev libreadline-dev`
Mac OS X: `brew install icu4c readline`
Впрочем, эти библиотеки не обязательны для работы и ClickHouse может быть собран без них. ICU используется для поддержки `COLLATE` в `ORDER BY` (например, для сортировки с учётом турецкого алфавита). Readline используется для более удобного набора команд в интерактивном режиме в clickhouse-client.
# Компилятор C++
В качестве компилятора C++ поддерживается GCC начиная с версии 9 или Clang начиная с версии 8.
Официальные сборки от Яндекса, на данный момент, используют GCC, так как он генерирует слегка более производительный машинный код (разница в среднем до нескольких процентов по нашим бенчмаркам). Clang обычно более удобен для разработки. Впрочем, наша среда continuous integration проверяет около десятка вариантов сборки.
Для установки GCC под Ubuntu, выполните: `sudo apt install gcc g++`.
Проверьте версию gcc: `gcc --version`. Если версия меньше 9, то следуйте инструкции: https://clickhouse.yandex/docs/en/development/build/#install-gcc-9
Для установки GCC под Mac OS X, выполните `brew install gcc`.
Если вы решили использовать Clang, вы также можете установить `libc++` и `lld`, если вы знаете, что это такое. При желании, установите `ccache`.
# Процесс сборки
Теперь вы готовы к сборке ClickHouse. Для размещения собранных файлов, рекомендуется создать отдельную директорию build внутри директории ClickHouse:
```
mkdir build
cd build
```
Вы можете иметь несколько разных директорий (build_release, build_debug) для разных вариантов сборки.
Находясь в директории build, выполните конфигурацию сборки с помощью CMake.
Перед первым запуском необходимо выставить переменные окружения, отвечающие за выбор компилятора (в данном примере это - gcc версии 9).
```
export CC=gcc-9 CXX=g++-9
cmake ..
```
Переменная CC отвечает за компилятор C (сокращение от слов C Compiler), переменная CXX отвечает за выбор компилятора C++ (символ X - это как плюс, но положенный набок, ради того, чтобы превратить его в букву).
Для более быстрой сборки, можно использовать debug вариант - сборку без оптимизаций. Для этого, укажите параметр `-D CMAKE_BUILD_TYPE=Debug`:
```
cmake -D CMAKE_BUILD_TYPE=Debug ..
```
Вы можете изменить вариант сборки, выполнив эту команду в директории build.
Запустите ninja для сборки:
```
ninja clickhouse-server clickhouse-client
```
В этом примере собираются только нужные в первую очередь программы.
Если вы хотите собрать все программы (утилиты и тесты), то запустите ninja без параметров:
```
ninja
```
Для полной сборки требуется около 30 GB свободного места на диске или 15 GB для сборки только основных программ.
При наличии небольшого количества оперативной памяти на компьютере, следует ограничить количество параллельных задач с помощью параметра `-j`:
```
ninja -j 1 clickhouse-server clickhouse-client
```
На машинах с 4 GB памяти, рекомендуется указывать значение 1, а если памяти до 8 GB, укажите значение 2.
Если вы получили сообщение `ninja: error: loading 'build.ninja': No such file or directory`, значит конфигурация сборки прошла с ошибкой и вам необходимо посмотреть на сообщение об ошибке выше.
В случае успешного запуска, вы увидите прогресс сборки - количество обработанных задач и общее количество задач.
В процессе сборки могут появится сообщения `libprotobuf WARNING` про protobuf файлы в библиотеке libhdfs2. Это не имеет значения.
При успешной сборке, вы получите готовый исполняемый файл `ClickHouse/build/dbms/programs/clickhouse`:
```
ls -l dbms/programs/clickhouse
```
# Запуск собранной версии ClickHouse
Для запуска сервера из под текущего пользователя, с выводом логов в терминал и с использованием примеров конфигурационных файлов, расположенных в исходниках, перейдите в директорию `ClickHouse/dbms/programs/server/` (эта директория находится не в директории build) и выполните:
```
../../../build/dbms/programs/clickhouse server
```
В этом случае, ClickHouse будет использовать конфигурационные файлы, расположенные в текущей директории. Вы можете запустить `clickhouse server` из любой директории, передав ему путь к конфигурационному файлу в аргументе командной строки `--config-file`.
Для подключения к ClickHouse с помощью clickhouse-client, в соседнем терминале, зайдите в директорию `ClickHouse/build/dbms/programs/` и выполните `clickhouse client`.
Если вы получили сообщение `Connection refused` на Mac OS X или FreeBSD, то укажите для клиента 127.0.0.1 в качестве имени хоста:
```
clickhouse client --host 127.0.0.1
```
Вы можете заменить собранным вами ClickHouse продакшен версию, установленную в системе. Для этого, установите ClickHouse на свою машину по инструкции с официального сайта. Затем выполните:
```
sudo service clickhouse-server stop
sudo cp ClickHouse/build/dbms/programs/clickhouse /usr/bin/
sudo service clickhouse-server start
```
Обратите внимание, что `clickhouse-client`, `clickhouse-server` и другие, являеются симлинками на общий бинарник `clickhouse`.
Также вы можете запустить собранный вами ClickHouse с конфигурационным файлом системного ClickHouse:
```
sudo service clickhouse-server stop
sudo -u clickhouse ClickHouse/build/dbms/programs/clickhouse server --config-file /etc/clickhouse-server/config.xml
```
# Среда разработки
Если вы не знаете, какую среду разработки использовать, то рекомендуется использовать CLion. CLion является платным ПО, но его можно использовать бесплатно в течение пробного периода. Также он бесплатен для учащихся. CLion можно использовать как под Linux, так и под Mac OS X.
Также в качестве среды разработки, вы можете использовать KDevelop или QTCreator. KDevelop - очень удобная, но нестабильная среда разработки. Если KDevelop вылетает через небольшое время после открытия проекта, вам следует нажать на кнопку "Stop All" как только он открыл список файлов проекта. После этого, KDevelop можно будет использовать.
В качестве простых редакторов кода можно использовать Sublime Text или Visual Studio Code или Kate (все варианты доступны под Linux).
На всякий случай заметим, что CLion самостоятельно создаёт свою build директорию, самостоятельно выбирает тип сборки debug по-умолчанию, для конфигурации использует встроенную в CLion версию CMake вместо установленного вами, а для запуска задач использует make вместо ninja. Это нормально, просто имейте это ввиду, чтобы не возникало путаницы.
# Написание кода
Описание архитектуры ClickHouse: https://clickhouse.yandex/docs/ru/development/architecture/
Стиль кода: https://clickhouse.yandex/docs/ru/development/style/
Разработка тестов: https://clickhouse.yandex/docs/ru/development/tests/
Список задач: https://github.com/ClickHouse/ClickHouse/blob/master/dbms/tests/instructions/easy_tasks_sorted_ru.md
# Тестовые данные
Разработка ClickHouse часто требует загрузки реалистичных наборов данных. Особенно это важно для тестирования производительности. Специально для вас мы подготовили набор данных, представляющий собой анонимизированные данные Яндекс.Метрики. Загрузка этих данных потребует ещё 3 GB места на диске. Для выполнения большинства задач разработки, загружать эти данные не обязательно.
```
sudo apt install wget xz-utils
wget https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_v1.tsv.xz
wget https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz
xz -v -d hits_v1.tsv.xz
xz -v -d visits_v1.tsv.xz
clickhouse-client
CREATE TABLE test.hits ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime);
CREATE TABLE test.visits ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), `Goals.ID` Array(UInt32), `Goals.Serial` Array(UInt32), `Goals.EventTime` Array(DateTime), `Goals.Price` Array(Int64), `Goals.OrderID` Array(String), `Goals.CurrencyID` Array(UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, `TraficSource.ID` Array(Int8), `TraficSource.SearchEngineID` Array(UInt16), `TraficSource.AdvEngineID` Array(UInt8), `TraficSource.PlaceID` Array(UInt16), `TraficSource.SocialSourceNetworkID` Array(UInt8), `TraficSource.Domain` Array(String), `TraficSource.SearchPhrase` Array(String), `TraficSource.SocialSourcePage` Array(String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `Market.Type` Array(UInt8), `Market.GoalID` Array(UInt32), `Market.OrderID` Array(String), `Market.OrderPrice` Array(Int64), `Market.PP` Array(UInt32), `Market.DirectPlaceID` Array(UInt32), `Market.DirectOrderID` Array(UInt32), `Market.DirectBannerID` Array(UInt32), `Market.GoodID` Array(String), `Market.GoodName` Array(String), `Market.GoodQuantity` Array(Int32), `Market.GoodPrice` Array(Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID);
clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.hits FORMAT TSV" < hits_v1.tsv
clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.visits FORMAT TSV" < visits_v1.tsv
```
# Создание pull request
Откройте свой форк репозитория в интерфейсе GitHub. Если вы вели разработку в бранче, выберите этот бранч. На странице будет доступна кнопка "Pull request". По сути, это означает "создать заявку на принятие моих изменений в основной репозиторий".
Pull request можно создать, даже если работа над задачей ещё не завершена. В этом случае, добавьте в его название слово "WIP" (work in progress). Название можно будет изменить позже. Это полезно для совместного просмотра и обсуждения изменений, а также для запуска всех имеющихся тестов. Введите краткое описание изменений - впоследствии, оно будет использовано для релизных changelog.
Тесты будут запущены, как только сотрудники Яндекса поставят для pull request тег "Can be tested". Результаты первых проверок (стиль кода) появятся уже через несколько минут. Результаты сборки появятся примерно через пол часа. Результаты основного набора тестов будут доступны в пределах часа.
Система подготовит сборки ClickHouse специально для вашего pull request. Для их получения, нажмите на ссылку "Details" у проверки "Clickhouse build check". Там вы сможете найти прямые ссылки на собранные .deb пакеты ClickHouse, которые, при желании, вы даже сможете установить на свои продакшен серверы (если не страшно).
Вероятнее всего, часть сборок не будет успешной с первого раза. Ведь мы проверяем сборку кода и gcc и clang, а при сборке с помощью clang включаются почти все существующие в природе warnings (всегда с флагом `-Werror`). На той же странице, вы сможете найти логи сборки - вам не обязательно самому собирать ClickHouse всеми возможными способами.

View File

@ -0,0 +1 @@
../../../docs/ru/development/developer_instruction.md

View File

@ -14,9 +14,9 @@ Don't use Docker from your system repository.
* [pip](https://pypi.python.org/pypi/pip) and `libpq-dev`. To install: `sudo apt-get install python-pip libpq-dev` * [pip](https://pypi.python.org/pypi/pip) and `libpq-dev`. To install: `sudo apt-get install python-pip libpq-dev`
* [py.test](https://docs.pytest.org/) testing framework. To install: `sudo -H pip install pytest` * [py.test](https://docs.pytest.org/) testing framework. To install: `sudo -H pip install pytest`
* [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: `sudo -H pip install docker-compose docker dicttoxml kazoo PyMySQL psycopg2 pymongo tzlocal kafka-python protobuf pytest-timeout` * [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: `sudo -H pip install docker-compose docker dicttoxml kazoo PyMySQL psycopg2 pymongo tzlocal kafka-python protobuf pytest-timeout minio`
(highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python-pytest python-dicttoxml python-docker python-pymysql python-pymongo python-tzlocal python-kazoo python-psycopg2 python-kafka python-pytest-timeout` (highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python-pytest python-dicttoxml python-docker python-pymysql python-pymongo python-tzlocal python-kazoo python-psycopg2 python-kafka python-pytest-timeout python-minio`
If you want to run the tests under a non-privileged user, you must add this user to `docker` group: `sudo usermod -aG docker $USER` and re-login. If you want to run the tests under a non-privileged user, you must add this user to `docker` group: `sudo usermod -aG docker $USER` and re-login.
(You must close all your sessions (for example, restart your computer)) (You must close all your sessions (for example, restart your computer))

View File

@ -1,31 +1,29 @@
import base64
import distutils.dir_util
import errno
import os import os
import os.path as p import os.path as p
import pwd import pwd
import re import re
import subprocess
import shutil import shutil
import distutils.dir_util
import socket import socket
import subprocess
import time import time
import errno import urllib
from dicttoxml import dicttoxml
import pymysql
import xml.dom.minidom import xml.dom.minidom
import logging
import docker
import psycopg2
import pymongo
import pymysql
from dicttoxml import dicttoxml
from kazoo.client import KazooClient from kazoo.client import KazooClient
from kazoo.exceptions import KazooException from kazoo.exceptions import KazooException
import psycopg2 from minio import Minio
import requests
import base64
import pymongo
import urllib
import docker from .client import Client
from docker.errors import ContainerError
from .client import Client, CommandRequest
from .hdfs_api import HDFSApi from .hdfs_api import HDFSApi
HELPERS_DIR = p.dirname(__file__) HELPERS_DIR = p.dirname(__file__)
DEFAULT_ENV_NAME = 'env_file' DEFAULT_ENV_NAME = 'env_file'
@ -39,16 +37,19 @@ def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME):
f.write("=".join([var, value]) + "\n") f.write("=".join([var, value]) + "\n")
return full_path return full_path
def subprocess_check_call(args): def subprocess_check_call(args):
# Uncomment for debugging # Uncomment for debugging
# print('run:', ' ' . join(args)) # print('run:', ' ' . join(args))
subprocess.check_call(args) subprocess.check_call(args)
def subprocess_call(args): def subprocess_call(args):
# Uncomment for debugging # Uncomment for debugging
# print('run:', ' ' . join(args)) # print('run:', ' ' . join(args))
subprocess.call(args) subprocess.call(args)
def get_odbc_bridge_path(): def get_odbc_bridge_path():
path = os.environ.get('CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH') path = os.environ.get('CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH')
if path is None: if path is None:
@ -74,11 +75,15 @@ class ClickHouseCluster:
self.base_dir = p.dirname(base_path) self.base_dir = p.dirname(base_path)
self.name = name if name is not None else '' self.name = name if name is not None else ''
self.base_configs_dir = base_configs_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR', '/etc/clickhouse-server/') self.base_configs_dir = base_configs_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR',
self.server_bin_path = p.realpath(server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse')) '/etc/clickhouse-server/')
self.server_bin_path = p.realpath(
server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse'))
self.odbc_bridge_bin_path = p.realpath(odbc_bridge_bin_path or get_odbc_bridge_path()) self.odbc_bridge_bin_path = p.realpath(odbc_bridge_bin_path or get_odbc_bridge_path())
self.client_bin_path = p.realpath(client_bin_path or os.environ.get('CLICKHOUSE_TESTS_CLIENT_BIN_PATH', '/usr/bin/clickhouse-client')) self.client_bin_path = p.realpath(
self.zookeeper_config_path = p.join(self.base_dir, zookeeper_config_path) if zookeeper_config_path else p.join(HELPERS_DIR, 'zookeeper_config.xml') client_bin_path or os.environ.get('CLICKHOUSE_TESTS_CLIENT_BIN_PATH', '/usr/bin/clickhouse-client'))
self.zookeeper_config_path = p.join(self.base_dir, zookeeper_config_path) if zookeeper_config_path else p.join(
HELPERS_DIR, 'zookeeper_config.xml')
self.project_name = pwd.getpwuid(os.getuid()).pw_name + p.basename(self.base_dir) + self.name self.project_name = pwd.getpwuid(os.getuid()).pw_name + p.basename(self.base_dir) + self.name
# docker-compose removes everything non-alphanumeric from project names so we do it too. # docker-compose removes everything non-alphanumeric from project names so we do it too.
@ -109,17 +114,29 @@ class ClickHouseCluster:
self.with_net_trics = False self.with_net_trics = False
self.with_redis = False self.with_redis = False
self.with_minio = False
self.minio_host = "minio1"
self.minio_bucket = "root"
self.minio_port = 9001
self.minio_client = None # type: Minio
self.minio_redirect_host = "redirect"
self.minio_redirect_port = 80
self.docker_client = None self.docker_client = None
self.is_up = False self.is_up = False
def get_client_cmd(self): def get_client_cmd(self):
cmd = self.client_bin_path cmd = self.client_bin_path
if p.basename(cmd) == 'clickhouse': if p.basename(cmd) == 'clickhouse':
cmd += " client" cmd += " client"
return cmd return cmd
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, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, with_redis=False, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=[]): 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,
with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False,
with_redis=False, with_minio=False,
hostname=None, env_variables={}, image="yandex/clickhouse-integration-test",
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=[]):
"""Add an instance to the cluster. """Add an instance to the cluster.
name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. name - the name of the instance directory and the value of the 'instance' macro in ClickHouse.
@ -137,9 +154,11 @@ class ClickHouseCluster:
instance = ClickHouseInstance( instance = ClickHouseInstance(
self, self.base_dir, name, config_dir, main_configs, user_configs, macros, with_zookeeper, self, self.base_dir, name, config_dir, main_configs, user_configs, macros, with_zookeeper,
self.zookeeper_config_path, with_mysql, with_kafka, with_mongo, with_redis, self.base_configs_dir, self.server_bin_path, self.zookeeper_config_path, with_mysql, with_kafka, with_mongo, with_redis, with_minio,
self.base_configs_dir, self.server_bin_path,
self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname, self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname,
env_variables=env_variables, image=image, stay_alive=stay_alive, ipv4_address=ipv4_address, ipv6_address=ipv6_address, env_variables=env_variables, image=image, stay_alive=stay_alive, ipv4_address=ipv4_address,
ipv6_address=ipv6_address,
with_installed_binary=with_installed_binary, tmpfs=tmpfs) with_installed_binary=with_installed_binary, tmpfs=tmpfs)
self.instances[name] = instance self.instances[name] = instance
@ -185,7 +204,8 @@ class ClickHouseCluster:
self.with_postgres = True self.with_postgres = True
self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')]) self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')])
self.base_postgres_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', self.base_postgres_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')] self.project_name, '--file',
p.join(HELPERS_DIR, 'docker_compose_postgres.yml')]
cmds.append(self.base_postgres_cmd) cmds.append(self.base_postgres_cmd)
if with_kafka and not self.with_kafka: if with_kafka and not self.with_kafka:
@ -219,10 +239,15 @@ class ClickHouseCluster:
self.base_redis_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', self.base_redis_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_redis.yml')] self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_redis.yml')]
if with_minio and not self.with_minio:
self.with_minio = True
self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_minio.yml')])
self.base_minio_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_minio.yml')]
cmds.append(self.base_minio_cmd)
return instance return instance
def get_instance_docker_id(self, instance_name): def get_instance_docker_id(self, instance_name):
# According to how docker-compose names containers. # According to how docker-compose names containers.
return self.project_name + '_' + instance_name + '_1' return self.project_name + '_' + instance_name + '_1'
@ -331,19 +356,39 @@ class ClickHouseCluster:
print "Can't connect to Mongo " + str(ex) print "Can't connect to Mongo " + str(ex)
time.sleep(1) time.sleep(1)
def wait_minio_to_start(self, timeout=10):
minio_client = Minio('localhost:9001',
access_key='minio',
secret_key='minio123',
secure=False)
start = time.time()
while time.time() - start < timeout:
try:
buckets = minio_client.list_buckets()
self.minio_client = minio_client
logging.info("Connected to Minio %s", buckets)
return
except Exception as ex:
logging.warning("Can't connect to Minio: %s", str(ex))
time.sleep(1)
def start(self, destroy_dirs=True): def start(self, destroy_dirs=True):
if self.is_up: if self.is_up:
return return
# Just in case kill unstopped containers from previous launch # Just in case kill unstopped containers from previous launch
try: try:
logging.info("Trying to kill unstopped containers...")
if not subprocess_call(['docker-compose', 'kill']): if not subprocess_call(['docker-compose', 'kill']):
subprocess_call(['docker-compose', 'down', '--volumes']) subprocess_call(['docker-compose', 'down', '--volumes'])
except: except:
pass pass
logging.info("Unstopped containers killed")
if destroy_dirs and p.exists(self.instances_dir): if destroy_dirs and p.exists(self.instances_dir):
print "Removing instances dir", self.instances_dir logging.info("Removing instances dir %s", self.instances_dir)
shutil.rmtree(self.instances_dir) shutil.rmtree(self.instances_dir)
for instance in self.instances.values(): for instance in self.instances.values():
@ -383,21 +428,31 @@ class ClickHouseCluster:
subprocess_check_call(self.base_redis_cmd + ['up', '-d', '--force-recreate']) subprocess_check_call(self.base_redis_cmd + ['up', '-d', '--force-recreate'])
time.sleep(10) time.sleep(10)
if self.with_minio and self.base_minio_cmd:
minio_start_cmd = self.base_minio_cmd + common_opts
logging.info("Trying to create Minio instance by command %s", ' '.join(map(str, minio_start_cmd)))
subprocess_check_call(minio_start_cmd)
logging.info("Trying to connect to Minio...")
self.wait_minio_to_start()
subprocess_check_call(self.base_cmd + ['up', '-d', '--no-recreate']) clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate']
logging.info("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd)))
subprocess_check_call(clickhouse_start_cmd)
logging.info("ClickHouse instance created")
start_deadline = time.time() + 20.0 # seconds start_deadline = time.time() + 20.0 # seconds
for instance in self.instances.itervalues(): for instance in self.instances.itervalues():
instance.docker_client = self.docker_client instance.docker_client = self.docker_client
instance.ip_address = self.get_instance_ip(instance.name) instance.ip_address = self.get_instance_ip(instance.name)
logging.info("Waiting for ClickHouse start...")
instance.wait_for_start(start_deadline) instance.wait_for_start(start_deadline)
logging.info("ClickHouse started")
instance.client = Client(instance.ip_address, command=self.client_bin_path) instance.client = Client(instance.ip_address, command=self.client_bin_path)
self.is_up = True self.is_up = True
def shutdown(self, kill=True): def shutdown(self, kill=True):
sanitizer_assert_instance = None sanitizer_assert_instance = None
with open(self.docker_logs_path, "w+") as f: with open(self.docker_logs_path, "w+") as f:
@ -411,6 +466,7 @@ class ClickHouseCluster:
if kill: if kill:
subprocess_check_call(self.base_cmd + ['kill']) subprocess_check_call(self.base_cmd + ['kill'])
subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans']) subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans'])
self.is_up = False self.is_up = False
self.docker_client = None self.docker_client = None
@ -427,13 +483,11 @@ class ClickHouseCluster:
def open_bash_shell(self, instance_name): def open_bash_shell(self, instance_name):
os.system(' '.join(self.base_cmd + ['exec', instance_name, '/bin/bash'])) os.system(' '.join(self.base_cmd + ['exec', instance_name, '/bin/bash']))
def get_kazoo_client(self, zoo_instance_name): def get_kazoo_client(self, zoo_instance_name):
zk = KazooClient(hosts=self.get_instance_ip(zoo_instance_name)) zk = KazooClient(hosts=self.get_instance_ip(zoo_instance_name))
zk.start() zk.start()
return zk return zk
def run_kazoo_commands_with_retries(self, kazoo_callback, zoo_instance_name='zoo1', repeats=1, sleep_for=1): def run_kazoo_commands_with_retries(self, kazoo_callback, zoo_instance_name='zoo1', repeats=1, sleep_for=1):
for i in range(repeats - 1): for i in range(repeats - 1):
try: try:
@ -445,7 +499,6 @@ class ClickHouseCluster:
kazoo_callback(self.get_kazoo_client(zoo_instance_name)) kazoo_callback(self.get_kazoo_client(zoo_instance_name))
def add_zookeeper_startup_command(self, command): def add_zookeeper_startup_command(self, command):
self.pre_zookeeper_commands.append(command) self.pre_zookeeper_commands.append(command)
@ -488,8 +541,10 @@ class ClickHouseInstance:
def __init__( def __init__(
self, cluster, base_path, name, custom_config_dir, custom_main_configs, custom_user_configs, macros, self, cluster, base_path, name, custom_config_dir, custom_main_configs, custom_user_configs, macros,
with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_mongo, with_redis, base_configs_dir, server_bin_path, odbc_bridge_bin_path, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_mongo, with_redis, with_minio,
clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", base_configs_dir, server_bin_path, odbc_bridge_bin_path,
clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables={},
image="yandex/clickhouse-integration-test",
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=[]): stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=[]):
self.name = name self.name = name
@ -515,6 +570,7 @@ class ClickHouseInstance:
self.with_kafka = with_kafka self.with_kafka = with_kafka
self.with_mongo = with_mongo self.with_mongo = with_mongo
self.with_redis = with_redis self.with_redis = with_redis
self.with_minio = with_minio
self.path = p.join(self.cluster.instances_dir, name) self.path = p.join(self.cluster.instances_dir, name)
self.docker_compose_path = p.join(self.path, 'docker_compose.yml') self.docker_compose_path = p.join(self.path, 'docker_compose.yml')
@ -539,7 +595,8 @@ class ClickHouseInstance:
def query(self, sql, stdin=None, timeout=None, settings=None, user=None, ignore_error=False): def query(self, sql, stdin=None, timeout=None, settings=None, user=None, ignore_error=False):
return self.client.query(sql, stdin, timeout, settings, user, ignore_error) return self.client.query(sql, stdin, timeout, settings, user, ignore_error)
def query_with_retry(self, sql, stdin=None, timeout=None, settings=None, user=None, ignore_error=False, retry_count=20, sleep_time=0.5, check_callback=lambda x: True): def query_with_retry(self, sql, stdin=None, timeout=None, settings=None, user=None, ignore_error=False,
retry_count=20, sleep_time=0.5, check_callback=lambda x: True):
result = None result = None
for i in range(retry_count): for i in range(retry_count):
try: try:
@ -591,17 +648,21 @@ class ClickHouseInstance:
return output return output
def contains_in_log(self, substring): def contains_in_log(self, substring):
result = self.exec_in_container(["bash", "-c", "grep '{}' /var/log/clickhouse-server/clickhouse-server.log || true".format(substring)]) result = self.exec_in_container(
["bash", "-c", "grep '{}' /var/log/clickhouse-server/clickhouse-server.log || true".format(substring)])
return len(result) > 0 return len(result) > 0
def copy_file_to_container(self, local_path, dest_path): def copy_file_to_container(self, local_path, dest_path):
with open(local_path, 'r') as fdata: with open(local_path, 'r') as fdata:
data = fdata.read() data = fdata.read()
encoded_data = base64.b64encode(data) encoded_data = base64.b64encode(data)
self.exec_in_container(["bash", "-c", "echo {} | base64 --decode > {}".format(encoded_data, dest_path)], user='root') self.exec_in_container(["bash", "-c", "echo {} | base64 --decode > {}".format(encoded_data, dest_path)],
user='root')
def get_process_pid(self, process_name): def get_process_pid(self, process_name):
output = self.exec_in_container(["bash", "-c", "ps ax | grep '{}' | grep -v 'grep' | grep -v 'bash -c' | awk '{{print $1}}'".format(process_name)]) output = self.exec_in_container(["bash", "-c",
"ps ax | grep '{}' | grep -v 'grep' | grep -v 'bash -c' | awk '{{print $1}}'".format(
process_name)])
if output: if output:
try: try:
pid = int(output.split('\n')[0].strip()) pid = int(output.split('\n')[0].strip())
@ -610,7 +671,6 @@ class ClickHouseInstance:
return None return None
return None return None
def restart_with_latest_version(self, stop_start_wait_sec=10, callback_onstop=None, signal=15): def restart_with_latest_version(self, stop_start_wait_sec=10, callback_onstop=None, signal=15):
if not self.stay_alive: if not self.stay_alive:
raise Exception("Cannot restart not stay alive container") raise Exception("Cannot restart not stay alive container")
@ -626,8 +686,12 @@ class ClickHouseInstance:
if callback_onstop: if callback_onstop:
callback_onstop(self) callback_onstop(self)
self.exec_in_container(["bash", "-c", "cp /usr/share/clickhouse_fresh /usr/bin/clickhouse && chmod 777 /usr/bin/clickhouse"], user='root') self.exec_in_container(
self.exec_in_container(["bash", "-c", "cp /usr/share/clickhouse-odbc-bridge_fresh /usr/bin/clickhouse-odbc-bridge && chmod 777 /usr/bin/clickhouse"], user='root') ["bash", "-c", "cp /usr/share/clickhouse_fresh /usr/bin/clickhouse && chmod 777 /usr/bin/clickhouse"],
user='root')
self.exec_in_container(["bash", "-c",
"cp /usr/share/clickhouse-odbc-bridge_fresh /usr/bin/clickhouse-odbc-bridge && chmod 777 /usr/bin/clickhouse"],
user='root')
self.exec_in_container(["bash", "-c", "{} --daemon".format(CLICKHOUSE_START_COMMAND)], user=str(os.getuid())) self.exec_in_container(["bash", "-c", "{} --daemon".format(CLICKHOUSE_START_COMMAND)], user=str(os.getuid()))
from helpers.test_tools import assert_eq_with_retry from helpers.test_tools import assert_eq_with_retry
# wait start # wait start
@ -636,15 +700,12 @@ class ClickHouseInstance:
def get_docker_handle(self): def get_docker_handle(self):
return self.docker_client.containers.get(self.docker_id) return self.docker_client.containers.get(self.docker_id)
def stop(self): def stop(self):
self.get_docker_handle().stop() self.get_docker_handle().stop()
def start(self): def start(self):
self.get_docker_handle().start() self.get_docker_handle().start()
def wait_for_start(self, deadline=None, timeout=None): def wait_for_start(self, deadline=None, timeout=None):
start_time = time.time() start_time = time.time()
@ -653,9 +714,11 @@ class ClickHouseInstance:
while True: while True:
handle = self.get_docker_handle() handle = self.get_docker_handle()
status = handle.status; status = handle.status
if status == 'exited': if status == 'exited':
raise Exception("Instance `{}' failed to start. Container status: {}, logs: {}".format(self.name, status, handle.logs())) raise Exception(
"Instance `{}' failed to start. Container status: {}, logs: {}".format(self.name, status,
handle.logs()))
current_time = time.time() current_time = time.time()
time_left = deadline - current_time time_left = deadline - current_time
@ -680,7 +743,6 @@ class ClickHouseInstance:
finally: finally:
sock.close() sock.close()
@staticmethod @staticmethod
def dict_to_xml(dictionary): def dict_to_xml(dictionary):
xml_str = dicttoxml(dictionary, custom_root="yandex", attr_type=False) xml_str = dicttoxml(dictionary, custom_root="yandex", attr_type=False)
@ -805,6 +867,10 @@ class ClickHouseInstance:
depends_on.append("zoo2") depends_on.append("zoo2")
depends_on.append("zoo3") depends_on.append("zoo3")
if self.with_minio:
depends_on.append("minio1")
depends_on.append("redirect")
env_file = _create_env_file(os.path.dirname(self.docker_compose_path), self.env_variables) env_file = _create_env_file(os.path.dirname(self.docker_compose_path), self.env_variables)
odbc_ini_path = "" odbc_ini_path = ""
@ -836,7 +902,6 @@ class ClickHouseInstance:
binary_volume = "- " + self.server_bin_path + ":/usr/share/clickhouse_fresh" binary_volume = "- " + self.server_bin_path + ":/usr/share/clickhouse_fresh"
odbc_bridge_volume = "- " + self.odbc_bridge_bin_path + ":/usr/share/clickhouse-odbc-bridge_fresh" odbc_bridge_volume = "- " + self.odbc_bridge_bin_path + ":/usr/share/clickhouse-odbc-bridge_fresh"
with open(self.docker_compose_path, 'w') as docker_compose: with open(self.docker_compose_path, 'w') as docker_compose:
docker_compose.write(DOCKER_COMPOSE_TEMPLATE.format( docker_compose.write(DOCKER_COMPOSE_TEMPLATE.format(
image=self.image, image=self.image,
@ -860,7 +925,6 @@ class ClickHouseInstance:
ipv6_address=ipv6_address, ipv6_address=ipv6_address,
)) ))
def destroy_dir(self): def destroy_dir(self):
if p.exists(self.path): if p.exists(self.path):
shutil.rmtree(self.path) shutil.rmtree(self.path)

View File

@ -0,0 +1,31 @@
version: '2.2'
services:
minio1:
image: minio/minio
volumes:
- data1-1:/data1
ports:
- "9001:9001"
environment:
MINIO_ACCESS_KEY: minio
MINIO_SECRET_KEY: minio123
command: server --address :9001 /data1-1
healthcheck:
test: ["CMD", "curl", "-f", "http://localhost:9001/minio/health/live"]
interval: 30s
timeout: 20s
retries: 3
# Redirects all requests to origin Minio.
redirect:
image: schmunk42/nginx-redirect
volumes:
- /nginx:/nginx
environment:
- SERVER_REDIRECT=minio1:9001
- SERVER_REDIRECT_CODE=307
- SERVER_ACCESS_LOG=/nginx/access.log
volumes:
data1-1:

View File

@ -768,12 +768,42 @@ def test_concurrent_alter_move_and_drop(start_cluster, name, engine):
node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) node1.query("DROP TABLE IF EXISTS {name}".format(name=name))
@pytest.mark.parametrize("name,engine", [
("detach_attach_mt","MergeTree()"),
("replicated_detach_attach_mt","ReplicatedMergeTree('/clickhouse/replicated_detach_attach_mt', '1')",),
])
def test_detach_attach(start_cluster, name, engine):
try:
node1.query("""
CREATE TABLE {name} (
s1 String
) ENGINE = {engine}
ORDER BY tuple()
SETTINGS storage_policy='moving_jbod_with_external'
""".format(name=name, engine=engine))
data = [] # 5MB in total
for i in range(5):
data.append(get_random_string(1024 * 1024)) # 1MB row
node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data])))
node1.query("ALTER TABLE {} DETACH PARTITION tuple()".format(name))
assert node1.query("SELECT count() FROM {}".format(name)).strip() == "0"
assert node1.query("SELECT disk FROM system.detached_parts WHERE table = '{}'".format(name)).strip() == "jbod1"
node1.query("ALTER TABLE {} ATTACH PARTITION tuple()".format(name))
assert node1.query("SELECT count() FROM {}".format(name)).strip() == "5"
finally:
node1.query("DROP TABLE IF EXISTS {name}".format(name=name))
@pytest.mark.parametrize("name,engine", [ @pytest.mark.parametrize("name,engine", [
("mutating_mt","MergeTree()"), ("mutating_mt","MergeTree()"),
("replicated_mutating_mt","ReplicatedMergeTree('/clickhouse/replicated_mutating_mt', '1')",), ("replicated_mutating_mt","ReplicatedMergeTree('/clickhouse/replicated_mutating_mt', '1')",),
]) ])
def test_mutate_to_another_disk(start_cluster, name, engine): def test_mutate_to_another_disk(start_cluster, name, engine):
try: try:
node1.query(""" node1.query("""
CREATE TABLE {name} ( CREATE TABLE {name} (

View File

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

View File

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

View File

@ -0,0 +1,49 @@
import time
import pytest
import os
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', config_dir="configs")
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
CONFIG_PATH = os.path.join(SCRIPT_DIR, './_instances/node/configs/config.xml')
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
node.query("CREATE TABLE test(date Date, id UInt32) ENGINE = MergeTree() PARTITION BY date ORDER BY id")
yield cluster
finally:
cluster.shutdown()
def test_reload_max_table_size_to_drop(start_cluster):
node.query("INSERT INTO test VALUES (now(), 0)")
time.sleep(5) # wait for data part commit
drop = node.get_query_request("DROP TABLE test")
out, err = drop.get_answer_and_error()
assert out == ""
assert err != ""
config = open(CONFIG_PATH, 'r')
config_lines = config.readlines()
config.close()
config_lines = map(lambda line: line.replace("<max_table_size_to_drop>1", "<max_table_size_to_drop>1000000"),
config_lines)
config = open(CONFIG_PATH, 'w')
config.writelines(config_lines)
config.close()
node.query("SYSTEM RELOAD CONFIG")
drop = node.get_query_request("DROP TABLE test")
out, err = drop.get_answer_and_error()
assert out == ""
assert err == ""

View File

@ -1,368 +0,0 @@
try:
from BaseHTTPServer import BaseHTTPRequestHandler
except ImportError:
from http.server import BaseHTTPRequestHandler
try:
from BaseHTTPServer import HTTPServer
except ImportError:
from http.server import HTTPServer
try:
import urllib.parse as urlparse
except ImportError:
import urlparse
import json
import logging
import os
import socket
import sys
import threading
import time
import uuid
import xml.etree.ElementTree
BASE_DIR = os.path.dirname(__file__)
logging.getLogger().setLevel(logging.INFO)
file_handler = logging.FileHandler(os.path.join(BASE_DIR, "test-server.log"), "a", encoding="utf-8")
file_handler.setFormatter(logging.Formatter("%(asctime)s %(message)s"))
logging.getLogger().addHandler(file_handler)
logging.getLogger().addHandler(logging.StreamHandler())
communication_port = int(sys.argv[1])
bucket = sys.argv[2]
def GetFreeTCPPortsAndIP(n):
result = []
sockets = []
for i in range(n):
tcp = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
tcp.bind((socket.gethostname(), 0))
addr, port = tcp.getsockname()
result.append(port)
sockets.append(tcp)
[ s.close() for s in sockets ]
return result, addr
(
redirecting_to_http_port,
simple_server_port,
preserving_data_port,
multipart_preserving_data_port,
redirecting_preserving_data_port
), localhost = GetFreeTCPPortsAndIP(5)
data = {
"redirecting_to_http_port": redirecting_to_http_port,
"preserving_data_port": preserving_data_port,
"multipart_preserving_data_port": multipart_preserving_data_port,
"redirecting_preserving_data_port": redirecting_preserving_data_port,
}
class SimpleHTTPServerHandler(BaseHTTPRequestHandler):
def do_GET(self):
logging.info("GET {}".format(self.path))
if self.path == "/milovidov/test.csv":
self.send_response(200)
self.send_header("Content-type", "text/plain")
self.end_headers()
data["redirect_csv_data"] = [[42, 87, 44], [55, 33, 81], [1, 0, 9]]
self.wfile.write("".join([ "{},{},{}\n".format(*row) for row in data["redirect_csv_data"]]))
else:
self.send_response(404)
self.end_headers()
self.finish()
class RedirectingToHTTPHandler(BaseHTTPRequestHandler):
def do_GET(self):
self.send_response(307)
self.send_header("Content-type", "text/xml")
self.send_header("Location", "http://{}:{}/milovidov/test.csv".format(localhost, simple_server_port))
self.end_headers()
self.wfile.write(r"""<?xml version="1.0" encoding="UTF-8"?>
<Error>
<Code>TemporaryRedirect</Code>
<Message>Please re-send this request to the specified temporary endpoint.
Continue to use the original request endpoint for future requests.</Message>
<Endpoint>storage.yandexcloud.net</Endpoint>
</Error>""".encode())
self.finish()
class PreservingDataHandler(BaseHTTPRequestHandler):
protocol_version = "HTTP/1.1"
def parse_request(self):
result = BaseHTTPRequestHandler.parse_request(self)
# Adaptation to Python 3.
if sys.version_info.major == 2 and result == True:
expect = self.headers.get("Expect", "")
if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"):
if not self.handle_expect_100():
return False
return result
def send_response_only(self, code, message=None):
if message is None:
if code in self.responses:
message = self.responses[code][0]
else:
message = ""
if self.request_version != "HTTP/0.9":
self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message))
def handle_expect_100(self):
logging.info("Received Expect-100")
self.send_response_only(100)
self.end_headers()
return True
def do_POST(self):
self.send_response(200)
query = urlparse.urlparse(self.path).query
logging.info("PreservingDataHandler POST ?" + query)
if query == "uploads":
post_data = r"""<?xml version="1.0" encoding="UTF-8"?>
<hi><UploadId>TEST</UploadId></hi>""".encode()
self.send_header("Content-length", str(len(post_data)))
self.send_header("Content-type", "text/plain")
self.end_headers()
self.wfile.write(post_data)
else:
post_data = self.rfile.read(int(self.headers.get("Content-Length")))
self.send_header("Content-type", "text/plain")
self.end_headers()
data["received_data_completed"] = True
data["finalize_data"] = post_data
data["finalize_data_query"] = query
self.finish()
def do_PUT(self):
self.send_response(200)
self.send_header("Content-type", "text/plain")
self.send_header("ETag", "hello-etag")
self.end_headers()
query = urlparse.urlparse(self.path).query
path = urlparse.urlparse(self.path).path
logging.info("Content-Length = " + self.headers.get("Content-Length"))
logging.info("PUT " + query)
assert self.headers.get("Content-Length")
assert self.headers["Expect"] == "100-continue"
put_data = self.rfile.read()
data.setdefault("received_data", []).append(put_data)
logging.info("PUT to {}".format(path))
self.server.storage[path] = put_data
self.finish()
def do_GET(self):
path = urlparse.urlparse(self.path).path
if path in self.server.storage:
self.send_response(200)
self.send_header("Content-type", "text/plain")
self.send_header("Content-length", str(len(self.server.storage[path])))
self.end_headers()
self.wfile.write(self.server.storage[path])
else:
self.send_response(404)
self.end_headers()
self.finish()
class MultipartPreservingDataHandler(BaseHTTPRequestHandler):
protocol_version = "HTTP/1.1"
def parse_request(self):
result = BaseHTTPRequestHandler.parse_request(self)
# Adaptation to Python 3.
if sys.version_info.major == 2 and result == True:
expect = self.headers.get("Expect", "")
if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"):
if not self.handle_expect_100():
return False
return result
def send_response_only(self, code, message=None):
if message is None:
if code in self.responses:
message = self.responses[code][0]
else:
message = ""
if self.request_version != "HTTP/0.9":
self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message))
def handle_expect_100(self):
logging.info("Received Expect-100")
self.send_response_only(100)
self.end_headers()
return True
def do_POST(self):
query = urlparse.urlparse(self.path).query
logging.info("MultipartPreservingDataHandler POST ?" + query)
if query == "uploads":
self.send_response(200)
post_data = r"""<?xml version="1.0" encoding="UTF-8"?>
<hi><UploadId>TEST</UploadId></hi>""".encode()
self.send_header("Content-length", str(len(post_data)))
self.send_header("Content-type", "text/plain")
self.end_headers()
self.wfile.write(post_data)
else:
try:
assert query == "uploadId=TEST"
logging.info("Content-Length = " + self.headers.get("Content-Length"))
post_data = self.rfile.read(int(self.headers.get("Content-Length")))
root = xml.etree.ElementTree.fromstring(post_data)
assert root.tag == "CompleteMultipartUpload"
assert len(root) > 1
content = ""
for i, part in enumerate(root):
assert part.tag == "Part"
assert len(part) == 2
assert part[0].tag == "PartNumber"
assert part[1].tag == "ETag"
assert int(part[0].text) == i + 1
content += self.server.storage["@"+part[1].text]
data.setdefault("multipart_received_data", []).append(content)
data["multipart_parts"] = len(root)
self.send_response(200)
self.send_header("Content-type", "text/plain")
self.end_headers()
logging.info("Sending 200")
except:
logging.error("Sending 500")
self.send_response(500)
self.finish()
def do_PUT(self):
uid = uuid.uuid4()
self.send_response(200)
self.send_header("Content-type", "text/plain")
self.send_header("ETag", str(uid))
self.end_headers()
query = urlparse.urlparse(self.path).query
path = urlparse.urlparse(self.path).path
logging.info("Content-Length = " + self.headers.get("Content-Length"))
logging.info("PUT " + query)
assert self.headers.get("Content-Length")
assert self.headers["Expect"] == "100-continue"
put_data = self.rfile.read()
data.setdefault("received_data", []).append(put_data)
logging.info("PUT to {}".format(path))
self.server.storage["@"+str(uid)] = put_data
self.finish()
def do_GET(self):
path = urlparse.urlparse(self.path).path
if path in self.server.storage:
self.send_response(200)
self.send_header("Content-type", "text/plain")
self.send_header("Content-length", str(len(self.server.storage[path])))
self.end_headers()
self.wfile.write(self.server.storage[path])
else:
self.send_response(404)
self.end_headers()
self.finish()
class RedirectingPreservingDataHandler(BaseHTTPRequestHandler):
protocol_version = "HTTP/1.1"
def parse_request(self):
result = BaseHTTPRequestHandler.parse_request(self)
# Adaptation to Python 3.
if sys.version_info.major == 2 and result == True:
expect = self.headers.get("Expect", "")
if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"):
if not self.handle_expect_100():
return False
return result
def send_response_only(self, code, message=None):
if message is None:
if code in self.responses:
message = self.responses[code][0]
else:
message = ""
if self.request_version != "HTTP/0.9":
self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message))
def handle_expect_100(self):
logging.info("Received Expect-100")
return True
def do_POST(self):
query = urlparse.urlparse(self.path).query
if query:
query = "?{}".format(query)
self.send_response(307)
self.send_header("Content-type", "text/xml")
self.send_header("Location", "http://{host}:{port}/{bucket}/test.csv{query}".format(host=localhost, port=preserving_data_port, bucket=bucket, query=query))
self.end_headers()
self.wfile.write(r"""<?xml version="1.0" encoding="UTF-8"?>
<Error>
<Code>TemporaryRedirect</Code>
<Message>Please re-send this request to the specified temporary endpoint.
Continue to use the original request endpoint for future requests.</Message>
<Endpoint>{host}:{port}</Endpoint>
</Error>""".format(host=localhost, port=preserving_data_port).encode())
self.finish()
def do_PUT(self):
query = urlparse.urlparse(self.path).query
if query:
query = "?{}".format(query)
self.send_response(307)
self.send_header("Content-type", "text/xml")
self.send_header("Location", "http://{host}:{port}/{bucket}/test.csv{query}".format(host=localhost, port=preserving_data_port, bucket=bucket, query=query))
self.end_headers()
self.wfile.write(r"""<?xml version="1.0" encoding="UTF-8"?>
<Error>
<Code>TemporaryRedirect</Code>
<Message>Please re-send this request to the specified temporary endpoint.
Continue to use the original request endpoint for future requests.</Message>
<Endpoint>{host}:{port}</Endpoint>
</Error>""".format(host=localhost, port=preserving_data_port).encode())
self.finish()
class CommunicationServerHandler(BaseHTTPRequestHandler):
def do_GET(self):
self.send_response(200)
self.end_headers()
self.wfile.write(json.dumps(data))
self.finish()
def do_PUT(self):
self.send_response(200)
self.end_headers()
logging.info(self.rfile.read())
self.finish()
servers = []
servers.append(HTTPServer((localhost, communication_port), CommunicationServerHandler))
servers.append(HTTPServer((localhost, redirecting_to_http_port), RedirectingToHTTPHandler))
servers.append(HTTPServer((localhost, preserving_data_port), PreservingDataHandler))
servers[-1].storage = {}
servers.append(HTTPServer((localhost, multipart_preserving_data_port), MultipartPreservingDataHandler))
servers[-1].storage = {}
servers.append(HTTPServer((localhost, simple_server_port), SimpleHTTPServerHandler))
servers.append(HTTPServer((localhost, redirecting_preserving_data_port), RedirectingPreservingDataHandler))
jobs = [ threading.Thread(target=server.serve_forever) for server in servers ]
[ job.start() for job in jobs ]
time.sleep(60) # Timeout
logging.info("Shutting down")
[ server.shutdown() for server in servers ]
logging.info("Joining threads")
[ job.join() for job in jobs ]
logging.info("Done")

View File

@ -1,159 +1,191 @@
import httplib
import json import json
import logging import logging
import os
import time
import traceback
import pytest import pytest
from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseCluster, ClickHouseInstance
logging.getLogger().setLevel(logging.INFO) logging.getLogger().setLevel(logging.INFO)
logging.getLogger().addHandler(logging.StreamHandler()) logging.getLogger().addHandler(logging.StreamHandler())
def get_communication_data(started_cluster): # Creates S3 bucket for tests and allows anonymous read-write access to it.
conn = httplib.HTTPConnection(started_cluster.instances["dummy"].ip_address, started_cluster.communication_port) def prepare_s3_bucket(cluster):
conn.request("GET", "/") minio_client = cluster.minio_client
r = conn.getresponse()
raw_data = r.read() if minio_client.bucket_exists(cluster.minio_bucket):
conn.close() minio_client.remove_bucket(cluster.minio_bucket)
return json.loads(raw_data)
minio_client.make_bucket(cluster.minio_bucket)
# Allows read-write access for bucket without authorization.
bucket_read_write_policy = {"Version": "2012-10-17",
"Statement": [
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:GetBucketLocation",
"Resource": "arn:aws:s3:::root"
},
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:ListBucket",
"Resource": "arn:aws:s3:::root"
},
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:GetObject",
"Resource": "arn:aws:s3:::root/*"
},
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:PutObject",
"Resource": "arn:aws:s3:::root/*"
}
]}
minio_client.set_bucket_policy(cluster.minio_bucket, json.dumps(bucket_read_write_policy))
def put_communication_data(started_cluster, body): # Returns content of given S3 file as string.
conn = httplib.HTTPConnection(started_cluster.instances["dummy"].ip_address, started_cluster.communication_port) def get_s3_file_content(cluster, filename):
conn.request("PUT", "/", body) # type: (ClickHouseCluster, str) -> str
r = conn.getresponse()
conn.close() data = cluster.minio_client.get_object(cluster.minio_bucket, filename)
data_str = ""
for chunk in data.stream():
data_str += chunk
return data_str
# Returns nginx access log lines.
def get_nginx_access_logs():
handle = open("/nginx/access.log", "r")
data = handle.readlines()
handle.close()
return data
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
def started_cluster(): def cluster():
try: try:
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance("dummy") cluster.add_instance("dummy", with_minio=True)
logging.info("Starting cluster...")
cluster.start() cluster.start()
logging.info("Cluster started")
cluster.communication_port = 10000 prepare_s3_bucket(cluster)
instance.copy_file_to_container(os.path.join(os.path.dirname(__file__), "server.py"), "test_server.py") logging.info("S3 bucket created")
cluster.bucket = "abc"
instance.exec_in_container(["python", "test_server.py", str(cluster.communication_port), cluster.bucket], detach=True)
cluster.mock_host = instance.ip_address
for i in range(10):
try:
data = get_communication_data(cluster)
cluster.redirecting_to_http_port = data["redirecting_to_http_port"]
cluster.preserving_data_port = data["preserving_data_port"]
cluster.multipart_preserving_data_port = data["multipart_preserving_data_port"]
cluster.redirecting_preserving_data_port = data["redirecting_preserving_data_port"]
except:
logging.error(traceback.format_exc())
time.sleep(0.5)
else:
break
else:
assert False, "Could not initialize mock server"
yield cluster yield cluster
finally: finally:
cluster.shutdown() cluster.shutdown()
def run_query(instance, query, stdin=None, settings=None): def run_query(instance, query, stdin=None, settings=None):
# type: (ClickHouseInstance, str, object, dict) -> str
logging.info("Running query '{}'...".format(query)) logging.info("Running query '{}'...".format(query))
result = instance.query(query, stdin=stdin, settings=settings) result = instance.query(query, stdin=stdin, settings=settings)
logging.info("Query finished") logging.info("Query finished")
return result return result
def test_get_with_redirect(started_cluster): # Test simple put.
instance = started_cluster.instances["dummy"] def test_put(cluster):
format = "column1 UInt32, column2 UInt32, column3 UInt32" # type: (ClickHouseCluster) -> None
put_communication_data(started_cluster, "=== Get with redirect test ===") instance = cluster.instances["dummy"] # type: ClickHouseInstance
query = "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(started_cluster.mock_host, started_cluster.redirecting_to_http_port, format) table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
stdout = run_query(instance, query)
data = get_communication_data(started_cluster)
expected = [ [str(row[0]), str(row[1]), str(row[2]), str(row[0]*row[1]*row[2])] for row in data["redirect_csv_data"] ]
assert list(map(str.split, stdout.splitlines())) == expected
def test_put(started_cluster):
instance = started_cluster.instances["dummy"]
format = "column1 UInt32, column2 UInt32, column3 UInt32"
logging.info("Phase 3")
put_communication_data(started_cluster, "=== Put test ===")
values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)"
put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(started_cluster.mock_host, started_cluster.preserving_data_port, started_cluster.bucket, format, values) values_csv = "1,2,3\n3,2,1\n78,43,45\n"
filename = "test.csv"
put_query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format(
cluster.minio_host, cluster.minio_port, cluster.minio_bucket, filename, table_format, values)
run_query(instance, put_query) run_query(instance, put_query)
data = get_communication_data(started_cluster)
received_data_completed = data["received_data_completed"] assert values_csv == get_s3_file_content(cluster, filename)
received_data = data["received_data"]
finalize_data = data["finalize_data"]
finalize_data_query = data["finalize_data_query"]
assert received_data[-1].decode() == "1,2,3\n3,2,1\n78,43,45\n"
assert received_data_completed
assert finalize_data == "<CompleteMultipartUpload><Part><PartNumber>1</PartNumber><ETag>hello-etag</ETag></Part></CompleteMultipartUpload>"
assert finalize_data_query == "uploadId=TEST"
def test_put_csv(started_cluster): # Test put values in CSV format.
instance = started_cluster.instances["dummy"] def test_put_csv(cluster):
format = "column1 UInt32, column2 UInt32, column3 UInt32" # type: (ClickHouseCluster) -> None
put_communication_data(started_cluster, "=== Put test CSV ===") instance = cluster.instances["dummy"] # type: ClickHouseInstance
put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') format CSV".format(started_cluster.mock_host, started_cluster.preserving_data_port, started_cluster.bucket, format) table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
filename = "test.csv"
put_query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') format CSV".format(
cluster.minio_host, cluster.minio_port, cluster.minio_bucket, filename, table_format)
csv_data = "8,9,16\n11,18,13\n22,14,2\n" csv_data = "8,9,16\n11,18,13\n22,14,2\n"
run_query(instance, put_query, stdin=csv_data) run_query(instance, put_query, stdin=csv_data)
data = get_communication_data(started_cluster)
received_data_completed = data["received_data_completed"] assert csv_data == get_s3_file_content(cluster, filename)
received_data = data["received_data"]
finalize_data = data["finalize_data"]
finalize_data_query = data["finalize_data_query"]
assert received_data[-1].decode() == csv_data
assert received_data_completed
assert finalize_data == "<CompleteMultipartUpload><Part><PartNumber>1</PartNumber><ETag>hello-etag</ETag></Part></CompleteMultipartUpload>"
assert finalize_data_query == "uploadId=TEST"
def test_put_with_redirect(started_cluster): # Test put and get with S3 server redirect.
instance = started_cluster.instances["dummy"] def test_put_get_with_redirect(cluster):
format = "column1 UInt32, column2 UInt32, column3 UInt32" # type: (ClickHouseCluster) -> None
put_communication_data(started_cluster, "=== Put with redirect test ===") instance = cluster.instances["dummy"] # type: ClickHouseInstance
other_values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)" table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(started_cluster.mock_host, started_cluster.redirecting_preserving_data_port, started_cluster.bucket, format, other_values) values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)"
values_csv = "1,1,1\n1,1,1\n11,11,11\n"
filename = "test.csv"
query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format(
cluster.minio_redirect_host, cluster.minio_redirect_port, cluster.minio_bucket, filename, table_format, values)
run_query(instance, query) run_query(instance, query)
query = "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format(started_cluster.mock_host, started_cluster.preserving_data_port, started_cluster.bucket, format) assert values_csv == get_s3_file_content(cluster, filename)
query = "select *, column1*column2*column3 from s3('http://{}:{}/{}/{}', 'CSV', '{}')".format(
cluster.minio_redirect_host, cluster.minio_redirect_port, cluster.minio_bucket, filename, table_format)
stdout = run_query(instance, query) stdout = run_query(instance, query)
assert list(map(str.split, stdout.splitlines())) == [ assert list(map(str.split, stdout.splitlines())) == [
["1", "1", "1", "1"], ["1", "1", "1", "1"],
["1", "1", "1", "1"], ["1", "1", "1", "1"],
["11", "11", "11", "1331"], ["11", "11", "11", "1331"],
] ]
data = get_communication_data(started_cluster)
received_data = data["received_data"]
assert received_data[-1].decode() == "1,1,1\n1,1,1\n11,11,11\n"
def test_multipart_put(started_cluster): # Test multipart put.
instance = started_cluster.instances["dummy"] def test_multipart_put(cluster):
format = "column1 UInt32, column2 UInt32, column3 UInt32" # type: (ClickHouseCluster) -> None
put_communication_data(started_cluster, "=== Multipart test ===") instance = cluster.instances["dummy"] # type: ClickHouseInstance
long_data = [[i, i+1, i+2] for i in range(100000)] table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
long_values = "".join([ "{},{},{}\n".format(x,y,z) for x, y, z in long_data ])
put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') format CSV".format(started_cluster.mock_host, started_cluster.multipart_preserving_data_port, started_cluster.bucket, format) # Minimum size of part is 5 Mb for Minio.
run_query(instance, put_query, stdin=long_values, settings={'s3_min_upload_part_size': 1000000}) # See: https://github.com/minio/minio/blob/master/docs/minio-limits.md
data = get_communication_data(started_cluster) min_part_size_bytes = 5 * 1024 * 1024
assert "multipart_received_data" in data csv_size_bytes = int(min_part_size_bytes * 1.5) # To have 2 parts.
received_data = data["multipart_received_data"]
assert received_data[-1].decode() == "".join([ "{},{},{}\n".format(x, y, z) for x, y, z in long_data ]) one_line_length = 6 # 3 digits, 2 commas, 1 line separator.
assert 1 < data["multipart_parts"] < 10000
# Generate data having size more than one part
int_data = [[1, 2, 3] for i in range(csv_size_bytes / one_line_length)]
csv_data = "".join(["{},{},{}\n".format(x, y, z) for x, y, z in int_data])
assert len(csv_data) > min_part_size_bytes
filename = "test_multipart.csv"
put_query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') format CSV".format(
cluster.minio_redirect_host, cluster.minio_redirect_port, cluster.minio_bucket, filename, table_format)
run_query(instance, put_query, stdin=csv_data, settings={'s3_min_upload_part_size': min_part_size_bytes})
# Use Nginx access logs to count number of parts uploaded to Minio.
nginx_logs = get_nginx_access_logs()
uploaded_parts = filter(lambda log_line: log_line.find(filename) >= 0 and log_line.find("PUT") >= 0, nginx_logs)
assert uploaded_parts > 1
assert csv_data == get_s3_file_content(cluster, filename)

View File

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

View File

@ -0,0 +1,12 @@
drop table if exists test_max_parallel_replicas_lr;
-- If you wonder why the table is named with "_lr" suffix in this test.
-- No reason. Actually it is the name of the table in Yandex.Market and they provided this test case for us.
CREATE TABLE test_max_parallel_replicas_lr (timestamp UInt64) ENGINE = MergeTree ORDER BY (intHash32(timestamp)) SAMPLE BY intHash32(timestamp);
INSERT INTO test_max_parallel_replicas_lr select number as timestamp from system.numbers limit 100;
SET max_parallel_replicas = 2;
select count() FROM remote('127.0.0.{2|3}', currentDatabase(), test_max_parallel_replicas_lr) PREWHERE timestamp > 0;
drop table test_max_parallel_replicas_lr;

View File

@ -0,0 +1,10 @@
count
1000000
count final
666667
count sample
557632
count sample final
371758
count final max_parallel_replicas
666667

View File

@ -0,0 +1,17 @@
drop table if exists sample_final;
create table sample_final (CounterID UInt32, EventDate Date, EventTime DateTime, UserID UInt64, Sign Int8) engine = CollapsingMergeTree(Sign) order by (CounterID, EventDate, intHash32(UserID), EventTime) sample by intHash32(UserID);
insert into sample_final select number / (8192 * 4), toDate('2019-01-01'), toDateTime('2019-01-01 00:00:01') + number, number / (8192 * 2), number % 3 = 1 ? -1 : 1 from numbers(1000000);
select 'count';
select count() from sample_final;
select 'count final';
select count() from sample_final final;
select 'count sample';
select count() from sample_final sample 1/2;
select 'count sample final';
select count() from sample_final final sample 1/2;
select 'count final max_parallel_replicas';
set max_parallel_replicas=2;
select count() from remote('127.0.0.{2|3}', currentDatabase(), sample_final) final;
drop table if exists sample_final;

View File

@ -0,0 +1,36 @@
DROP TABLE IF EXISTS test;
CREATE TABLE test (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE = MergeTree ORDER BY a;
SELECT b FROM test PREWHERE c = 1;
DROP TABLE test;
drop table if exists audience_local;
create table audience_local
(
Date Date,
AudienceType Enum8('other' = 0, 'client' = 1, 'group' = 2),
UMA UInt64,
APIKey String,
TrialNameID UInt32,
TrialGroupID UInt32,
AppVersion String,
Arch Enum8('other' = 0, 'x32' = 1, 'x64' = 2),
UserID UInt32,
GroupID UInt8,
OSName Enum8('other' = 0, 'Android' = 1, 'iOS' = 2, 'macOS' = 3, 'Windows' = 4, 'Linux' = 5),
Channel Enum8('other' = 0, 'Canary' = 1, 'Dev' = 2, 'Beta' = 3, 'Stable' = 4),
Hits UInt64,
Sum Int64,
Release String alias splitByChar('-', AppVersion)[1]
)
engine = SummingMergeTree
PARTITION BY (toISOYear(Date), toISOWeek(Date))
ORDER BY (AudienceType, UMA, APIKey, Date, TrialNameID, TrialGroupID, AppVersion, Arch, UserID, GroupID, OSName, Channel)
SETTINGS index_granularity = 8192;
SELECT DISTINCT UserID
FROM audience_local
PREWHERE Date = toDate('2019-07-25') AND Release = '17.11.0.542';
drop table if exists audience_local;

View File

@ -1,6 +0,0 @@
DROP TABLE IF EXISTS test.test;
CREATE TABLE test.test (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE = MergeTree ORDER BY a;
SELECT b FROM test.test PREWHERE c = 1;
DROP TABLE test;

View File

@ -0,0 +1,268 @@
Building of ClickHouse is supported on Linux, FreeBSD and Mac OS X.
# If you use Windows
If you use Windows, you need to create a virtual machine with Ubuntu. To start working with a virtual machine please install VirtualBox. You can download Ubuntu from the website: https://www.ubuntu.com/#download. Please create a virtual machine from the downloaded image (you should reserve at least 4GB of RAM for it). To run a command line terminal in Ubuntu, please locate a program containing the word "terminal" in its name (gnome-terminal, konsole etc.) or just press Ctrl+Alt+T.
# Creating a repository on GitHub
To start working with ClickHouse repository you will need a GitHub account.
You probably already have one, but if you don't, please register at https://github.com. In case you do not have SSH keys, you should generate them and then upload them on GitHub. It is required for sending over your patches. It is also possible to use the same SSH keys that you use with any other SSH servers - probably you already have those.
Create a fork of ClickHouse repository. To do that please click on the "fork" button in the upper right corner at https://github.com/ClickHouse/ClickHouse. It will fork your own copy of ClickHouse/ClickHouse to your account.
Development process consists of first committing the intended changes into your fork of ClickHouse and then creating a "pull request" for these changes to be accepted into the main repository (ClickHouse/ClickHouse).
To work with git repositories, please install `git`.
To do that in Ubuntu you would run in the command line terminal:
```
sudo apt update
sudo apt install git
```
A brief manual on using Git can be found here: https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf.
For a detailed manual on Git see: https://git-scm.com/book/ru/v2.
# Cloning a repository to your development machine
Next, you need to download the source files onto your working machine. This is called "to clone a repository" because it creates a local copy of the repository on your working machine.
In the command line terminal run:
```
git clone --recursive git@guthub.com:your_github_username/ClickHouse.git
cd ClickHouse
```
Note: please, substitute *your_github_username* with what is appropriate!
This command will create a directory `ClickHouse` containing the working copy of the project.
It is important that the path to the working directory contains no whitespaces as it may lead to problems with running the build system.
Please note that ClickHouse repository uses `submodules`. That is what the references to additional repositories are called (i.e. external libraries on which the project depends). It means that when cloning the repository you need to specify the `--recursive` flag as in the example above. If the repository has been cloned without submodules, to download them you need to run the following:
```
git submodule init
git submodule update
```
You can check status with command: `git submodule status`.
If you get the following error message:
```
Permission denied (publickey).
fatal: Could not read from remote repository.
Please make sure you have the correct access rights
and the repository exists.
```
It generally means that the SSH keys for connecting to GitHub are missing. These keys are normally located in `~/.ssh`. For SSH keys to be accepted you need to upload them in the settings section of GitHub UI.
You can also clone the repository via https protocol:
```
git clone https://github.com/ClickHouse/ClickHouse.git
```
This however will not let you send your changes to the server. You can still use it temporarily and add the SSH keys later replacing the remote address of the repository with `git remote` command.
You can also add original ClickHouse repo's address to your local repository to pull updates from there:
```
git remote add upstream git@github.com:ClickHouse/ClickHouse.git
```
After successfully running this command you will be able to pull updates from the main ClickHouse repo by running `git pull upstream master`.
# Build System
ClickHouse uses CMake and Ninja for building.
CMake - a meta-build system that can generate Ninja files (build tasks).
Ninja - a smaller build system with focus on speed used to execute those cmake generated tasks.
To install on Ubuntu, Debian or Mint run `sudo apt install cmake ninja-build`.
On CentOS, RedHat run `sudo yum install cmake ninja-build`.
If you use Arch or Gentoo, you probably know it yourself how to install CMake.
For installing CMake and Ninja on Mac OS X first install Homebrew and then install everything else via brew:
```
/usr/bin/ruby -e "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/master/install)"
brew install cmake ninja
```
Next, check the version of CMake: `cmake --version`. If it is below 3.3, you should install a newer version from the website: https://cmake.org/download/.
# Optional External Libraries
ClickHouse uses several external libraries for building. Most of them do not need to be installed separately as they are built together with ClickHouse from the sources located in the submodules. You can check the list in `contrib`.
There is a couple of libraries that are not built from sources but are supplied by the system: ICU and Readline, and thus are recommended to be installed.
Ubuntu: `sudo apt install libicu-dev libreadline-dev`
Mac OS X: `brew install icu4c readline`
However, these libraries are optional and ClickHouse can well be built without them. ICU is used for support of `COLLATE` in `ORDER BY` (i.e. for sorting in turkish alphabet). Readline is used for more convenient command input in clickhouse-client.
# C++ Compiler
Compilers GCC starting from version 9 and Clang version 8 or above are supported for building ClickHouse.
Official Yandex builds currently use GCC because it generates machine code of slightly better performance (yielding a difference of up to several percent according to our benchmarks). And Clang is more convenient for development usually. Though, our continuous integration (CI) platform runs checks for about a dozen of build combinations.
To install GCC on Ubuntu run: `sudo apt install gcc g++`
Check the version of gcc: `gcc --version`. If it is below 9, then follow the instruction here: https://clickhouse.yandex/docs/en/development/build/#install-gcc-9.
To install GCC on Mac OS X run: `brew install gcc`.
If you decide to use Clang, you can also install `libc++` and `lld`, if you know what it is. Using `ccache` is also recommended.
# The Building process
Now that you are ready to build ClickHouse we recommend you to create a separate directory `build` inside `ClickHouse` that will contain all of the build artefacts:
```
mkdir build
cd build
```
You can have several different directories (build_release, build_debug, etc.) for different types of build.
While inside the `build` directory, configure your build by running CMake. Before the first run you need to define environment variables that specify compiler (version 9 gcc compiler in this example).
```
export CC=gcc-9 CXX=g++-9
cmake ..
```
The `CC` variable specifies the compiler for C (short for C Compiler), and `CXX` variable instructs which C++ compiler is to be used for building.
For a faster build you can resort to the `debug` build type - a build with no optimizations. For that supply the following parameter `-D CMAKE_BUILD_TYPE=Debug`:
```
cmake -D CMAKE_BUILD_TYPE=Debug ..
```
You can change the type of build by running this command in the `build` directory.
Run ninja to build:
```
ninja clickhouse-server clickhouse-client
```
Only the required binaries are going to be built in this example.
If you require to build all the binaries (utilities and tests), you should run ninja with no parameters:
```
ninja
```
Full build requires about 30GB of free disk space or 15GB to build the main binaries.
When large amount of RAM is available on build machine you should limit the number of build tasks run in parallel with `-j` param:
```
ninja -j 1 clickhouse-server clickhouse-client
```
On machines with 4GB of RAM it is recommended to specify 1, for 8GB of RAM `-j 2` is recommended.
If you get the message: `ninja: error: loading 'build.ninja': No such file or directory`, it means that generating a build configuration has failed and you need to inspect the message above.
Upon successful start of the building process you'll see the build progress - the number of processed tasks and the total number of tasks.
While building messages about protobuf files in libhdfs2 library like `libprotobuf WARNING` may show up. They affect nothing and are safe to be ignored.
Upon successful build you get an executable file `ClickHouse/<build_dir>/dbms/programs/clickhouse`:
```
ls -l dbms/programs/clickhouse
```
# Running the built executable of ClickHouse
To run the server under the current user you need to navigate to `ClickHouse/dbms/programs/server/` (located outside of `build`) and run:
```
../../../build/dbms/programs/clickhouse server
```
In this case ClickHouse will use config files located in the current directory. You can run `clickhouse server` from any directory specifying the path to a config file as a command line parameter `--config-file`.
To connect to ClickHouse with clickhouse-client in another terminal navigate to `ClickHouse/build/dbms/programs/` and run `clickhouse client`.
If you get `Connection refused` message on Mac OS X or FreeBSD, try specifying host address 127.0.0.1:
```
clickhouse client --host 127.0.0.1
```
You can replace production version of ClickHouse binary installed in your system with your custom built ClickHouse binary. To do that install ClickHouse on your machine following the instructions from the official website. Next, run the following:
```
sudo service clickhouse-server stop
sudo cp ClickHouse/build/dbms/programs/clickhouse /usr/bin/
sudo service clickhouse-server start
```
Note that `clickhouse-client`, `clickhouse-server` and others are symlinks to the commonly shared `clickhouse` binary.
You can also run your custom built ClickHouse binary with the config file from the ClickHouse package installed on your system:
```
sudo service clickhouse-server stop
sudo -u clickhouse ClickHouse/build/dbms/programs/clickhouse server --config-file /etc/clickhouse-server/config.xml
```
# IDE (Integrated Development Environment)
If you do not know which IDE to use, we recommend that you use CLion. CLion is a commercial software, but it offers 30 day free trial period. It is also free of charge for students. CLion can be used both on Linux and on Mac OS X.
KDevelop and QTCreator are another great alternatives of an IDE for developing ClickHouse. KDevelop comes in as a very handy IDE although unstable. If KDevelop crashes after a while upon opening project, you should click "Stop All" button as soon as it has opened the list of project's files. After doing so KDevelop should be fine to work with.
As simple code editors you can use Sublime Text or Visual Studio Code, or Kate (all of which are available on Linux).
Just in case, it is worth mentioning that CLion creates `build` path on its own, it also on its own selects `debug` for build type, for configuration it uses a version of CMake that is defined in CLion and not the one installed by you, and finally CLion will use `make` to run build tasks instead of `ninja`. This is a normal behaviour, just keep that in mind to avoid confusion.
# Writing Code
The description of ClickHouse architecture can be found here: https://clickhouse.yandex/docs/en/development/architecture/
The Code Style Guide: https://clickhouse.yandex/docs/en/development/style/
Writing tests: https://clickhouse.yandex/docs/en/development/tests/
List of tasks: https://github.com/yandex/ClickHouse/blob/master/dbms/tests/instructions/easy_tasks_sorted_en.md
# Test Data
Developing ClickHouse often requires loading realistic datasets. It is particularly important for performance testing. We have a specially prepared set of anonymized data from Yandex.Metrica. It requires additionally some 3GB of free disk space. Note that this data is not required to accomplish most of development tasks.
```
sudo apt install wget xz-utils
wget https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_v1.tsv.xz
wget https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz
xz -v -d hits_v1.tsv.xz
xz -v -d visits_v1.tsv.xz
clickhouse-client
CREATE TABLE test.hits ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime);
CREATE TABLE test.visits ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), `Goals.ID` Array(UInt32), `Goals.Serial` Array(UInt32), `Goals.EventTime` Array(DateTime), `Goals.Price` Array(Int64), `Goals.OrderID` Array(String), `Goals.CurrencyID` Array(UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, `TraficSource.ID` Array(Int8), `TraficSource.SearchEngineID` Array(UInt16), `TraficSource.AdvEngineID` Array(UInt8), `TraficSource.PlaceID` Array(UInt16), `TraficSource.SocialSourceNetworkID` Array(UInt8), `TraficSource.Domain` Array(String), `TraficSource.SearchPhrase` Array(String), `TraficSource.SocialSourcePage` Array(String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `Market.Type` Array(UInt8), `Market.GoalID` Array(UInt32), `Market.OrderID` Array(String), `Market.OrderPrice` Array(Int64), `Market.PP` Array(UInt32), `Market.DirectPlaceID` Array(UInt32), `Market.DirectOrderID` Array(UInt32), `Market.DirectBannerID` Array(UInt32), `Market.GoodID` Array(String), `Market.GoodName` Array(String), `Market.GoodQuantity` Array(Int32), `Market.GoodPrice` Array(Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID);
clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.hits FORMAT TSV" < hits_v1.tsv
clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.visits FORMAT TSV" < visits_v1.tsv
```
# Creating Pull Request
Navigate to your fork repository in GitHub's UI. If you have been developing in a branch, you need to select that branch. There will be a "Pull request" button located on the screen. In essence this means "create a request for accepting my changes into the main repository".
A pull request can be created even if the work is not completed yet. In this case please put the word "WIP" (work in progress) at the beginning of the title, it can be changed later. This is useful for cooperative reviewing and discussion of changes as well as for running all of the available tests. It is important that you provide a brief description of your changes, it will later be used for generating realease changelogs.
Testing will commence as soon as Yandex employees label your PR with a tag "can be tested". The results of some first checks (e.g. code style) will come in within several minutes. Build check results will arrive within a half an hour. And the main set of tests will report itself within an hour.
The system will prepare ClickHouse binary builds for your pull request individually. To retrieve these builds click the "Details" link next to "ClickHouse build check" entry in the list of checks. There you will find direct links to the built .deb packages of ClickHouse which you can deploy even on your production servers (if you have no fear).
Most probably some of the builds will fail at first times. This is due to the fact that we check builds both with gcc as well as with clang, with almost all of existing warnings (always with the `-Werror` flag) enabled for clang. On that same page you can find all of the build logs so that you do not have to build ClickHouse in all of the possible ways.

View File

@ -999,7 +999,7 @@ Default value: 0.
- Type: bool - Type: bool
- Default value: True - Default value: True
Enable order-preserving parallel parsing of data formats. Supported only for TSV format. Enable order-preserving parallel parsing of data formats. Supported only for TSV, TKSV, CSV and JSONEachRow formats.
## min_chunk_bytes_for_parallel_parsing ## min_chunk_bytes_for_parallel_parsing

View File

@ -20,12 +20,183 @@ The result type is an integer with bits equal to the maximum bits of its argumen
## bitRotateRight(a, b) ## bitRotateRight(a, b)
## bitTest(a, b) ## bitTest {#bittest}
## bitTestAll(a, b) Takes any integer and converts it into [binary form](https://en.wikipedia.org/wiki/Binary_number), returns the value of a bit at specified position. The countdown starts from 0 from the right to the left.
## bitTestAny(a, b) **Syntax**
```sql
SELECT bitTest(number, index)
```
**Parameters**
- `number` integer number.
- `index` position of bit.
**Returned values**
Returns a value of bit at specified position.
Type: `UInt8`.
**Example**
For example, the number 43 in base-2 (binary) numeral system is 101011.
Query:
```sql
SELECT bitTest(43, 1)
```
Result:
```text
┌─bitTest(43, 1)─┐
│ 1 │
└────────────────┘
```
Another example:
Query:
```sql
SELECT bitTest(43, 2)
```
Result:
```text
┌─bitTest(43, 2)─┐
│ 0 │
└────────────────┘
```
## bitTestAll {#bittestall}
Returns result of [logical conjuction](https://en.wikipedia.org/wiki/Logical_conjunction) (AND operator) of all bits at given positions. The countdown starts from 0 from the right to the left.
The conjuction for bitwise operations:
0 AND 0 = 0
0 AND 1 = 0
1 AND 0 = 0
1 AND 1 = 1
**Syntax**
```sql
SELECT bitTestAll(number, index1, index2, index3, index4, ...)
```
**Parameters**
- `number` integer number.
- `index1`, `index2`, `index3`, `index4` positions of bit. For example, for set of positions (`index1`, `index2`, `index3`, `index4`) is true if and only if all of its positions are true (`index1` ⋀ `index2`, ⋀ `index3``index4`).
**Returned values**
Returns result of logical conjuction.
Type: `UInt8`.
**Example**
For example, the number 43 in base-2 (binary) numeral system is 101011.
Query:
```sql
SELECT bitTestAll(43, 0, 1, 3, 5)
```
Result:
```text
┌─bitTestAll(43, 0, 1, 3, 5)─┐
│ 1 │
└────────────────────────────┘
```
Another example:
Query:
```sql
SELECT bitTestAll(43, 0, 1, 3, 5, 2)
```
Result:
```text
┌─bitTestAll(43, 0, 1, 3, 5, 2)─┐
│ 0 │
└───────────────────────────────┘
```
## bitTestAny {#bittestany}
Returns result of [logical disjunction](https://en.wikipedia.org/wiki/Logical_disjunction) (OR operator) of all bits at given positions. The countdown starts from 0 from the right to the left.
The disjunction for bitwise operations:
0 OR 0 = 0
0 OR 1 = 1
1 OR 0 = 1
1 OR 1 = 1
**Syntax**
```sql
SELECT bitTestAny(number, index1, index2, index3, index4, ...)
```
**Parameters**
- `number` integer number.
- `index1`, `index2`, `index3`, `index4` positions of bit.
**Returned values**
Returns result of logical disjuction.
Type: `UInt8`.
**Example**
For example, the number 43 in base-2 (binary) numeral system is 101011.
Query:
```sql
SELECT bitTestAny(43, 0, 2)
```
Result:
```text
┌─bitTestAny(43, 0, 2)─┐
│ 1 │
└──────────────────────┘
```
Another example:
Query:
```sql
SELECT bitTestAny(43, 4, 2)
```
Result:
```text
┌─bitTestAny(43, 4, 2)─┐
│ 0 │
└──────────────────────┘
```
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/bit_functions/) <!--hide--> [Original article](https://clickhouse.yandex/docs/en/query_language/functions/bit_functions/) <!--hide-->

View File

@ -40,8 +40,36 @@ SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8)
## toInt(8|16|32|64)OrZero ## toInt(8|16|32|64)OrZero
It takes an argument of type String and tries to parse it into Int (8 | 16 | 32 | 64). If failed, returns 0.
**Example**
```sql
select toInt64OrZero('123123'), toInt8OrZero('123qwe123')
```
```text
┌─toInt64OrZero('123123')─┬─toInt8OrZero('123qwe123')─┐
│ 123123 │ 0 │
└─────────────────────────┴───────────────────────────┘
```
## toInt(8|16|32|64)OrNull ## toInt(8|16|32|64)OrNull
It takes an argument of type String and tries to parse it into Int (8 | 16 | 32 | 64). If failed, returns NULL.
**Example**
```sql
select toInt64OrNull('123123'), toInt8OrNull('123qwe123')
```
```text
┌─toInt64OrNull('123123')─┬─toInt8OrNull('123qwe123')─┐
│ 123123 │ ᴺᵁᴸᴸ │
└─────────────────────────┴───────────────────────────┘
```
## toUInt(8|16|32|64) ## toUInt(8|16|32|64)
Converts an input value to the [UInt](../../data_types/int_uint.md) data type. This function family includes: Converts an input value to the [UInt](../../data_types/int_uint.md) data type. This function family includes:

View File

@ -95,7 +95,7 @@ Provides possibility to stop background merges for tables in the MergeTree famil
```sql ```sql
SYSTEM STOP MERGES [[db.]merge_tree_family_table_name] SYSTEM STOP MERGES [[db.]merge_tree_family_table_name]
``` ```
!!! note "Note": !!! note "Note"
`DETACH / ATTACH` table will start background merges for the table even in case when merges have been stopped for all MergeTree tables before. `DETACH / ATTACH` table will start background merges for the table even in case when merges have been stopped for all MergeTree tables before.

View File

@ -0,0 +1 @@
../../en/development/developer_instruction.md

View File

@ -0,0 +1,268 @@
Сборка ClickHouse поддерживается на Linux, FreeBSD, Mac OS X.
# Если вы используете Windows
Если вы используете Windows, вам потребуется создать виртуальную машину с Ubuntu. Для работы с виртуальной машиной, установите VirtualBox. Скачать Ubuntu можно на сайте: https://www.ubuntu.com/#download Создайте виртуальную машину из полученного образа. Выделите для неё не менее 4 GB оперативной памяти. Для запуска терминала в Ubuntu, найдите в меню программу со словом terminal (gnome-terminal, konsole или что-то в этом роде) или нажмите Ctrl+Alt+T.
# Создание репозитория на GitHub
Для работы с репозиторием ClickHouse, вам потребуется аккаунт на GitHub. Наверное, он у вас уже есть.
Если аккаунта нет - зарегистрируйтесь на https://github.com/. Создайте ssh ключи, если их нет, и загрузите публичные ключи на GitHub. Это потребуется для отправки изменений. Для работы с GitHub можно использовать такие же ssh ключи, как и для работы с другими ssh серверами - скорее всего, они уже у вас есть.
Создайте fork репозитория ClickHouse. Для этого, на странице https://github.com/ClickHouse/ClickHouse нажмите на кнопку "fork" в правом верхнем углу. Вы получите полную копию репозитория ClickHouse на своём аккаунте, которая называется "форк". Процесс разработки состоит в том, чтобы внести нужные изменения в свой форк репозитория, а затем создать "pull request" для принятия изменений в основной репозиторий.
Для работы с git репозиториями, установите `git`.
В Ubuntu выполните в терминале:
```
sudo apt update
sudo apt install git
```
Краткое руководство по использованию Git: https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf
Подробное руководство по использованию Git: https://git-scm.com/book/ru/v2
# Клонирование репозитория на рабочую машину
Затем вам потребуется загрузить исходники для работы на свой компьютер. Это называется "клонирование репозитория", потому что создаёт на вашем компьютере локальную копию репозитория, с которой вы будете работать.
Выполните в терминале:
```
git clone --recursive git@github.com:yandex/ClickHouse.git
cd ClickHouse
```
Замените *yandex* на имя вашего аккаунта на GitHub.
Эта команда создаст директорию ClickHouse, содержащую рабочую копию проекта.
Необходимо, чтобы путь к рабочей копии не содержал пробелы в именах директорий. Это может привести к проблемам в работе системы сборки.
Обратите внимание, что репозиторий ClickHouse использует submodules. Так называются ссылки на дополнительные репозитории (например, внешние библиотеки, от которых зависит проект). Это значит, что при клонировании репозитория, следует указывать ключ `--recursive`, как в примере выше. Если репозиторий был клонирован без submodules, то для их скачивания, необходимо выполнить:
```
git submodule init
git submodule update
```
Проверить наличие submodules можно с помощью команды `git submodule status`.
Если вы получили сообщение об ошибке:
```
Permission denied (publickey).
fatal: Could not read from remote repository.
Please make sure you have the correct access rights
and the repository exists.
```
Как правило это означает, что отсутствуют ssh ключи для соединения с GitHub. Ключи расположены в директории `~/.ssh`. В интерфейсе GitHub, в настройках, необходимо загрузить публичные ключи, чтобы он их понимал.
Вы также можете клонировать репозиторий по протоколу https:
```
git clone https://github.com/ClickHouse/ClickHouse.git
```
Этот вариант не подходит для отправки изменений на сервер. Вы можете временно его использовать, а затем добавить ssh ключи и заменить адрес репозитория с помощью команды `git remote`.
Вы можете также добавить для своего локального репозитория адрес оригинального репозитория Яндекса, чтобы притягивать оттуда обновления:
```
git remote add upstream git@github.com:yandex/ClickHouse.git
```
После этого, вы сможете добавлять в свой репозиторий обновления из репозитория Яндекса с помощью команды `git pull upstream master`.
# Система сборки
ClickHouse использует систему сборки CMake и Ninja.
CMake - генератор задач сборки.
Ninja - система запуска сборочных задач.
Для установки на Ubuntu или Debian, Mint, выполните `sudo apt install cmake ninja-build`.
Для установки на CentOS, RedHat, выполните `sudo yum install cmake ninja-build`.
Если у вас Arch или Gentoo, то вы сами знаете, как установить CMake.
Для установки CMake и Ninja на Mac OS X, сначала установите Homebrew, а затем, с помощью него, установите всё остальное.
```
/usr/bin/ruby -e "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/master/install)"
brew install cmake ninja
```
Проверьте версию CMake: `cmake --version`. Если версия меньше 3.3, то установите новую версию с сайта https://cmake.org/download/
# Необязательные внешние библиотеки
ClickHouse использует для сборки некоторое количество внешних библиотек. Большинство из них не требуется отдельно устанавливать, так как они собираются вместе с ClickHouse, из исходников, которые расположены в submodules. Посмотреть набор этих библиотек можно в директории contrib.
Пара библиотек не собирается из исходников, а используется из системы: ICU и Readline, и их рекомендуется установить.
Ubuntu: `sudo apt install libicu-dev libreadline-dev`
Mac OS X: `brew install icu4c readline`
Впрочем, эти библиотеки не обязательны для работы и ClickHouse может быть собран без них. ICU используется для поддержки `COLLATE` в `ORDER BY` (например, для сортировки с учётом турецкого алфавита). Readline используется для более удобного набора команд в интерактивном режиме в clickhouse-client.
# Компилятор C++
В качестве компилятора C++ поддерживается GCC начиная с версии 9 или Clang начиная с версии 8.
Официальные сборки от Яндекса, на данный момент, используют GCC, так как он генерирует слегка более производительный машинный код (разница в среднем до нескольких процентов по нашим бенчмаркам). Clang обычно более удобен для разработки. Впрочем, наша среда continuous integration проверяет около десятка вариантов сборки.
Для установки GCC под Ubuntu, выполните: `sudo apt install gcc g++`.
Проверьте версию gcc: `gcc --version`. Если версия меньше 9, то следуйте инструкции: https://clickhouse.yandex/docs/en/development/build/#install-gcc-9
Для установки GCC под Mac OS X, выполните `brew install gcc`.
Если вы решили использовать Clang, вы также можете установить `libc++` и `lld`, если вы знаете, что это такое. При желании, установите `ccache`.
# Процесс сборки
Теперь вы готовы к сборке ClickHouse. Для размещения собранных файлов, рекомендуется создать отдельную директорию build внутри директории ClickHouse:
```
mkdir build
cd build
```
Вы можете иметь несколько разных директорий (build_release, build_debug) для разных вариантов сборки.
Находясь в директории build, выполните конфигурацию сборки с помощью CMake.
Перед первым запуском необходимо выставить переменные окружения, отвечающие за выбор компилятора (в данном примере это - gcc версии 9).
```
export CC=gcc-9 CXX=g++-9
cmake ..
```
Переменная CC отвечает за компилятор C (сокращение от слов C Compiler), переменная CXX отвечает за выбор компилятора C++ (символ X - это как плюс, но положенный набок, ради того, чтобы превратить его в букву).
Для более быстрой сборки, можно использовать debug вариант - сборку без оптимизаций. Для этого, укажите параметр `-D CMAKE_BUILD_TYPE=Debug`:
```
cmake -D CMAKE_BUILD_TYPE=Debug ..
```
Вы можете изменить вариант сборки, выполнив эту команду в директории build.
Запустите ninja для сборки:
```
ninja clickhouse-server clickhouse-client
```
В этом примере собираются только нужные в первую очередь программы.
Если вы хотите собрать все программы (утилиты и тесты), то запустите ninja без параметров:
```
ninja
```
Для полной сборки требуется около 30 GB свободного места на диске или 15 GB для сборки только основных программ.
При наличии небольшого количества оперативной памяти на компьютере, следует ограничить количество параллельных задач с помощью параметра `-j`:
```
ninja -j 1 clickhouse-server clickhouse-client
```
На машинах с 4 GB памяти, рекомендуется указывать значение 1, а если памяти до 8 GB, укажите значение 2.
Если вы получили сообщение `ninja: error: loading 'build.ninja': No such file or directory`, значит конфигурация сборки прошла с ошибкой и вам необходимо посмотреть на сообщение об ошибке выше.
В случае успешного запуска, вы увидите прогресс сборки - количество обработанных задач и общее количество задач.
В процессе сборки могут появится сообщения `libprotobuf WARNING` про protobuf файлы в библиотеке libhdfs2. Это не имеет значения.
При успешной сборке, вы получите готовый исполняемый файл `ClickHouse/build/dbms/programs/clickhouse`:
```
ls -l dbms/programs/clickhouse
```
# Запуск собранной версии ClickHouse
Для запуска сервера из под текущего пользователя, с выводом логов в терминал и с использованием примеров конфигурационных файлов, расположенных в исходниках, перейдите в директорию `ClickHouse/dbms/programs/server/` (эта директория находится не в директории build) и выполните:
```
../../../build/dbms/programs/clickhouse server
```
В этом случае, ClickHouse будет использовать конфигурационные файлы, расположенные в текущей директории. Вы можете запустить `clickhouse server` из любой директории, передав ему путь к конфигурационному файлу в аргументе командной строки `--config-file`.
Для подключения к ClickHouse с помощью clickhouse-client, в соседнем терминале, зайдите в директорию `ClickHouse/build/dbms/programs/` и выполните `clickhouse client`.
Если вы получили сообщение `Connection refused` на Mac OS X или FreeBSD, то укажите для клиента 127.0.0.1 в качестве имени хоста:
```
clickhouse client --host 127.0.0.1
```
Вы можете заменить собранным вами ClickHouse продакшен версию, установленную в системе. Для этого, установите ClickHouse на свою машину по инструкции с официального сайта. Затем выполните:
```
sudo service clickhouse-server stop
sudo cp ClickHouse/build/dbms/programs/clickhouse /usr/bin/
sudo service clickhouse-server start
```
Обратите внимание, что `clickhouse-client`, `clickhouse-server` и другие, являеются симлинками на общий бинарник `clickhouse`.
Также вы можете запустить собранный вами ClickHouse с конфигурационным файлом системного ClickHouse:
```
sudo service clickhouse-server stop
sudo -u clickhouse ClickHouse/build/dbms/programs/clickhouse server --config-file /etc/clickhouse-server/config.xml
```
# Среда разработки
Если вы не знаете, какую среду разработки использовать, то рекомендуется использовать CLion. CLion является платным ПО, но его можно использовать бесплатно в течение пробного периода. Также он бесплатен для учащихся. CLion можно использовать как под Linux, так и под Mac OS X.
Также в качестве среды разработки, вы можете использовать KDevelop или QTCreator. KDevelop - очень удобная, но нестабильная среда разработки. Если KDevelop вылетает через небольшое время после открытия проекта, вам следует нажать на кнопку "Stop All" как только он открыл список файлов проекта. После этого, KDevelop можно будет использовать.
В качестве простых редакторов кода можно использовать Sublime Text или Visual Studio Code или Kate (все варианты доступны под Linux).
На всякий случай заметим, что CLion самостоятельно создаёт свою build директорию, самостоятельно выбирает тип сборки debug по-умолчанию, для конфигурации использует встроенную в CLion версию CMake вместо установленного вами, а для запуска задач использует make вместо ninja. Это нормально, просто имейте это ввиду, чтобы не возникало путаницы.
# Написание кода
Описание архитектуры ClickHouse: https://clickhouse.yandex/docs/ru/development/architecture/
Стиль кода: https://clickhouse.yandex/docs/ru/development/style/
Разработка тестов: https://clickhouse.yandex/docs/ru/development/tests/
Список задач: https://github.com/ClickHouse/ClickHouse/blob/master/dbms/tests/instructions/easy_tasks_sorted_ru.md
# Тестовые данные
Разработка ClickHouse часто требует загрузки реалистичных наборов данных. Особенно это важно для тестирования производительности. Специально для вас мы подготовили набор данных, представляющий собой анонимизированные данные Яндекс.Метрики. Загрузка этих данных потребует ещё 3 GB места на диске. Для выполнения большинства задач разработки, загружать эти данные не обязательно.
```
sudo apt install wget xz-utils
wget https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_v1.tsv.xz
wget https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz
xz -v -d hits_v1.tsv.xz
xz -v -d visits_v1.tsv.xz
clickhouse-client
CREATE TABLE test.hits ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime);
CREATE TABLE test.visits ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), `Goals.ID` Array(UInt32), `Goals.Serial` Array(UInt32), `Goals.EventTime` Array(DateTime), `Goals.Price` Array(Int64), `Goals.OrderID` Array(String), `Goals.CurrencyID` Array(UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, `TraficSource.ID` Array(Int8), `TraficSource.SearchEngineID` Array(UInt16), `TraficSource.AdvEngineID` Array(UInt8), `TraficSource.PlaceID` Array(UInt16), `TraficSource.SocialSourceNetworkID` Array(UInt8), `TraficSource.Domain` Array(String), `TraficSource.SearchPhrase` Array(String), `TraficSource.SocialSourcePage` Array(String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `Market.Type` Array(UInt8), `Market.GoalID` Array(UInt32), `Market.OrderID` Array(String), `Market.OrderPrice` Array(Int64), `Market.PP` Array(UInt32), `Market.DirectPlaceID` Array(UInt32), `Market.DirectOrderID` Array(UInt32), `Market.DirectBannerID` Array(UInt32), `Market.GoodID` Array(String), `Market.GoodName` Array(String), `Market.GoodQuantity` Array(Int32), `Market.GoodPrice` Array(Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID);
clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.hits FORMAT TSV" < hits_v1.tsv
clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.visits FORMAT TSV" < visits_v1.tsv
```
# Создание pull request
Откройте свой форк репозитория в интерфейсе GitHub. Если вы вели разработку в бранче, выберите этот бранч. На странице будет доступна кнопка "Pull request". По сути, это означает "создать заявку на принятие моих изменений в основной репозиторий".
Pull request можно создать, даже если работа над задачей ещё не завершена. В этом случае, добавьте в его название слово "WIP" (work in progress). Название можно будет изменить позже. Это полезно для совместного просмотра и обсуждения изменений, а также для запуска всех имеющихся тестов. Введите краткое описание изменений - впоследствии, оно будет использовано для релизных changelog.
Тесты будут запущены, как только сотрудники Яндекса поставят для pull request тег "Can be tested". Результаты первых проверок (стиль кода) появятся уже через несколько минут. Результаты сборки появятся примерно через пол часа. Результаты основного набора тестов будут доступны в пределах часа.
Система подготовит сборки ClickHouse специально для вашего pull request. Для их получения, нажмите на ссылку "Details" у проверки "Clickhouse build check". Там вы сможете найти прямые ссылки на собранные .deb пакеты ClickHouse, которые, при желании, вы даже сможете установить на свои продакшен серверы (если не страшно).
Вероятнее всего, часть сборок не будет успешной с первого раза. Ведь мы проверяем сборку кода и gcc и clang, а при сборке с помощью clang включаются почти все существующие в природе warnings (всегда с флагом `-Werror`). На той же странице, вы сможете найти логи сборки - вам не обязательно самому собирать ClickHouse всеми возможными способами.

View File

@ -571,6 +571,8 @@ Fuzzing тестирование - это тестирование случай
### 8.14. Запись данных в ORC. ### 8.14. Запись данных в ORC.
Возможно, Андрей Коняев, ArenaData (зависит от желания).
### 8.15. Запись данных в CapNProto. ### 8.15. Запись данных в CapNProto.
### 8.16. Поддержка формата Avro. ### 8.16. Поддержка формата Avro.
@ -599,10 +601,16 @@ Fuzzing тестирование - это тестирование случай
### 8.20. Интеграция с SQS. ### 8.20. Интеграция с SQS.
Низкий приоритет.
### 8.21. Поддержка произвольного количества языков для имён регионов. ### 8.21. Поддержка произвольного количества языков для имён регионов.
Нужно для БК. Декабрь 2019. Нужно для БК. Декабрь 2019.
### 8.22. Поддержка синтаксиса для переменных в стиле MySQL.
При парсинге запроса преобразовывать синтаксис вида `@@version_full` в вызов функции `getGlobalVariable('version_full')`. Поддержать популярные MySQL переменные. Может быть поможет Юрий Баранов, если будет энтузиазм.
## 9. Безопасность. ## 9. Безопасность.

View File

@ -16,4 +16,183 @@
## bitShiftRight(a, b) ## bitShiftRight(a, b)
## bitTest {#bittest}
Принимает любое целое число и конвертирует его в [двоичное число](https://en.wikipedia.org/wiki/Binary_number), возвращает значение бита в указанной позиции. Отсчет начинается с 0 справа налево.
**Синтаксис**
```sql
SELECT bitTest(number, index)
```
**Параметры**
- `number` целое число.
- `index` position of bit.
**Возвращаемое значение**
Возвращает значение бита в указанной позиции.
Тип: `UInt8`.
**Пример**
Например, число 43 в двоичной системе счисления равно: 101011.
Запрос:
```sql
SELECT bitTest(43, 1)
```
Ответ:
```text
┌─bitTest(43, 1)─┐
│ 1 │
└────────────────┘
```
Другой пример:
Запрос:
```sql
SELECT bitTest(43, 2)
```
Ответ:
```text
┌─bitTest(43, 2)─┐
│ 0 │
└────────────────┘
```
## bitTestAll {#bittestall}
Возвращает результат [логической конъюнкции](https://en.wikipedia.org/wiki/Logical_conjunction) (оператор AND) всех битов в указанных позициях. Отсчет начинается с 0 справа налево.
Бинарная конъюнкция:
0 AND 0 = 0
0 AND 1 = 0
1 AND 0 = 0
1 AND 1 = 1
**Синтаксис**
```sql
SELECT bitTestAll(number, index1, index2, index3, index4, ...)
```
**Параметры**
- `number` целое число.
- `index1`, `index2`, `index3`, `index4` позиция бита. Например, конъюнкция для набора позиций `index1`, `index2`, `index3`, `index4` является истинной, если все его позиции истинны `index1``index2``index3``index4`.
**Возвращаемое значение**
Возвращает результат логической конъюнкции.
Тип: `UInt8`.
**Пример**
Например, число 43 в двоичной системе счисления равно: 101011.
Запрос:
```sql
SELECT bitTestAll(43, 0, 1, 3, 5)
```
Ответ:
```text
┌─bitTestAll(43, 0, 1, 3, 5)─┐
│ 1 │
└────────────────────────────┘
```
Другой пример:
Запрос:
```sql
SELECT bitTestAll(43, 0, 1, 3, 5, 2)
```
Ответ:
```text
┌─bitTestAll(43, 0, 1, 3, 5, 2)─┐
│ 0 │
└───────────────────────────────┘
```
## bitTestAny {#bittestany}
Возвращает результат [логической дизъюнкции](https://en.wikipedia.org/wiki/Logical_disjunction) (оператор OR) всех битов в указанных позициях. Отсчет начинается с 0 справа налево.
Бинарная дизъюнкция:
0 OR 0 = 0
0 OR 1 = 1
1 OR 0 = 1
1 OR 1 = 1
**Синтаксис**
```sql
SELECT bitTestAny(number, index1, index2, index3, index4, ...)
```
**Параметры**
- `number` целое число.
- `index1`, `index2`, `index3`, `index4` позиции бита.
**Возвращаемое значение**
Возвращает результат логической дизъюнкции.
Тип: `UInt8`.
**Пример**
Например, число 43 в двоичной системе счисления равно: 101011.
Запрос:
```sql
SELECT bitTestAny(43, 0, 2)
```
Ответ:
```text
┌─bitTestAny(43, 0, 2)─┐
│ 1 │
└──────────────────────┘
```
Другой пример:
Запрос:
```sql
SELECT bitTestAny(43, 4, 2)
```
Ответ:
```text
┌─bitTestAny(43, 4, 2)─┐
│ 0 │
└──────────────────────┘
```
[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/bit_functions/) <!--hide--> [Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/bit_functions/) <!--hide-->

View File

@ -40,8 +40,36 @@ SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8)
## toInt(8|16|32|64)OrZero ## toInt(8|16|32|64)OrZero
Принимает аргумент типа String и пытается его распарсить в Int(8|16|32|64). Если не удалось - возвращает 0.
**Пример**
```sql
select toInt64OrZero('123123'), toInt8OrZero('123qwe123')
```
```text
┌─toInt64OrZero('123123')─┬─toInt8OrZero('123qwe123')─┐
│ 123123 │ 0 │
└─────────────────────────┴───────────────────────────┘
```
## toInt(8|16|32|64)OrNull ## toInt(8|16|32|64)OrNull
Принимает аргумент типа String и пытается его распарсить в Int(8|16|32|64). Если не удалось - возвращает NULL.
**Пример**
```sql
select toInt64OrNull('123123'), toInt8OrNull('123qwe123')
```
```text
┌─toInt64OrNull('123123')─┬─toInt8OrNull('123qwe123')─┐
│ 123123 │ ᴺᵁᴸᴸ │
└─────────────────────────┴───────────────────────────┘
```
## toUInt(8|16|32|64) ## toUInt(8|16|32|64)
Преобраует входное значение к типу [UInt](../../data_types/int_uint.md). Семейство функций включает: Преобраует входное значение к типу [UInt](../../data_types/int_uint.md). Семейство функций включает:

View File

@ -91,7 +91,7 @@ SYSTEM START DISTRIBUTED SENDS [db.]<distributed_table_name>
```sql ```sql
SYSTEM STOP MERGES [[db.]merge_tree_family_table_name] SYSTEM STOP MERGES [[db.]merge_tree_family_table_name]
``` ```
!!! note "Note": !!! note "Note"
`DETACH / ATTACH` таблицы восстанавливает фоновые мержи для этой таблицы (даже в случае отключения фоновых мержей для всех таблиц семейства MergeTree до `DETACH`). `DETACH / ATTACH` таблицы восстанавливает фоновые мержи для этой таблицы (даже в случае отключения фоновых мержей для всех таблиц семейства MergeTree до `DETACH`).

View File

@ -218,6 +218,7 @@ nav:
- 'How to Build ClickHouse on Linux for Mac OS X': 'development/build_cross.md' - 'How to Build ClickHouse on Linux for Mac OS X': 'development/build_cross.md'
- 'How to Write C++ code': 'development/style.md' - 'How to Write C++ code': 'development/style.md'
- 'How to Run ClickHouse Tests': 'development/tests.md' - 'How to Run ClickHouse Tests': 'development/tests.md'
- 'The Beginner ClickHouse Developer Instruction': 'development/developer_instruction.md'
- 'Third-Party Libraries Used': 'development/contrib.md' - 'Third-Party Libraries Used': 'development/contrib.md'
- 'What''s New': - 'What''s New':

View File

@ -216,6 +216,7 @@ nav:
- 'How to Build ClickHouse on Mac OS X': 'development/build_osx.md' - 'How to Build ClickHouse on Mac OS X': 'development/build_osx.md'
- 'How to Write C++ code': 'development/style.md' - 'How to Write C++ code': 'development/style.md'
- 'How to Run ClickHouse Tests': 'development/tests.md' - 'How to Run ClickHouse Tests': 'development/tests.md'
- 'The Beginner ClickHouse Developer Instruction': 'development/developer_instruction.md'
- 'Third-Party Libraries Used': 'development/contrib.md' - 'Third-Party Libraries Used': 'development/contrib.md'
- 'What''s New': - 'What''s New':

View File

@ -217,6 +217,7 @@ nav:
- 'Как собрать ClickHouse на Linux для Mac OS X': 'development/build_cross.md' - 'Как собрать ClickHouse на Linux для Mac OS X': 'development/build_cross.md'
- 'Как писать код на C++': 'development/style.md' - 'Как писать код на C++': 'development/style.md'
- 'Как запустить тесты': 'development/tests.md' - 'Как запустить тесты': 'development/tests.md'
- 'Инструкция для начинающего разработчика ClickHouse': 'development/developer_instruction.md'
- 'Сторонние библиотеки': 'development/contrib.md' - 'Сторонние библиотеки': 'development/contrib.md'
- 'Что нового': - 'Что нового':

View File

@ -212,9 +212,10 @@ nav:
- 'ClickHouse架构概述': 'development/architecture.md' - 'ClickHouse架构概述': 'development/architecture.md'
- '如何在Linux中编译ClickHouse': 'development/build.md' - '如何在Linux中编译ClickHouse': 'development/build.md'
- '如何在Mac OS X中编译ClickHouse': 'development/build_osx.md' - '如何在Mac OS X中编译ClickHouse': 'development/build_osx.md'
- 'How to Build ClickHouse on Linux for Mac OS X': 'development/build_cross.md' - '如何在Linux中编译Mac OS X ClickHouse': 'development/build_cross.md'
- '如何编写C++代码': 'development/style.md' - '如何编写C++代码': 'development/style.md'
- '如何运行ClickHouse测试': 'development/tests.md' - '如何运行ClickHouse测试': 'development/tests.md'
- '开发者指南': 'development/developer_instruction.md'
- '使用的第三方库': 'development/contrib.md' - '使用的第三方库': 'development/contrib.md'
- '新功能特性': - '新功能特性':

View File

@ -0,0 +1 @@
../../en/development/developer_instruction.md

View File

@ -542,7 +542,7 @@ LIMIT 20
<div class="spoiler"><a class="spoiler_title">Set ZooKeeper locations in configuration file</a> <div class="spoiler"><a class="spoiler_title">Set ZooKeeper locations in configuration file</a>
<div class="spoiler_body"> <div class="spoiler_body">
<pre> <pre>
&lt;zookeeper-servers&gt; &lt;zookeeper&gt;
&lt;node&gt; &lt;node&gt;
&lt;host&gt;zoo01.yandex.ru&lt;/host&gt; &lt;host&gt;zoo01.yandex.ru&lt;/host&gt;
&lt;port&gt;2181&lt;/port&gt; &lt;port&gt;2181&lt;/port&gt;
@ -555,7 +555,7 @@ LIMIT 20
&lt;host&gt;zoo03.yandex.ru&lt;/host&gt; &lt;host&gt;zoo03.yandex.ru&lt;/host&gt;
&lt;port&gt;2181&lt;/port&gt; &lt;port&gt;2181&lt;/port&gt;
&lt;/node&gt; &lt;/node&gt;
&lt;/zookeeper-servers&gt; &lt;/zookeeper&gt;
</pre> </pre>
</div> </div>
</div> </div>