diff --git a/CHANGELOG.md b/CHANGELOG.md index cab51478199..258936d6b52 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -677,7 +677,7 @@ fix comments to make obvious that it may throw. ### Backward Incompatible Change * Removed rarely used table function `catBoostPool` and storage `CatBoostPool`. If you have used this table function, please write email to `clickhouse-feedback@yandex-team.com`. Note that CatBoost integration remains and will be supported. [#6279](https://github.com/ClickHouse/ClickHouse/pull/6279) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Disable `ANY RIGHT JOIN` and `ANY FULL JOIN` by default. Set `any_join_get_any_from_right_table` setting to enable them. [#5126](https://github.com/ClickHouse/ClickHouse/issues/5126) [#6351](https://github.com/ClickHouse/ClickHouse/pull/6351) ([Artem Zuikov](https://github.com/4ertus2)) +* Disable `ANY RIGHT JOIN` and `ANY FULL JOIN` by default. Set `any_join_distinct_right_table_keys` setting to enable them. [#5126](https://github.com/ClickHouse/ClickHouse/issues/5126) [#6351](https://github.com/ClickHouse/ClickHouse/pull/6351) ([Artem Zuikov](https://github.com/4ertus2)) ## ClickHouse release 19.13.6.51, 2019-10-02 diff --git a/contrib/libhdfs3-cmake/CMakeLists.txt b/contrib/libhdfs3-cmake/CMakeLists.txt index 28f32e948b2..e68f0bacf46 100644 --- a/contrib/libhdfs3-cmake/CMakeLists.txt +++ b/contrib/libhdfs3-cmake/CMakeLists.txt @@ -182,6 +182,9 @@ set(SRCS ${HDFS3_SOURCE_DIR}/common/FileWrapper.h ) +# old kernels (< 3.17) doens't have SYS_getrandom. Always use POSIX implementation to have better compatibility +set_source_files_properties(${HDFS3_SOURCE_DIR}/rpc/RpcClient.cpp PROPERTIES COMPILE_FLAGS "-DBOOST_UUID_RANDOM_PROVIDER_FORCE_POSIX=1") + # target add_library(hdfs3 ${SRCS} ${PROTO_SOURCES} ${PROTO_HEADERS}) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 32395ff9101..f6d50b22db8 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -97,8 +97,7 @@ #define BRACK_PASTE_LAST '~' #define BRACK_PASTE_SLEN 6 -/// Make sure we don't get ^J for the enter character. -/// This handler also bypasses some unused macro/event checkings. +/// This handler bypasses some unused macro/event checkings. static int clickhouse_rl_bracketed_paste_begin(int /* count */, int /* key */) { std::string buf; @@ -106,10 +105,10 @@ static int clickhouse_rl_bracketed_paste_begin(int /* count */, int /* key */) RL_SETSTATE(RL_STATE_MOREINPUT); SCOPE_EXIT(RL_UNSETSTATE(RL_STATE_MOREINPUT)); - char c; + int c; while ((c = rl_read_key()) >= 0) { - if (c == '\r' || c == '\n') + if (c == '\r') c = '\n'; buf.push_back(c); if (buf.size() >= BRACK_PASTE_SLEN && c == BRACK_PASTE_LAST && buf.substr(buf.size() - BRACK_PASTE_SLEN) == BRACK_PASTE_SUFF) diff --git a/dbms/src/Columns/getLeastSuperColumn.cpp b/dbms/src/Columns/getLeastSuperColumn.cpp index 11c5b1b58fd..663337b6f87 100644 --- a/dbms/src/Columns/getLeastSuperColumn.cpp +++ b/dbms/src/Columns/getLeastSuperColumn.cpp @@ -18,7 +18,7 @@ static bool sameConstants(const IColumn & a, const IColumn & b) return assert_cast(a).getField() == assert_cast(b).getField(); } -ColumnWithTypeAndName getLeastSuperColumn(std::vector columns) +ColumnWithTypeAndName getLeastSuperColumn(const std::vector & columns) { if (columns.empty()) throw Exception("Logical error: no src columns for supercolumn", ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Columns/getLeastSuperColumn.h b/dbms/src/Columns/getLeastSuperColumn.h index d761de29a93..cd8dc53a895 100644 --- a/dbms/src/Columns/getLeastSuperColumn.h +++ b/dbms/src/Columns/getLeastSuperColumn.h @@ -7,6 +7,6 @@ namespace DB { /// getLeastSupertype + related column changes -ColumnWithTypeAndName getLeastSuperColumn(std::vector columns); +ColumnWithTypeAndName getLeastSuperColumn(const std::vector & columns); } diff --git a/dbms/src/DataStreams/NativeBlockInputStream.cpp b/dbms/src/DataStreams/NativeBlockInputStream.cpp index 246d1882a5d..1c73456154a 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockInputStream.cpp @@ -159,7 +159,7 @@ Block NativeBlockInputStream::readImpl() auto & header_column = header.getByName(column.name); if (!header_column.type->equals(*column.type)) { - column.column = recursiveLowCardinalityConversion(column.column, column.type, header.getByPosition(i).type); + column.column = recursiveTypeConversion(column.column, column.type, header.getByPosition(i).type); column.type = header.getByPosition(i).type; } } diff --git a/dbms/src/DataStreams/TTLBlockInputStream.cpp b/dbms/src/DataStreams/TTLBlockInputStream.cpp index e98ce4eb1b7..02191aec52d 100644 --- a/dbms/src/DataStreams/TTLBlockInputStream.cpp +++ b/dbms/src/DataStreams/TTLBlockInputStream.cpp @@ -203,8 +203,15 @@ UInt32 TTLBlockInputStream::getTimestampByIndex(const IColumn * column, size_t i return date_lut.fromDayNum(DayNum(column_date->getData()[ind])); else if (const ColumnUInt32 * column_date_time = typeid_cast(column)) return column_date_time->getData()[ind]; - else - throw Exception("Unexpected type of result ttl column", ErrorCodes::LOGICAL_ERROR); + else if (const ColumnConst * column_const = typeid_cast(column)) + { + if (typeid_cast(&column_const->getDataColumn())) + return date_lut.fromDayNum(DayNum(column_const->getValue())); + else if (typeid_cast(&column_const->getDataColumn())) + return column_const->getValue(); + } + + throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR); } } diff --git a/dbms/src/DataTypes/DataTypeLowCardinality.h b/dbms/src/DataTypes/DataTypeLowCardinality.h index 668045cc159..f8c314909b8 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinality.h +++ b/dbms/src/DataTypes/DataTypeLowCardinality.h @@ -126,6 +126,6 @@ DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type); ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column); /// Convert column of type from_type to type to_type by converting nested LowCardinality columns. -ColumnPtr recursiveLowCardinalityConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type); +ColumnPtr recursiveTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type); } diff --git a/dbms/src/DataTypes/DataTypeLowCardinalityHelpers.cpp b/dbms/src/DataTypes/DataTypeLowCardinalityHelpers.cpp index 6ebd6586c18..0fddd15b404 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinalityHelpers.cpp +++ b/dbms/src/DataTypes/DataTypeLowCardinalityHelpers.cpp @@ -84,7 +84,7 @@ ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column) return column; } -ColumnPtr recursiveLowCardinalityConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type) +ColumnPtr recursiveTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type) { if (!column) return column; @@ -92,10 +92,14 @@ ColumnPtr recursiveLowCardinalityConversion(const ColumnPtr & column, const Data if (from_type->equals(*to_type)) return column; + /// We can allow insert enum column if it's numeric type is the same as the column's type in table. + if (WhichDataType(to_type).isEnum() && from_type->getTypeId() == to_type->getTypeId()) + return column; + if (const auto * column_const = typeid_cast(column.get())) { auto & nested = column_const->getDataColumnPtr(); - auto nested_no_lc = recursiveLowCardinalityConversion(nested, from_type, to_type); + auto nested_no_lc = recursiveTypeConversion(nested, from_type, to_type); if (nested.get() == nested_no_lc.get()) return column; @@ -131,7 +135,7 @@ ColumnPtr recursiveLowCardinalityConversion(const ColumnPtr & column, const Data auto & nested_to = to_array_type->getNestedType(); return ColumnArray::create( - recursiveLowCardinalityConversion(column_array->getDataPtr(), nested_from, nested_to), + recursiveTypeConversion(column_array->getDataPtr(), nested_from, nested_to), column_array->getOffsetsPtr()); } } @@ -154,7 +158,7 @@ ColumnPtr recursiveLowCardinalityConversion(const ColumnPtr & column, const Data for (size_t i = 0; i < columns.size(); ++i) { auto & element = columns[i]; - auto element_no_lc = recursiveLowCardinalityConversion(element, from_elements.at(i), to_elements.at(i)); + auto element_no_lc = recursiveTypeConversion(element, from_elements.at(i), to_elements.at(i)); if (element.get() != element_no_lc.get()) { element = element_no_lc; diff --git a/dbms/src/IO/ReadHelpers.cpp b/dbms/src/IO/ReadHelpers.cpp index 860ce734c4c..ea54d37b1b1 100644 --- a/dbms/src/IO/ReadHelpers.cpp +++ b/dbms/src/IO/ReadHelpers.cpp @@ -1053,7 +1053,6 @@ 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()); diff --git a/dbms/src/IO/ReadHelpers.h b/dbms/src/IO/ReadHelpers.h index 4b162588d77..d0fdca24d75 100644 --- a/dbms/src/IO/ReadHelpers.h +++ b/dbms/src/IO/ReadHelpers.h @@ -924,8 +924,18 @@ if (method == DB::CompressionMethod::Gzip) return std::make_unique(args...); } -/// TODO (akuzm) - write comments for this and next function. +/** This function just copies the data from buffer's internal position (in.position()) + * to current position (from arguments) into memory. + */ void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current); + +/** This function is negative to eof(). + * In fact it returns whether the data was loaded to internal ReadBuffers's buffer or not. + * And saves data from buffer's position to current if there is no pending data in buffer. + * Why we have to use this strange function? Consider we have buffer's internal position in the middle + * of our buffer and the current cursor in the end of the buffer. When we call eof() it calls next(). + * And this function can fill the buffer with new data, so we will lose the data from previous buffer state. + */ bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current); } diff --git a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index a876633609a..2d7fde72875 100644 --- a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -124,17 +124,17 @@ Block InterpreterSelectWithUnionQuery::getCommonHeaderForUnion(const Blocks & he + "\nand\n" + headers[query_num].dumpNames() + "\n", ErrorCodes::UNION_ALL_RESULT_STRUCTURES_MISMATCH); + } - for (size_t column_num = 0; column_num < num_columns; ++column_num) - { - std::vector columns; - columns.reserve(num_selects); - for (size_t i = 0; i < num_selects; ++i) - columns.push_back(&headers[i].getByPosition(column_num)); + std::vector columns(num_selects); - ColumnWithTypeAndName & result_elem = common_header.getByPosition(column_num); - result_elem = getLeastSuperColumn(columns); - } + for (size_t column_num = 0; column_num < num_columns; ++column_num) + { + for (size_t i = 0; i < num_selects; ++i) + columns[i] = &headers[i].getByPosition(column_num); + + ColumnWithTypeAndName & result_elem = common_header.getByPosition(column_num); + result_elem = getLeastSuperColumn(columns); } return common_header; diff --git a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp index c6748601c83..5a6f13b0567 100644 --- a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp @@ -34,6 +34,8 @@ namespace DB namespace ErrorCodes { extern const int ABORTED; + extern const int UNKNOWN_CODEC; + extern const int CANNOT_DECOMPRESS; extern const int INCORRECT_FILE_NAME; extern const int CHECKSUM_DOESNT_MATCH; extern const int TOO_LARGE_SIZE_COMPRESSED; @@ -577,6 +579,8 @@ bool StorageDistributedDirectoryMonitor::isFileBrokenErrorCode(int code) return code == ErrorCodes::CHECKSUM_DOESNT_MATCH || code == ErrorCodes::TOO_LARGE_SIZE_COMPRESSED || code == ErrorCodes::CANNOT_READ_ALL_DATA + || code == ErrorCodes::UNKNOWN_CODEC + || code == ErrorCodes::CANNOT_DECOMPRESS || code == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index cb82921bfba..59ea60bb360 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -544,19 +544,6 @@ void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const Strin } } - bool has_date_column = false; - for (const auto & elem : ttl_expression->getRequiredColumnsWithTypes()) - { - if (typeid_cast(elem.type.get()) || typeid_cast(elem.type.get())) - { - has_date_column = true; - break; - } - } - - if (!has_date_column) - throw Exception("TTL expression should use at least one Date or DateTime column", ErrorCodes::BAD_TTL_EXPRESSION); - const auto & result_column = ttl_expression->getSampleBlock().getByName(result_column_name); if (!typeid_cast(result_column.type.get()) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp index ecad8b05600..cb76215897a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -96,8 +96,22 @@ void updateTTL(const MergeTreeData::TTLEntry & ttl_entry, MergeTreeDataPart::TTL for (const auto & val : column_date_time->getData()) ttl_info.update(val); } + else if (const ColumnConst * column_const = typeid_cast(column)) + { + if (typeid_cast(&column_const->getDataColumn())) + { + const auto & date_lut = DateLUT::instance(); + ttl_info.update(date_lut.fromDayNum(DayNum(column_const->getValue()))); + } + else if (typeid_cast(&column_const->getDataColumn())) + { + ttl_info.update(column_const->getValue()); + } + else + throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR); + } else - throw Exception("Unexpected type of result ttl column", ErrorCodes::LOGICAL_ERROR); + throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR); ttl_infos.updatePartMinMaxTTL(ttl_info.min, ttl_info.max); } diff --git a/dbms/tests/queries/0_stateless/00933_ttl_simple.reference b/dbms/tests/queries/0_stateless/00933_ttl_simple.reference index 09e5d7d1f02..e8b0c699aec 100644 --- a/dbms/tests/queries/0_stateless/00933_ttl_simple.reference +++ b/dbms/tests/queries/0_stateless/00933_ttl_simple.reference @@ -6,3 +6,11 @@ 2000-10-10 00:00:00 0 2100-10-10 00:00:00 3 2100-10-10 2 +CREATE TABLE default.ttl_00933_1 (`b` Int32, `a` Int32 TTL now() - 1000) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192 +1 0 +CREATE TABLE default.ttl_00933_1 (`b` Int32, `a` Int32 TTL now() + 1000) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192 +1 1 +CREATE TABLE default.ttl_00933_1 (`b` Int32, `a` Int32 TTL today() - 1) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192 +1 0 +CREATE TABLE default.ttl_00933_1 (`b` Int32, `a` Int32 TTL today() + 1) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192 +1 1 diff --git a/dbms/tests/queries/0_stateless/00933_ttl_simple.sql b/dbms/tests/queries/0_stateless/00933_ttl_simple.sql index 3a5cf465581..6c750f8749d 100644 --- a/dbms/tests/queries/0_stateless/00933_ttl_simple.sql +++ b/dbms/tests/queries/0_stateless/00933_ttl_simple.sql @@ -47,6 +47,42 @@ select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_1 final; select * from ttl_00933_1 order by d; +-- const DateTime TTL positive +drop table if exists ttl_00933_1; +create table ttl_00933_1 (b Int, a Int ttl now()-1000) engine = MergeTree order by tuple() partition by tuple(); +show create table ttl_00933_1; +insert into ttl_00933_1 values (1, 1); +select sleep(0.7) format Null; -- wait if very fast merge happen +optimize table ttl_00933_1 final; +select * from ttl_00933_1; + +-- const DateTime TTL negative +drop table if exists ttl_00933_1; +create table ttl_00933_1 (b Int, a Int ttl now()+1000) engine = MergeTree order by tuple() partition by tuple(); +show create table ttl_00933_1; +insert into ttl_00933_1 values (1, 1); +select sleep(0.7) format Null; -- wait if very fast merge happen +optimize table ttl_00933_1 final; +select * from ttl_00933_1; + +-- const Date TTL positive +drop table if exists ttl_00933_1; +create table ttl_00933_1 (b Int, a Int ttl today()-1) engine = MergeTree order by tuple() partition by tuple(); +show create table ttl_00933_1; +insert into ttl_00933_1 values (1, 1); +select sleep(0.7) format Null; -- wait if very fast merge happen +optimize table ttl_00933_1 final; +select * from ttl_00933_1; + +-- const Date TTL negative +drop table if exists ttl_00933_1; +create table ttl_00933_1 (b Int, a Int ttl today()+1) engine = MergeTree order by tuple() partition by tuple(); +show create table ttl_00933_1; +insert into ttl_00933_1 values (1, 1); +select sleep(0.7) format Null; -- wait if very fast merge happen +optimize table ttl_00933_1 final; +select * from ttl_00933_1; + set send_logs_level = 'none'; drop table if exists ttl_00933_1; @@ -54,7 +90,6 @@ drop table if exists ttl_00933_1; create table ttl_00933_1 (d DateTime ttl d) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 44} create table ttl_00933_1 (d DateTime, a Int ttl d) engine = MergeTree order by a partition by toSecond(d); -- { serverError 44} create table ttl_00933_1 (d DateTime, a Int ttl 2 + 2) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 450 } -create table ttl_00933_1 (d DateTime, a Int ttl toDateTime(1)) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 450 } create table ttl_00933_1 (d DateTime, a Int ttl d - d) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 450 } create table ttl_00933_1 (d DateTime, a Int ttl d + interval 1 day) engine = Log; -- { serverError 36 } diff --git a/dbms/tests/queries/0_stateless/01035_enum_conversion_native_format.reference b/dbms/tests/queries/0_stateless/01035_enum_conversion_native_format.reference new file mode 100644 index 00000000000..7e8a1653bf2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01035_enum_conversion_native_format.reference @@ -0,0 +1,2 @@ +a +a diff --git a/dbms/tests/queries/0_stateless/01035_enum_conversion_native_format.sh b/dbms/tests/queries/0_stateless/01035_enum_conversion_native_format.sh new file mode 100755 index 00000000000..47233914351 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01035_enum_conversion_native_format.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +set -e + +${CLICKHOUSE_CLIENT} --query="drop table if exists enum_source;" +${CLICKHOUSE_CLIENT} --query="drop table if exists enum_buf;" + +${CLICKHOUSE_CLIENT} --query="create table enum_source(e Enum8('a'=1)) engine = MergeTree order by tuple()" +${CLICKHOUSE_CLIENT} --query="insert into enum_source values ('a')" +${CLICKHOUSE_CLIENT} --query="create table enum_buf engine = Log as select * from enum_source;" +${CLICKHOUSE_CLIENT} --query="alter table enum_source modify column e Enum8('a'=1, 'b'=2);" + +${CLICKHOUSE_CLIENT} --query="select * from enum_buf format Native" \ + | ${CLICKHOUSE_CLIENT} --query="insert into enum_source format Native" + +${CLICKHOUSE_CLIENT} --query="select * from enum_source;" + +${CLICKHOUSE_CLIENT} --query="drop table enum_source;" +${CLICKHOUSE_CLIENT} --query="drop table enum_buf;" diff --git a/dbms/tests/queries/0_stateless/01036_union_different_columns.reference b/dbms/tests/queries/0_stateless/01036_union_different_columns.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/01036_union_different_columns.sql b/dbms/tests/queries/0_stateless/01036_union_different_columns.sql new file mode 100644 index 00000000000..f4936b948cb --- /dev/null +++ b/dbms/tests/queries/0_stateless/01036_union_different_columns.sql @@ -0,0 +1 @@ +select 1 as c1, 2 as c2, 3 as c3 union all (select 1 as c1, 2 as c2, 3 as c3 union all select 1 as c1, 2 as c2) -- { serverError 258 } diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index b046cfd6c6e..394313fccdb 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -131,7 +131,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va `clickhouse-client` uses the first existing file of the following: -- Defined in the `-config-file` parameter. +- Defined in the `--config-file` parameter. - `./clickhouse-client.xml` - `~/.clickhouse-client/config.xml` - `/etc/clickhouse-client/config.xml` diff --git a/docs/en/operations/table_engines/mergetree.md b/docs/en/operations/table_engines/mergetree.md index 014460981f9..bb995494cf3 100644 --- a/docs/en/operations/table_engines/mergetree.md +++ b/docs/en/operations/table_engines/mergetree.md @@ -72,7 +72,7 @@ For a description of parameters, see the [CREATE query description](../../query_ - `TTL` — An expression for setting storage time for rows. - It must depend on the `Date` or `DateTime` column and have one `Date` or `DateTime` column as a result. Example: + It must have one `Date` or `DateTime` column as a result. Example: `TTL date + INTERVAL 1 DAY` For more details, see [TTL for columns and tables](#table_engine-mergetree-ttl) @@ -373,7 +373,7 @@ Determines the lifetime of values. The `TTL` clause can be set for the whole table and for each individual column. If both `TTL` are set, ClickHouse uses that `TTL` which expires earlier. -The table must have the column in the [Date](../../data_types/date.md) or [DateTime](../../data_types/datetime.md) data type. To define the lifetime of data, use operations on this time column, for example: +To define the lifetime of data, use expression evaluating to [Date](../../data_types/date.md) or [DateTime](../../data_types/datetime.md) data type, for example: ```sql TTL time_column diff --git a/docs/en/query_language/alter.md b/docs/en/query_language/alter.md index e6667f3534a..023bb8cad9d 100644 --- a/docs/en/query_language/alter.md +++ b/docs/en/query_language/alter.md @@ -253,7 +253,7 @@ ALTER TABLE visits ATTACH PART 201901_2_2_0; Read more about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). -This query is replicated. Each replica checks whether there is data in the `detached` directory. If the data is in this directory, the query checks the integrity, verifies that it matches the data on the server that initiated the query. If everything is correct, the query adds data to the replica. If not, it downloads data from the query requestor replica, or from another replica where the data has already been added. +This query is replicated. The replica-initiator checks whether there is data in the `detached` directory. If data exists, the query checks its integrity. If everything is correct, the query adds the data to the table. All other replicas download the data from the replica-initiator. So you can put data to the `detached` directory on one replica, and use the `ALTER ... ATTACH` query to add it to the table on all replicas. diff --git a/docs/en/query_language/misc.md b/docs/en/query_language/misc.md index a6b112039c0..a9d8e7339e9 100644 --- a/docs/en/query_language/misc.md +++ b/docs/en/query_language/misc.md @@ -181,7 +181,7 @@ Changes already made by the mutation are not rolled back. ## OPTIMIZE {#misc_operations-optimize} ```sql -OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION ID 'partition_id'] [FINAL] +OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION ID 'partition_id'] [FINAL] [DEDUPLICATE] ``` This query tries to initialize an unscheduled merge of data parts for tables with a table engine from the [MergeTree](../operations/table_engines/mergetree.md) family. Other kinds of table engines aren't supported. @@ -191,6 +191,7 @@ When `OPTIMIZE` is used with the [ReplicatedMergeTree](../operations/table_engin - If `OPTIMIZE` doesn't perform a merge for any reason, it doesn't notify the client. To enable notifications, use the [optimize_throw_if_noop](../operations/settings/settings.md#setting-optimize_throw_if_noop) setting. - If you specify a `PARTITION`, only the specified partition is optimized. [How to set partition expression](alter.md#alter-how-to-specify-part-expr). - If you specify `FINAL`, optimization is performed even when all the data is already in one part. +- If you specify `DEDUPLICATE`, then completely identical rows will be deduplicated (all columns are compared), it makes sense only for the MergeTree engine. !!! warning "Warning" `OPTIMIZE` can't fix the "Too many parts" error. diff --git a/docs/fa/interfaces/cli.md b/docs/fa/interfaces/cli.md index 8501f46ecd7..7680348aef6 100644 --- a/docs/fa/interfaces/cli.md +++ b/docs/fa/interfaces/cli.md @@ -96,13 +96,13 @@ command line برا پایه 'readline' (و 'history' یا 'libedit'، یه بد - `--vertical, -E` اگر مشخص شود، از فرمت Vertical برای نمایش خروجی استفاده می شود. این گزینه مشابه '--format=Vertical' می باشد. در این فرمت، هر مقدار در یک خط جدید چاپ می شود، که در هنگام نمایش جداول عریض مفید است. - `--time, -t` اگر مشخص شود، در حالت non-interactive زمان اجرای query در 'stderr' جاپ می شود. - `--stacktrace` – اگر مشخص شود stack trase مربوط به اجرای query در هنگام رخ دادن یک exception چاپ می شود. -- `-config-file` – نام فایل پیکربندی. +- `--config-file` – نام فایل پیکربندی. ### فایل های پیکربندی `clickhouse-client` به ترتیب اولویت زیر از اولین فایل موجود برای ست کردن تنظیمات استفاده می کند: -- مشخص شده در پارامتر `-config-file` +- مشخص شده در پارامتر `--config-file` - `./clickhouse-client.xml` - `\~/.clickhouse-client/config.xml` - `/etc/clickhouse-client/config.xml` diff --git a/docs/ru/extended_roadmap.md b/docs/ru/extended_roadmap.md index c2f110664d5..2028828dd7e 100644 --- a/docs/ru/extended_roadmap.md +++ b/docs/ru/extended_roadmap.md @@ -563,12 +563,20 @@ Fuzzing тестирование - это тестирование случай ### 8.10. Запись в табличную функцию ODBC. +Артемий Бобровский, ВШЭ + ### 8.11. Движок таблиц для чтения из Mongo. +Артемий Бобровский, ВШЭ + ### 8.12. Пропуск столбцов в форматах Parquet, ORC. +Артемий Бобровский, ВШЭ + ### 8.13. Поддержка массивов в Parquet, ORC. +Артемий Бобровский, ВШЭ + ### 8.14. Запись данных в ORC. Возможно, Андрей Коняев, ArenaData (зависит от желания). diff --git a/docs/ru/operations/table_engines/mergetree.md b/docs/ru/operations/table_engines/mergetree.md index 187bc8ab6c6..4002a002c0d 100644 --- a/docs/ru/operations/table_engines/mergetree.md +++ b/docs/ru/operations/table_engines/mergetree.md @@ -72,7 +72,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] - `TTL` — выражение, определяющее длительность хранения строк. - Должно зависеть от столбца `Date` или `DateTime` и возвращать столбец `Date` или `DateTime`. Пример:`TTL date + INTERVAL 1 DAY` + Должно возвращать столбец `Date` или `DateTime`. Пример: `TTL date + INTERVAL 1 DAY`. Дополнительные сведения смотрите в разделе [TTL для столбцов и таблиц](#table_engine-mergetree-ttl) @@ -365,7 +365,7 @@ hasToken | ✗ | ✗ | ✗ | ✔ | ✗ Секция `TTL` может быть установлена как для всей таблицы, так и для каждого отдельного столбца. Если установлены оба `TTL`, то ClickHouse использует тот, что истекает раньше. -Таблица должна иметь столбец типа [Date](../../data_types/date.md) или [DateTime](../../data_types/datetime.md). Для установки времени жизни данных, следует использовать операцию со столбцом с временем, например: +Для установки времени жизни данных, следует использовать выражение, возвращающее тип [Date](../../data_types/date.md) или [DateTime](../../data_types/datetime.md), например: ```sql TTL time_column diff --git a/docs/ru/query_language/alter.md b/docs/ru/query_language/alter.md index 4c99b5152c1..62218ab9278 100644 --- a/docs/ru/query_language/alter.md +++ b/docs/ru/query_language/alter.md @@ -252,7 +252,7 @@ ALTER TABLE visits ATTACH PART 201901_2_2_0; Как корректно задать имя партиции или куска, см. в разделе [Как задавать имя партиции в запросах ALTER](#alter-how-to-specify-part-expr). -Этот запрос реплицируется. Каждая реплика проверяет, есть ли данные в директории `detached`. Если данные есть, то запрос проверяет их целостность и соответствие данным на сервере-инициаторе запроса. В случае успеха данные добавляются в таблицу. В противном случае, реплика загружает данные с реплики-инициатора запроса или с другой реплики, на которой эти данные уже добавлены. +Этот запрос реплицируется. Реплика-иницатор проверяет, есть ли данные в директории `detached`. Если данные есть, то запрос проверяет их целостность. В случае успеха данные добавляются в таблицу. Все остальные реплики загружают данные с реплики-инициатора запроса. Это означает, что вы можете разместить данные в директории `detached` на одной реплике и с помощью запроса `ALTER ... ATTACH` добавить их в таблицу на всех репликах. diff --git a/docs/ru/query_language/misc.md b/docs/ru/query_language/misc.md index 6a1d7c01a36..5a89856a143 100644 --- a/docs/ru/query_language/misc.md +++ b/docs/ru/query_language/misc.md @@ -173,7 +173,7 @@ KILL MUTATION WHERE database = 'default' AND table = 'table' AND mutation_id = ' ## OPTIMIZE {#misc_operations-optimize} ```sql -OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION ID 'partition_id'] [FINAL] +OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION ID 'partition_id'] [FINAL] [DEDUPLICATE] ``` Запрос пытается запустить внеплановый мёрж кусков данных для таблиц семейства [MergeTree](../operations/table_engines/mergetree.md). Другие движки таблиц не поддерживаются. @@ -183,6 +183,7 @@ OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION I - Если `OPTIMIZE` не выполняет мёрж по любой причине, ClickHouse не оповещает об этом клиента. Чтобы включить оповещения, используйте настройку [optimize_throw_if_noop](../operations/settings/settings.md#setting-optimize_throw_if_noop). - Если указать `PARTITION`, то оптимизация выполняется только для указанной партиции. [Как задавать имя партиции в запросах](alter.md#alter-how-to-specify-part-expr). - Если указать `FINAL`, то оптимизация выполняется даже в том случае, если все данные уже лежат в одном куске. +- Если указать `DEDUPLICATE`, то произойдет схлопывание полностью одинаковых строк (сравниваются значения во всех колонках), имеет смысл только для движка MergeTree. !!! warning "Внимание" Запрос `OPTIMIZE` не может устранить причину появления ошибки "Too many parts". diff --git a/docs/zh/development/developer_instruction.md b/docs/zh/development/developer_instruction.md deleted file mode 120000 index bdfa9047aa2..00000000000 --- a/docs/zh/development/developer_instruction.md +++ /dev/null @@ -1 +0,0 @@ -../../en/development/developer_instruction.md \ No newline at end of file diff --git a/docs/zh/development/developer_instruction.md b/docs/zh/development/developer_instruction.md new file mode 100644 index 00000000000..52e1d0fb1e2 --- /dev/null +++ b/docs/zh/development/developer_instruction.md @@ -0,0 +1,293 @@ +ClickHose支持Linux,FreeBSD 及 Mac OS X 系统。 + +# Windows使用指引 + +如果您的系统是Windows,则需要创建Ubuntu虚拟机。可以安装VirtualBox来构建虚拟机。Ubuntu的下载链接为:https://www.ubuntu.com/#download 。请使用下载好的镜像创建一个虚拟机(请确保虚拟机有至少4GB的内存容量)。在Ubuntu中使用"terminal"程序(gnome-terminal,konsole等)运行命令行终端,或使用快捷键Ctrl+Alt+T。 + + +# 在GitHub上创建源码库 + +您需要(申请)一个GitHub账户来使用ClickHouse。 + +如果没有账户,请在https://github.com上注册一个。如果没有SSH密钥,请在本地创建密钥并将其上传到GitHub上。这些交互都是必须的,也可以使用与其他任何SSH服务器相同的密钥。 + +要创建ClickHouse源码库的分支,请在https://github.com/ClickHouse/ClickHouse页面上点击右上角的"fork"按钮。它会在本账户上创建您个人的ClickHouse/ClickHouse分支。 + +若要参与开发,首先请在ClickHouse的分支中提交您期望的变更,然后创建一个“pull请求”,以便这些变更能够被(ClickHouse/ClickHouse)主库接受。 + +请先安装`git`来使用git源码库。 + +请在Ubuntu终端上使用下列的指令来安装`git`: + +``` +sudo apt update +sudo apt install git +``` + +在https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf中找到有关使用Git的简易手册。有关Git的详细手册,请参见: https://git-scm.com/book/ru/v2 。 + + +# 拷贝源码库到开发机 + +接下来,请将源码下载到开发机上。这步操作被称为“拷贝源码库”,是因为它在您的开发机上创建了源码库的本地副本。 + +在终端命令行输入下列指令: +``` +git clone --recursive git@guthub.com:your_github_username/ClickHouse.git +cd ClickHouse +``` +请注意,您需要将*your_github_username* 替换成实际使用的账户名! + +这个指令将创建一个包含项目副本的`ClickHouse`工作目录。 + +重要的是,工作目录的路径中不应包含空格,因为这可能会导致运行构建系统时出现问题。 + +请注意,ClickHouse源码库使用了`submodules`。这是对其他库的引用(即项目所依赖的外部库)。即在拷贝源码库时,需要如上述指令中那样指定`--recursive`。如果在拷贝源码库时没有包含子模块,需要执行使用下列的指令: + +``` +git submodule init +git submodule update +``` +可以通过 `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. +``` +这通常表示缺少用于连接GitHub的SSH密钥。这些密钥一般都在`~/.ssh`中。要接受SSH密钥,请在GitHub UI的设置页面中上传它们。 + +您还可以通过https协议来拷贝源码库: + +``` +git clone https://github.com/ClickHouse/ClickHouse.git +``` + +但是,这无法将变更提交到服务器上。您仍然可以暂时使用,并后续再添加SSH密钥,用`git remote`命令替换源码库的远程地址。 + +还可以将原始ClickHouse库的地址添加到本地库中,以便从那里获取更新: + +``` +git remote add upstream git@github.com:ClickHouse/ClickHouse.git +``` + +命令执行成功后,可以通过执行`git pull upstream master`,从ClickHouse的主分支中拉去更新。 + + +# 构建系统 + +ClickHouse使用 CMake 和 Ninja 来构建系统。 + +CMake - 一个可以生成Ninja文件的元构建系统(构建任务)。 +Ninja - 一个轻量级的构建系统,专注于速度,用于执行这些cmake生成的任务。 + +在Ubuntu,Debian或者Mint系统上执行`sudo apt install cmake ninja-build`来安装ninja。 + +在CentOS,RedHat系统上执行`sudo yum install cmake ninja-build`。 + +如果您曾经使用过Arch或Gentoo,那么也许知道如何安装CMake。 + +若要在Mac OS X上安装CMake和Ninja,请先安装Homebrew,然后再通过brew安装其他内容: + +``` +/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一起在子模块中构建。可以查看`contrib`中罗列的清单。 + +有一些库不是由源构建的,而是由系统提供,例如:ICU以及Readline,也建议安装。 + +Ubuntu: `sudo apt install libicu-dev libreadline-dev` + +Mac OS X: `brew install icu4c readline` + +但是,这些库本身都是可选的,ClickHouse即便没有它们也可以构建。ICU用于支持`ORDER BY`中的`COLLATE`(例如,对土耳其字母进行排序)。Readline用于在clickhouse-client中更便捷的指令输入。 + + +# C++ 编译器 + +GCC编译器从版本9开始,以及Clang版本>=8都可支持构建ClickHouse。 + +Yandex官方当前使用GCC构建ClickHouse,因为它生成的机器代码性能较好(根据测评,最多可以相差几个百分点)。Clang通常可以更加便捷的开发。我们的持续集成(CI)平台会运行大约十二种构建组合的检查。 + +在Ubuntu上安装GCC,请执行:`sudo apt install gcc g++` + +请使用`gcc --version`查看gcc的版本。如果gcc版本低于9,请参考此处的指示:https://clickhouse.yandex/docs/en/development/build/#install-gcc-9 。 + +在Mac OS X上安装GCC,请执行:`brew install gcc` + +如果您决定使用Clang,还可以同时安装 `libc++`以及`lld`,前提是您也熟悉它们。此外,也推荐使用`ccache`。 + + +# 构建的过程 + +如果当前已经准备好构建ClickHouse,我们建议您在`ClickHouse`中创建一个单独的目录`build`,其中包含所有构建组件: + +``` +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++编译器进行编译。 + +为了更快的构建,请使用`debug`构建类型-不含优化的构建。为此提供以下的参数`-D CMAKE_BUILD_TYPE=Debug`: + +``` +cmake -D CMAKE_BUILD_TYPE=Debug .. +``` + +您可以通过在`build`目录中运行此命令来更改构建类型。 + +运行ninja进行构建: + +``` +ninja clickhouse-server clickhouse-client +``` + +在此示例中,仅将构建所需的二进制文件。 + +如果您需要构建所有的二进制文件(utilities和tests),请运行不带参数的ninja: + +``` +ninja +``` + +全量构建需要大约30GB的可用磁盘空间或15GB的空间来构建主要的二进制文件。 + +当构建的机器上有大量内存时,可考虑设置与`-j`参数并行运行的构建任务数量: + +``` +ninja -j 1 clickhouse-server clickhouse-client +``` + +在拥有4GB内存的机器上,建议设置成1,在拥有8GB内存的机器上,建议按`-j 2`设置。 + +如果您收到以下消息: + +`ninja:error:loading'build.ninja':No such file or directory` + +则表示生成构建配置失败,请检查上述消息。 + +成功启动构建过程后,您将看到构建进度-已处理任务的数量和任务总数。 + +在libhdfs2库中生成有关protobuf文件的消息时,可能会显示诸如`libprotobuf WARNING`。它们没有影响,可以忽略不计。 + +成功构建后,会得到一个可执行文件`ClickHouse//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`。 + +如果您在Mac OS X 或者 FreeBSD上收到`Connection refused`的消息,请尝试指定主机地址为127.0.0.1: + +``` +clickhouse client --host 127.0.0.1 +``` + +您可以使用自定义构建的ClickHouse二进制文件替换系统中安装的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 +``` + + +# IDE (集成开发环境) + +如果您还不知道使用哪款IDE,我们推荐使用CLion。CLion是一款商业软件,但能够有30天的免费使用时间。它同时也对学生免费。CLion可以在Linux和Mac OS X上使用。 + +KDevelop和QTCreator是另外两款适合开发ClickHouse的替代IDE。尽管不太稳定,但KDevelop还是作为一款非常便捷的IDE。如果KDevelop在打开项目后不久崩溃,则您应该在打开项目文件列表后立即单击“全部停止”按钮。按此处理后,KDevelop可以正常使用。 + +作为简易的代码编辑器,您可以使用Sublime Text或Visual Studio Code或Kate(在Linux上都可用)。 + +值得一提的是CLion会创建自己的`build`路径,它还会自行选择`debug`作为构建类型。对于配置,它使用CLion中定义的CMake版本,而不是您安装的版本。最后,CLion会使用`make`而不是`ninja`去构建任务。这属于正常的现象,请记住这一点,以免造成混淆。 + + +# 编写代码 + +ClickHouse的架构描述可以在此处查看:https://clickhouse.yandex/docs/en/development/architecture/ + +代码风格指引:https://clickhouse.yandex/docs/en/development/style/ + +编写测试用例:https://clickhouse.yandex/docs/en/development/tests/ + +任务列表:https://github.com/yandex/ClickHouse/blob/master/dbms/tests/instructions/easy_tasks_sorted_en.md + + +# 测试数据 + +开发ClickHouse通常需要加载现实的数据集,尤其是在性能测试的场景。我们可以从Yandex.Metrica获取一组特别准备的匿名数据。这些数据需要额外使用3GB的空闲磁盘空间。请注意,完成大多数开发任务并不需要此数据。 + +``` +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 +``` + + +# 创建拉取请求 + +进入到GitHub 用户界面中的fork库。如果您已经在某个分支中进行开发,则需要选择该分支。在屏幕中有一个 "拉取请求"的按钮。实际上这等价于“创建一个请求以接受对主库的变更”。 + +即使工作尚未完成,也可以创建拉取请求。在这种情况下,请在标题的开头加上“WIP”(正在进行中),以便后续更改。这对于协同审查和讨论更改以及运行所有可用测试用例很有用。提供有关变更的简短描述很重要,这将在后续用于生成重新发布变更日志。 + +Yandex成员一旦在您的拉取请求上贴上“可以测试”标签,就会开始测试。一些初始检查项(例如,代码类型)的结果会在几分钟内反馈。构建的检查结果将在半小时内完成。而主要的测试用例集结果将在一小时内报告给您。 + +系统将分别为您的拉取请求准备ClickHouse二进制版本。若要检索这些构建信息,请在检查列表中单击“ ClickHouse构建检查”旁边的“详细信息”链接。在这里,您会找到指向ClickHouse的.deb软件包的直接链接,此外,甚至可以将其部署在生产服务器上(如果您不担心)。 + +某些构建项很可能会在首次构建时失败。这是因为我们同时检查了基于gcc和clang的构建,几乎所有现有的被clang启用的警告(总是带有`-Werror`标志)。在同一页面上,您可以找到所有构建的日志,因此不必以所有可能的方式构建ClickHouse。 diff --git a/docs/zh/interfaces/cli.md b/docs/zh/interfaces/cli.md index 933b43c9469..bc9be960c48 100644 --- a/docs/zh/interfaces/cli.md +++ b/docs/zh/interfaces/cli.md @@ -89,13 +89,13 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA - `--vertical, -E` – 如果指定,默认情况下使用垂直格式输出结果。这与 '--format=Vertical' 相同。在这种格式中,每个值都在单独的行上打印,这种方式对显示宽表很有帮助。 - `--time, -t` – 如果指定,非交互模式下会打印查询执行的时间到 'stderr' 中。 - `--stacktrace` – 如果指定,如果出现异常,会打印堆栈跟踪信息。 -- `-config-file` – 配置文件的名称。 +- `--config-file` – 配置文件的名称。 ### 配置文件 `clickhouse-client` 使用一下第一个存在的文件: -- 通过 `-config-file` 参数指定的文件. +- 通过 `--config-file` 参数指定的文件. - `./clickhouse-client.xml` - `\~/.clickhouse-client/config.xml` - `/etc/clickhouse-client/config.xml` diff --git a/libs/consistent-hashing/bitops.h b/libs/consistent-hashing/bitops.h index 697063ee77e..e07d5045cd3 100644 --- a/libs/consistent-hashing/bitops.h +++ b/libs/consistent-hashing/bitops.h @@ -26,7 +26,7 @@ inline uint32_t HI_32(uint64_t x) { return static_cast(x >> 32); } return std::numeric_limits::digits - __builtin_clzll(value); } #else - /// Stupid realization for non GCC-like compilers. Can use BSR from x86 instructions set. + /// Stupid implementation for non GCC-like compilers. Can use BSR from x86 instructions set. template inline unsigned GetValueBitCountImpl(T value) noexcept { unsigned result = 1; // result == 0 - impossible value, since value cannot be zero