diff --git a/cmake/find_llvm.cmake b/cmake/find_llvm.cmake index d9805b03303..e540c3eafa9 100644 --- a/cmake/find_llvm.cmake +++ b/cmake/find_llvm.cmake @@ -1,5 +1,8 @@ -option (ENABLE_EMBEDDED_COMPILER "Set to TRUE to enable support for 'compile' option for query execution" 1) -option (USE_INTERNAL_LLVM_LIBRARY "Use bundled or system LLVM library. Default: system library for quicker developer builds." ${APPLE}) +# Broken in macos. TODO: update clang, re-test, enable +if (NOT APPLE) + option (ENABLE_EMBEDDED_COMPILER "Set to TRUE to enable support for 'compile' option for query execution" 1) + option (USE_INTERNAL_LLVM_LIBRARY "Use bundled or system LLVM library. Default: system library for quicker developer builds." ${APPLE}) +endif () if (ENABLE_EMBEDDED_COMPILER) if (USE_INTERNAL_LLVM_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/llvm/llvm/CMakeLists.txt") diff --git a/contrib/boost-cmake/README b/contrib/boost-cmake/README index 7eec9460acd..69b266c7c7d 100644 --- a/contrib/boost-cmake/README +++ b/contrib/boost-cmake/README @@ -3,7 +3,7 @@ Motivation For reproducible build, we need to control, what exactly version of boost we build, because different versions of boost obviously have slightly different behaviour. -You may already have installed arbitary version of boost on your system, to build another projects. +You may already have installed arbitrary version of boost on your system, to build another projects. We need to have all libraries with C++ interface to be located in tree and to be build together. This is needed to allow quickly changing build options, that could introduce changes in ABI of that libraries. diff --git a/contrib/libsparsehash/README b/contrib/libsparsehash/README index 8c806b42a38..527cfa1d304 100644 --- a/contrib/libsparsehash/README +++ b/contrib/libsparsehash/README @@ -1,7 +1,7 @@ This directory contains several hash-map implementations, similar in API to SGI's hash_map class, but with different performance characteristics. sparse_hash_map uses very little space overhead, 1-2 -bits per entry. dense_hash_map is very fast, particulary on lookup. +bits per entry. dense_hash_map is very fast, particularly on lookup. (sparse_hash_set and dense_hash_set are the set versions of these routines.) On the other hand, these classes have requirements that may not make them appropriate for all applications. diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index d0e8bdaeac8..e62a8e4796c 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -2,10 +2,10 @@ set(VERSION_REVISION 54409 CACHE STRING "") set(VERSION_MAJOR 18 CACHE STRING "") set(VERSION_MINOR 14 CACHE STRING "") -set(VERSION_PATCH 6 CACHE STRING "") -set(VERSION_GITHASH 899f41741cf9824d94bf6562c21b90c62ea4fee0 CACHE STRING "") -set(VERSION_DESCRIBE v18.14.6-testing CACHE STRING "") -set(VERSION_STRING 18.14.6 CACHE STRING "") +set(VERSION_PATCH 9 CACHE STRING "") +set(VERSION_GITHASH 457f8fd495b2812940e69c15ab5b499cd863aae4 CACHE STRING "") +set(VERSION_DESCRIBE v18.14.9-testing CACHE STRING "") +set(VERSION_STRING 18.14.9 CACHE STRING "") # end of autochange set(VERSION_EXTRA "" CACHE STRING "") diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index b9dc202a5d7..161321f4d8c 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -110,6 +110,8 @@ private: /// Don't execute new queries after timelimit or SIGINT or exception std::atomic shutdown{false}; + std::atomic queries_executed{0}; + struct Stats { Stopwatch watch; @@ -238,10 +240,12 @@ private: size_t query_index = randomize ? distribution(generator) : i % queries.size(); if (!tryPushQueryInteractively(queries[query_index], interrupt_listener)) + { + shutdown = true; break; + } } - shutdown = true; pool.wait(); info_total.watch.stop(); @@ -274,11 +278,12 @@ private: { extracted = queue.tryPop(query, 100); - if (shutdown) + if (shutdown || (max_iterations && queries_executed == max_iterations)) return; } execute(connection, query); + ++queries_executed; } } catch (...) @@ -419,20 +424,20 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) boost::program_options::options_description desc("Allowed options"); desc.add_options() - ("help", "produce help message") - ("concurrency,c", value()->default_value(1), "number of parallel queries") - ("delay,d", value()->default_value(1), "delay between intermediate reports in seconds (set 0 to disable reports)") - ("stage", value()->default_value("complete"), "request query processing up to specified stage") - ("iterations,i", value()->default_value(0), "amount of queries to be executed") - ("timelimit,t", value()->default_value(0.), "stop launch of queries after specified time limit") - ("randomize,r", value()->default_value(false), "randomize order of execution") - ("json", value()->default_value(""), "write final report to specified file in JSON format") - ("host,h", value()->default_value("localhost"), "") - ("port", value()->default_value(9000), "") - ("user", value()->default_value("default"), "") - ("password", value()->default_value(""), "") - ("database", value()->default_value("default"), "") - ("stacktrace", "print stack traces of exceptions") + ("help", "produce help message") + ("concurrency,c", value()->default_value(1), "number of parallel queries") + ("delay,d", value()->default_value(1), "delay between intermediate reports in seconds (set 0 to disable reports)") + ("stage", value()->default_value("complete"), "request query processing up to specified stage") + ("iterations,i", value()->default_value(0), "amount of queries to be executed") + ("timelimit,t", value()->default_value(0.), "stop launch of queries after specified time limit") + ("randomize,r", value()->default_value(false), "randomize order of execution") + ("json", value()->default_value(""), "write final report to specified file in JSON format") + ("host,h", value()->default_value("localhost"), "") + ("port", value()->default_value(9000), "") + ("user", value()->default_value("default"), "") + ("password", value()->default_value(""), "") + ("database", value()->default_value("default"), "") + ("stacktrace", "print stack traces of exceptions") #define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, boost::program_options::value (), DESCRIPTION) APPLY_FOR_SETTINGS(DECLARE_SETTING) diff --git a/dbms/programs/obfuscator/Obfuscator.cpp b/dbms/programs/obfuscator/Obfuscator.cpp index d5c6b6bacf6..f7eb7c5222d 100644 --- a/dbms/programs/obfuscator/Obfuscator.cpp +++ b/dbms/programs/obfuscator/Obfuscator.cpp @@ -500,7 +500,7 @@ private: return CRC32Hash()(StringRef(reinterpret_cast(begin), (end - begin) * sizeof(CodePoint))); } - /// By the way, we don't have to use actual Unicode numbers. We use just arbitary bijective mapping. + /// By the way, we don't have to use actual Unicode numbers. We use just arbitrary bijective mapping. CodePoint readCodePoint(const char *& pos, const char * end) { size_t length = UTF8::seqLength(*pos); @@ -954,7 +954,7 @@ try ("structure,S", po::value(), "structure of the initial table (list of column and type names)") ("input-format", po::value(), "input format of the initial table data") ("output-format", po::value(), "default output format") - ("seed", po::value(), "seed (arbitary string), must be random string with at least 10 bytes length") + ("seed", po::value(), "seed (arbitrary string), must be random string with at least 10 bytes length") ("limit", po::value(), "if specified - stop after generating that number of rows") ("silent", po::value()->default_value(false), "don't print information messages to stderr") ("order", po::value()->default_value(5), "order of markov model to generate strings") diff --git a/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp b/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp index 51302410a08..f59abd5f587 100644 --- a/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -94,7 +94,8 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques { schema_name = params.get("schema"); LOG_TRACE(log, "Will fetch info for table '" << schema_name + "." + table_name << "'"); - } else + } + else LOG_TRACE(log, "Will fetch info for table '" << table_name << "'"); LOG_TRACE(log, "Got connection str '" << connection_string << "'"); diff --git a/dbms/programs/odbc-bridge/README.md b/dbms/programs/odbc-bridge/README.md index 91a6e476733..70b413c9c1f 100644 --- a/dbms/programs/odbc-bridge/README.md +++ b/dbms/programs/odbc-bridge/README.md @@ -5,8 +5,8 @@ was possible segfaults or another faults in ODBC implementations, which can crash whole clickhouse-server process. This tool works via HTTP, not via pipes, shared memory, or TCP because: -- It's simplier to implement -- It's simplier to debug +- It's simpler to implement +- It's simpler to debug - jdbc-bridge can be implemented in the same way ## Usage diff --git a/dbms/programs/odbc-bridge/validateODBCConnectionString.h b/dbms/programs/odbc-bridge/validateODBCConnectionString.h index f0f93b1de65..8d2a23ca877 100644 --- a/dbms/programs/odbc-bridge/validateODBCConnectionString.h +++ b/dbms/programs/odbc-bridge/validateODBCConnectionString.h @@ -6,10 +6,10 @@ namespace DB { -/** Passing arbitary connection string to ODBC Driver Manager is insecure, for the following reasons: +/** Passing arbitrary connection string to ODBC Driver Manager is insecure, for the following reasons: * 1. Driver Manager like unixODBC has multiple bugs like buffer overflow. * 2. Driver Manager can interpret some parameters as a path to library for dlopen or a file to read, - * thus allows arbitary remote code execution. + * thus allows arbitrary remote code execution. * * This function will throw exception if connection string has insecure parameters. * It may also modify connection string to harden it. diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h index b998ee97216..5ad93c042c2 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h @@ -107,7 +107,7 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { /// TODO Do positions need to be 1-based for this function? - size_t position = columns[1]->get64(row_num); + size_t position = columns[1]->getUInt(row_num); /// If position is larger than size to which array will be cutted - simply ignore value. if (length_to_resize && position >= length_to_resize) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index a2dbd21a342..51d1e8d1dd7 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -607,7 +607,7 @@ struct AggregateFunctionAnyLastData : Data /** Implement 'heavy hitters' algorithm. * Selects most frequent value if its frequency is more than 50% in each thread of execution. - * Otherwise, selects some arbitary value. + * Otherwise, selects some arbitrary value. * http://www.cs.umd.edu/~samir/498/karp.pdf */ template diff --git a/dbms/src/AggregateFunctions/AggregateFunctionNothing.h b/dbms/src/AggregateFunctions/AggregateFunctionNothing.h index 060561396fe..3a98807bb4a 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionNothing.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionNothing.h @@ -10,7 +10,7 @@ namespace DB { -/** Aggregate function that takes arbitary number of arbitary arguments and does nothing. +/** Aggregate function that takes arbitrary number of arbitrary arguments and does nothing. */ class AggregateFunctionNothing final : public IAggregateFunctionHelper { diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 1ed186085f7..727fa6fb5a0 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -231,6 +231,14 @@ void Connection::getServerVersion(String & name, UInt64 & version_major, UInt64 revision = server_revision; } +UInt64 Connection::getServerRevision() +{ + if (!connected) + connect(); + + return server_revision; +} + const String & Connection::getServerTimezone() { if (!connected) @@ -349,7 +357,7 @@ void Connection::sendQuery( { ClientInfo client_info_to_send; - if (!client_info) + if (!client_info || client_info->empty()) { /// No client info passed - means this query initiated by me. client_info_to_send.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; diff --git a/dbms/src/Client/Connection.h b/dbms/src/Client/Connection.h index ad98df3cc8f..d8229fc3463 100644 --- a/dbms/src/Client/Connection.h +++ b/dbms/src/Client/Connection.h @@ -106,6 +106,7 @@ public: void setDefaultDatabase(const String & database); void getServerVersion(String & name, UInt64 & version_major, UInt64 & version_minor, UInt64 & version_patch, UInt64 & revision); + UInt64 getServerRevision(); const String & getServerTimezone(); const String & getServerDisplayName(); diff --git a/dbms/src/Client/ConnectionPoolWithFailover.cpp b/dbms/src/Client/ConnectionPoolWithFailover.cpp index 8c2f1a07bc1..d71d1f08339 100644 --- a/dbms/src/Client/ConnectionPoolWithFailover.cpp +++ b/dbms/src/Client/ConnectionPoolWithFailover.cpp @@ -153,13 +153,9 @@ ConnectionPoolWithFailover::tryGetEntry( { result.entry = pool.get(settings, /* force_connected = */ false); - String server_name; - UInt64 server_version_major; - UInt64 server_version_minor; - UInt64 server_version_patch; - UInt64 server_revision; + UInt64 server_revision = 0; if (table_to_check) - result.entry->getServerVersion(server_name, server_version_major, server_version_minor, server_version_patch, server_revision); + server_revision = result.entry->getServerRevision(); if (!table_to_check || server_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS) { diff --git a/dbms/src/Client/MultiplexedConnections.cpp b/dbms/src/Client/MultiplexedConnections.cpp index 3e88a20caa3..5228f9b85b3 100644 --- a/dbms/src/Client/MultiplexedConnections.cpp +++ b/dbms/src/Client/MultiplexedConnections.cpp @@ -87,28 +87,36 @@ void MultiplexedConnections::sendQuery( if (sent_query) throw Exception("Query already sent.", ErrorCodes::LOGICAL_ERROR); - if (replica_states.size() > 1) + Settings modified_settings = settings; + + for (auto & replica : replica_states) { - Settings query_settings = settings; - query_settings.parallel_replicas_count = replica_states.size(); + if (!replica.connection) + throw Exception("MultiplexedConnections: Internal error", ErrorCodes::LOGICAL_ERROR); - for (size_t i = 0; i < replica_states.size(); ++i) + if (replica.connection->getServerRevision() < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) { - Connection * connection = replica_states[i].connection; - if (connection == nullptr) - throw Exception("MultiplexedConnections: Internal error", ErrorCodes::LOGICAL_ERROR); + /// Disable two-level aggregation due to version incompatibility. + modified_settings.group_by_two_level_threshold = 0; + modified_settings.group_by_two_level_threshold_bytes = 0; + } + } - query_settings.parallel_replica_offset = i; - connection->sendQuery(query, query_id, stage, &query_settings, client_info, with_pending_data); + size_t num_replicas = replica_states.size(); + if (num_replicas > 1) + { + /// Use multiple replicas for parallel query processing. + modified_settings.parallel_replicas_count = num_replicas; + for (size_t i = 0; i < num_replicas; ++i) + { + modified_settings.parallel_replica_offset = i; + replica_states[i].connection->sendQuery(query, query_id, stage, &modified_settings, client_info, with_pending_data); } } else { - Connection * connection = replica_states[0].connection; - if (connection == nullptr) - throw Exception("MultiplexedConnections: Internal error", ErrorCodes::LOGICAL_ERROR); - - connection->sendQuery(query, query_id, stage, &settings, client_info, with_pending_data); + /// Use single replica. + replica_states[0].connection->sendQuery(query, query_id, stage, &modified_settings, client_info, with_pending_data); } sent_query = true; diff --git a/dbms/src/Columns/ColumnAggregateFunction.cpp b/dbms/src/Columns/ColumnAggregateFunction.cpp index b50bb81f884..8f8a44be868 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.cpp +++ b/dbms/src/Columns/ColumnAggregateFunction.cpp @@ -373,7 +373,7 @@ const char * ColumnAggregateFunction::deserializeAndInsertFromArena(const char * /** We will read from src_arena. * There is no limit for reading - it is assumed, that we can read all that we need after src_arena pointer. - * Buf ReadBufferFromMemory requires some bound. We will use arbitary big enough number, that will not overflow pointer. + * Buf ReadBufferFromMemory requires some bound. We will use arbitrary big enough number, that will not overflow pointer. * NOTE Technically, this is not compatible with C++ standard, * as we cannot legally compare pointers after last element + 1 of some valid memory region. * Probably this will not work under UBSan. diff --git a/dbms/src/Columns/ColumnConst.h b/dbms/src/Columns/ColumnConst.h index bd40246b1e5..c9038cdf709 100644 --- a/dbms/src/Columns/ColumnConst.h +++ b/dbms/src/Columns/ColumnConst.h @@ -15,7 +15,7 @@ namespace ErrorCodes /** ColumnConst contains another column with single element, - * but looks like a column with arbitary amount of same elements. + * but looks like a column with arbitrary amount of same elements. */ class ColumnConst final : public COWPtrHelper { diff --git a/dbms/src/Columns/ColumnDecimal.cpp b/dbms/src/Columns/ColumnDecimal.cpp index 01ec69b7eb1..092ee74d662 100644 --- a/dbms/src/Columns/ColumnDecimal.cpp +++ b/dbms/src/Columns/ColumnDecimal.cpp @@ -1,6 +1,3 @@ -#include -#include - #include #include #include @@ -53,7 +50,7 @@ UInt64 ColumnDecimal::get64(size_t n) const { if constexpr (sizeof(T) > sizeof(UInt64)) throw Exception(String("Method get64 is not supported for ") + getFamilyName(), ErrorCodes::NOT_IMPLEMENTED); - return ext::bit_cast(data[n]); + return static_cast(data[n]); } template @@ -120,6 +117,14 @@ MutableColumnPtr ColumnDecimal::cloneResized(size_t size) const return std::move(res); } +template +void ColumnDecimal::insertData(const char * src, size_t /*length*/) +{ + T tmp; + memcpy(&tmp, src, sizeof(T)); + data.emplace_back(tmp); +} + template void ColumnDecimal::insertRangeFrom(const IColumn & src, size_t start, size_t length) { diff --git a/dbms/src/Columns/ColumnDecimal.h b/dbms/src/Columns/ColumnDecimal.h index e2e52bfe8ed..523064167d7 100644 --- a/dbms/src/Columns/ColumnDecimal.h +++ b/dbms/src/Columns/ColumnDecimal.h @@ -89,7 +89,7 @@ public: void reserve(size_t n) override { data.reserve(n); } void insertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast(src).getData()[n]); } - void insertData(const char * pos, size_t /*length*/) override { data.push_back(*reinterpret_cast(pos)); } + void insertData(const char * pos, size_t /*length*/) override; void insertDefault() override { data.push_back(T()); } void insert(const Field & x) override { data.push_back(DB::get::Type>(x)); } void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; @@ -112,6 +112,7 @@ public: bool getBool(size_t n) const override { return bool(data[n]); } Int64 getInt(size_t n) const override { return Int64(data[n] * scale); } UInt64 get64(size_t n) const override; + bool isDefaultAt(size_t n) const override { return data[n] == 0; } ColumnPtr filter(const IColumn::Filter & filt, ssize_t result_size_hint) const override; ColumnPtr permute(const IColumn::Permutation & perm, size_t limit) const override; diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 034bf9d5ca4..82a68662588 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -92,7 +92,7 @@ public: } /** If column is numeric, return value of n-th element, casted to UInt64. - * For NULL values of Nullable column it is allowed to return arbitary value. + * For NULL values of Nullable column it is allowed to return arbitrary value. * Otherwise throw an exception. */ virtual UInt64 getUInt(size_t /*n*/) const @@ -105,6 +105,7 @@ public: throw Exception("Method getInt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + virtual bool isDefaultAt(size_t n) const { return get64(n) == 0; } virtual bool isNullAt(size_t /*n*/) const { return false; } /** If column is numeric, return value of n-th element, casted to bool. @@ -174,7 +175,7 @@ public: virtual const char * deserializeAndInsertFromArena(const char * pos) = 0; /// Update state of hash function with value of n-th element. - /// On subsequent calls of this method for sequence of column values of arbitary types, + /// On subsequent calls of this method for sequence of column values of arbitrary types, /// passed bytes to hash must identify sequence of values unambiguously. virtual void updateHashWithValue(size_t n, SipHash & hash) const = 0; diff --git a/dbms/src/Common/XDBCBridgeHelper.h b/dbms/src/Common/XDBCBridgeHelper.h index 93e4a0d697f..8cb025df7ab 100644 --- a/dbms/src/Common/XDBCBridgeHelper.h +++ b/dbms/src/Common/XDBCBridgeHelper.h @@ -254,7 +254,8 @@ struct ODBCBridgeMixin static void startBridge(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log, const Poco::Timespan & http_timeout) { - Poco::Path path{config.getString("application.dir", "")}; + /// Path to executable folder + Poco::Path path{config.getString("application.dir", "/usr/bin")}; path.setFileName( #if CLICKHOUSE_SPLIT_BINARY @@ -264,9 +265,6 @@ struct ODBCBridgeMixin #endif ); - if (!Poco::File(path).exists()) - throw Exception("clickhouse binary (" + path.toString() + ") is not found", ErrorCodes::EXTERNAL_EXECUTABLE_NOT_FOUND); - std::stringstream command; command << path.toString() << diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index 27b107eeb09..fa0038e41e8 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -19,7 +19,7 @@ namespace DB * This is unit of data processing. * Also contains metadata - data types of columns and their names * (either original names from a table, or generated names during temporary calculations). - * Allows to insert, remove columns in arbitary position, to change order of columns. + * Allows to insert, remove columns in arbitrary position, to change order of columns. */ class Context; diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index d7a2ca419e3..cf7a0b621e0 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -47,6 +47,10 @@ #define DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME 54372 #define DBMS_MIN_REVISION_WITH_VERSION_PATCH 54401 #define DBMS_MIN_REVISION_WITH_SERVER_LOGS 54406 +/// Minimum revision with exactly the same set of aggregation methods and rules to select them. +/// Two-level (bucketed) aggregation is incompatible if servers are inconsistent in these rules +/// (keys will be placed in different buckets and result will not be fully aggregated). +#define DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 54408 /// Version of ClickHouse TCP protocol. Set to git tag with latest protocol change. #define DBMS_TCP_PROTOCOL_VERSION 54226 diff --git a/dbms/src/Core/callOnTypeIndex.h b/dbms/src/Core/callOnTypeIndex.h index e531e64d158..8f4424ec016 100644 --- a/dbms/src/Core/callOnTypeIndex.h +++ b/dbms/src/Core/callOnTypeIndex.h @@ -16,7 +16,7 @@ struct TypePair -template +template bool callOnBasicType(TypeIndex number, F && f) { if constexpr (_int) @@ -40,7 +40,7 @@ bool callOnBasicType(TypeIndex number, F && f) } } - if constexpr (_dec) + if constexpr (_decimal) { switch (number) { @@ -63,40 +63,51 @@ bool callOnBasicType(TypeIndex number, F && f) } } + if constexpr (_datetime) + { + switch (number) + { + case TypeIndex::Date: return f(TypePair()); + case TypeIndex::DateTime: return f(TypePair()); + default: + break; + } + } + return false; } /// Unroll template using TypeIndex -template +template inline bool callOnBasicTypes(TypeIndex type_num1, TypeIndex type_num2, F && f) { if constexpr (_int) { switch (type_num1) { - case TypeIndex::UInt8: return callOnBasicType(type_num2, std::forward(f)); - case TypeIndex::UInt16: return callOnBasicType(type_num2, std::forward(f)); - case TypeIndex::UInt32: return callOnBasicType(type_num2, std::forward(f)); - case TypeIndex::UInt64: return callOnBasicType(type_num2, std::forward(f)); - //case TypeIndex::UInt128: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::UInt8: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::UInt16: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::UInt32: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::UInt64: return callOnBasicType(type_num2, std::forward(f)); + //case TypeIndex::UInt128: return callOnBasicType(type_num2, std::forward(f)); - case TypeIndex::Int8: return callOnBasicType(type_num2, std::forward(f)); - case TypeIndex::Int16: return callOnBasicType(type_num2, std::forward(f)); - case TypeIndex::Int32: return callOnBasicType(type_num2, std::forward(f)); - case TypeIndex::Int64: return callOnBasicType(type_num2, std::forward(f)); - case TypeIndex::Int128: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::Int8: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::Int16: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::Int32: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::Int64: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::Int128: return callOnBasicType(type_num2, std::forward(f)); default: break; } } - if constexpr (_dec) + if constexpr (_decimal) { switch (type_num1) { - case TypeIndex::Decimal32: return callOnBasicType(type_num2, std::forward(f)); - case TypeIndex::Decimal64: return callOnBasicType(type_num2, std::forward(f)); - case TypeIndex::Decimal128: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::Decimal32: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::Decimal64: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::Decimal128: return callOnBasicType(type_num2, std::forward(f)); default: break; } @@ -106,8 +117,19 @@ inline bool callOnBasicTypes(TypeIndex type_num1, TypeIndex type_num2, F && f) { switch (type_num1) { - case TypeIndex::Float32: return callOnBasicType(type_num2, std::forward(f)); - case TypeIndex::Float64: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::Float32: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::Float64: return callOnBasicType(type_num2, std::forward(f)); + default: + break; + } + } + + if constexpr (_datetime) + { + switch (type_num1) + { + case TypeIndex::Date: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::DateTime: return callOnBasicType(type_num2, std::forward(f)); default: break; } diff --git a/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.cpp b/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.cpp index 294f98d20c3..a2740dabdaa 100644 --- a/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.cpp @@ -132,7 +132,7 @@ void GraphiteRollupSortedBlockInputStream::merge(MutableColumns & merged_columns is_first = false; - time_t next_row_time = next_cursor->all_columns[time_column_num]->get64(next_cursor->pos); + time_t next_row_time = next_cursor->all_columns[time_column_num]->getUInt(next_cursor->pos); /// Is new key before rounding. bool is_new_key = new_path || next_row_time != current_time; diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp index 00f3f55c3a8..d2af46f198e 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp @@ -216,7 +216,7 @@ void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & me if (desc.column_numbers.size() == 1) { // Flag row as non-empty if at least one column number if non-zero - current_row_is_zero = current_row_is_zero && desc.merged_column->get64(desc.merged_column->size() - 1) == 0; + current_row_is_zero = current_row_is_zero && desc.merged_column->isDefaultAt(desc.merged_column->size() - 1); } else { diff --git a/dbms/src/DataTypes/DataTypeDateTime.cpp b/dbms/src/DataTypes/DataTypeDateTime.cpp index d012d45d2ae..519f419e8c4 100644 --- a/dbms/src/DataTypes/DataTypeDateTime.cpp +++ b/dbms/src/DataTypes/DataTypeDateTime.cpp @@ -172,6 +172,7 @@ static DataTypePtr create(const ASTPtr & arguments) void registerDataTypeDateTime(DataTypeFactory & factory) { factory.registerDataType("DateTime", create, DataTypeFactory::CaseInsensitive); + factory.registerAlias("TIMESTAMP", "DateTime", DataTypeFactory::CaseInsensitive); } diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index e3f02c9b4f6..727d80540ce 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -304,7 +304,7 @@ public: virtual bool shouldAlignRightInPrettyFormats() const { return false; } /** Does formatted value in any text format can contain anything but valid UTF8 sequences. - * Example: String (because it can contain arbitary bytes). + * Example: String (because it can contain arbitrary bytes). * Counterexamples: numbers, Date, DateTime. * For Enum, it depends. */ diff --git a/dbms/src/DataTypes/NumberTraits.h b/dbms/src/DataTypes/NumberTraits.h index 099a7199a7d..3ebe9420052 100644 --- a/dbms/src/DataTypes/NumberTraits.h +++ b/dbms/src/DataTypes/NumberTraits.h @@ -1,8 +1,10 @@ #pragma once -#include #include +#include +#include + namespace DB { @@ -146,6 +148,7 @@ template struct ResultOfBitNot * UInt, Int -> Int * Float, [U]Int -> Float * Decimal, Decimal -> Decimal + * UUID, UUID -> UUID * UInt64 , Int -> Error * Float, [U]Int64 -> Error */ @@ -168,7 +171,9 @@ struct ResultOfIf ? max(sizeof(A), sizeof(B)) * 2 : max(sizeof(A), sizeof(B))>::Type; - using Type = std::conditional_t && !IsDecimalNumber, ConstructedType, + using ConstructedWithUUID = std::conditional_t && std::is_same_v, A, ConstructedType>; + + using Type = std::conditional_t && !IsDecimalNumber, ConstructedWithUUID, std::conditional_t && IsDecimalNumber, std::conditional_t<(sizeof(A) > sizeof(B)), A, B>, Error>>; }; diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 43cb96ec05a..7bb4ae298e6 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -1,5 +1,6 @@ #include +#include #include #include @@ -41,7 +42,6 @@ namespace ErrorCodes static constexpr size_t PRINT_MESSAGE_EACH_N_TABLES = 256; static constexpr size_t PRINT_MESSAGE_EACH_N_SECONDS = 5; static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768; -static constexpr size_t TABLES_PARALLEL_LOAD_BUNCH_SIZE = 100; namespace detail { @@ -149,6 +149,9 @@ void DatabaseOrdinary::loadTables( ErrorCodes::INCORRECT_FILE_NAME); } + if (file_names.empty()) + return; + /** Tables load faster if they are loaded in sorted (by name) order. * Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order, * which does not correspond to order tables creation and does not correspond to order of their location on disk. @@ -160,36 +163,27 @@ void DatabaseOrdinary::loadTables( AtomicStopwatch watch; std::atomic tables_processed {0}; + Poco::Event all_tables_processed; - auto task_function = [&](FileNames::const_iterator begin, FileNames::const_iterator end) + auto task_function = [&](const String & table) { - for (auto it = begin; it != end; ++it) + /// Messages, so that it's not boring to wait for the server to load for a long time. + if ((tables_processed + 1) % PRINT_MESSAGE_EACH_N_TABLES == 0 + || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS)) { - const String & table = *it; - - /// Messages, so that it's not boring to wait for the server to load for a long time. - if ((++tables_processed) % PRINT_MESSAGE_EACH_N_TABLES == 0 - || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS)) - { - LOG_INFO(log, std::fixed << std::setprecision(2) << tables_processed * 100.0 / total_tables << "%"); - watch.restart(); - } - - loadTable(context, metadata_path, *this, name, data_path, table, has_force_restore_data_flag); + LOG_INFO(log, std::fixed << std::setprecision(2) << tables_processed * 100.0 / total_tables << "%"); + watch.restart(); } + + loadTable(context, metadata_path, *this, name, data_path, table, has_force_restore_data_flag); + + if (++tables_processed == total_tables) + all_tables_processed.set(); }; - const size_t bunch_size = TABLES_PARALLEL_LOAD_BUNCH_SIZE; - size_t num_bunches = (total_tables + bunch_size - 1) / bunch_size; - - for (size_t i = 0; i < num_bunches; ++i) + for (const auto & filename : file_names) { - auto begin = file_names.begin() + i * bunch_size; - auto end = (i + 1 == num_bunches) - ? file_names.end() - : (file_names.begin() + (i + 1) * bunch_size); - - auto task = std::bind(task_function, begin, end); + auto task = std::bind(task_function, filename); if (thread_pool) thread_pool->schedule(task); @@ -198,7 +192,7 @@ void DatabaseOrdinary::loadTables( } if (thread_pool) - thread_pool->wait(); + all_tables_processed.wait(); /// After all tables was basically initialized, startup them. startupTables(thread_pool); @@ -212,47 +206,38 @@ void DatabaseOrdinary::startupTables(ThreadPool * thread_pool) AtomicStopwatch watch; std::atomic tables_processed {0}; size_t total_tables = tables.size(); + Poco::Event all_tables_processed; - auto task_function = [&](Tables::iterator begin, Tables::iterator end) + if (!total_tables) + return; + + auto task_function = [&](const StoragePtr & table) { - for (auto it = begin; it != end; ++it) + if ((tables_processed + 1) % PRINT_MESSAGE_EACH_N_TABLES == 0 + || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS)) { - if ((++tables_processed) % PRINT_MESSAGE_EACH_N_TABLES == 0 - || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS)) - { - LOG_INFO(log, std::fixed << std::setprecision(2) << tables_processed * 100.0 / total_tables << "%"); - watch.restart(); - } - - it->second->startup(); + LOG_INFO(log, std::fixed << std::setprecision(2) << tables_processed * 100.0 / total_tables << "%"); + watch.restart(); } + + table->startup(); + + if (++tables_processed == total_tables) + all_tables_processed.set(); }; - const size_t bunch_size = TABLES_PARALLEL_LOAD_BUNCH_SIZE; - size_t num_bunches = (total_tables + bunch_size - 1) / bunch_size; - - auto begin = tables.begin(); - for (size_t i = 0; i < num_bunches; ++i) + for (const auto & name_storage : tables) { - auto end = begin; - - if (i + 1 == num_bunches) - end = tables.end(); - else - std::advance(end, bunch_size); - - auto task = std::bind(task_function, begin, end); + auto task = std::bind(task_function, name_storage.second); if (thread_pool) thread_pool->schedule(task); else task(); - - begin = end; } if (thread_pool) - thread_pool->wait(); + all_tables_processed.wait(); } diff --git a/dbms/src/Dictionaries/DictionarySourceFactory.cpp b/dbms/src/Dictionaries/DictionarySourceFactory.cpp index 87ad425bab3..bb778fbce62 100644 --- a/dbms/src/Dictionaries/DictionarySourceFactory.cpp +++ b/dbms/src/Dictionaries/DictionarySourceFactory.cpp @@ -155,7 +155,8 @@ DictionarySourcePtr DictionarySourceFactory::create( else if ("odbc" == source_type) { #if USE_POCO_SQLODBC || USE_POCO_DATAODBC - BridgeHelperPtr bridge = std::make_shared>(config, context.getSettings().http_connection_timeout, config.getString(config_prefix + ".odbc.connection_string")); + const auto & global_config = context.getConfigRef(); + BridgeHelperPtr bridge = std::make_shared>(global_config, context.getSettings().http_connection_timeout, config.getString(config_prefix + ".odbc.connection_string")); return std::make_unique(dict_struct, config, config_prefix + ".odbc", sample_block, context, bridge); #else throw Exception{"Dictionary source of type `odbc` is disabled because poco library was built without ODBC support.", diff --git a/dbms/src/Dictionaries/ExternalQueryBuilder.cpp b/dbms/src/Dictionaries/ExternalQueryBuilder.cpp index 156c05932dd..bb420fdec8f 100644 --- a/dbms/src/Dictionaries/ExternalQueryBuilder.cpp +++ b/dbms/src/Dictionaries/ExternalQueryBuilder.cpp @@ -14,17 +14,28 @@ namespace DB namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; + extern const int LOGICAL_ERROR; } ExternalQueryBuilder::ExternalQueryBuilder( - const DictionaryStructure & dict_struct, - const std::string & db, - const std::string & table, - const std::string & where, - IdentifierQuotingStyle quoting_style) - : dict_struct(dict_struct), db(db), table(table), where(where), quoting_style(quoting_style) + const DictionaryStructure & dict_struct_, + const std::string & db_, + const std::string & table_, + const std::string & where_, + IdentifierQuotingStyle quoting_style_) + : dict_struct(dict_struct_), db(db_), where(where_), quoting_style(quoting_style_) { + if (auto pos = table_.find('.'); pos != std::string::npos) + { + schema = table_.substr(0, pos); + table = table_.substr(pos + 1); + } + else + { + schema = ""; + table = table_; + } } @@ -124,6 +135,11 @@ std::string ExternalQueryBuilder::composeLoadAllQuery() const writeQuoted(db, out); writeChar('.', out); } + if (!schema.empty()) + { + writeQuoted(schema, out); + writeChar('.', out); + } writeQuoted(table, out); if (!where.empty()) @@ -187,6 +203,12 @@ std::string ExternalQueryBuilder::composeLoadIdsQuery(const std::vector writeQuoted(db, out); writeChar('.', out); } + if (!schema.empty()) + { + writeQuoted(schema, out); + writeChar('.', out); + } + writeQuoted(table, out); writeString(" WHERE ", out); @@ -250,6 +272,12 @@ std::string ExternalQueryBuilder::composeLoadKeysQuery( writeQuoted(db, out); writeChar('.', out); } + if (!schema.empty()) + { + writeQuoted(schema, out); + writeChar('.', out); + } + writeQuoted(table, out); writeString(" WHERE ", out); diff --git a/dbms/src/Dictionaries/ExternalQueryBuilder.h b/dbms/src/Dictionaries/ExternalQueryBuilder.h index aedf7b86a56..b2f790e07d7 100644 --- a/dbms/src/Dictionaries/ExternalQueryBuilder.h +++ b/dbms/src/Dictionaries/ExternalQueryBuilder.h @@ -18,19 +18,20 @@ class WriteBuffer; struct ExternalQueryBuilder { const DictionaryStructure & dict_struct; - const std::string & db; - const std::string & table; + std::string db; + std::string table; + std::string schema; const std::string & where; IdentifierQuotingStyle quoting_style; ExternalQueryBuilder( - const DictionaryStructure & dict_struct, - const std::string & db, - const std::string & table, - const std::string & where, - IdentifierQuotingStyle quoting_style); + const DictionaryStructure & dict_struct_, + const std::string & db_, + const std::string & table_, + const std::string & where_, + IdentifierQuotingStyle quoting_style_); /** Generate a query to load all data. */ std::string composeLoadAllQuery() const; diff --git a/dbms/src/Dictionaries/ExternalResultDescription.cpp b/dbms/src/Dictionaries/ExternalResultDescription.cpp index 4ac883d1c39..a997cb4d07e 100644 --- a/dbms/src/Dictionaries/ExternalResultDescription.cpp +++ b/dbms/src/Dictionaries/ExternalResultDescription.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -20,57 +21,48 @@ void ExternalResultDescription::init(const Block & sample_block_) { sample_block = sample_block_; - const auto num_columns = sample_block.columns(); - types.reserve(num_columns); - names.reserve(num_columns); - sample_columns.reserve(num_columns); + types.reserve(sample_block.columns()); - for (const auto idx : ext::range(0, num_columns)) + for (auto & elem : sample_block) { - const auto & column = sample_block.safeGetByPosition(idx); - const auto type = column.type.get(); + /// If default value for column was not provided, use default from data type. + if (elem.column->empty()) + elem.column = elem.type->createColumnConstWithDefaultValue(1)->convertToFullColumnIfConst(); + + bool is_nullable = elem.type->isNullable(); + DataTypePtr type_not_nullable = removeNullable(elem.type); + const IDataType * type = type_not_nullable.get(); if (typeid_cast(type)) - types.push_back(ValueType::UInt8); + types.emplace_back(ValueType::UInt8, is_nullable); else if (typeid_cast(type)) - types.push_back(ValueType::UInt16); + types.emplace_back(ValueType::UInt16, is_nullable); else if (typeid_cast(type)) - types.push_back(ValueType::UInt32); + types.emplace_back(ValueType::UInt32, is_nullable); else if (typeid_cast(type)) - types.push_back(ValueType::UInt64); + types.emplace_back(ValueType::UInt64, is_nullable); else if (typeid_cast(type)) - types.push_back(ValueType::Int8); + types.emplace_back(ValueType::Int8, is_nullable); else if (typeid_cast(type)) - types.push_back(ValueType::Int16); + types.emplace_back(ValueType::Int16, is_nullable); else if (typeid_cast(type)) - types.push_back(ValueType::Int32); + types.emplace_back(ValueType::Int32, is_nullable); else if (typeid_cast(type)) - types.push_back(ValueType::Int64); + types.emplace_back(ValueType::Int64, is_nullable); else if (typeid_cast(type)) - types.push_back(ValueType::Float32); + types.emplace_back(ValueType::Float32, is_nullable); else if (typeid_cast(type)) - types.push_back(ValueType::Float64); + types.emplace_back(ValueType::Float64, is_nullable); else if (typeid_cast(type)) - types.push_back(ValueType::String); + types.emplace_back(ValueType::String, is_nullable); else if (typeid_cast(type)) - types.push_back(ValueType::Date); + types.emplace_back(ValueType::Date, is_nullable); else if (typeid_cast(type)) - types.push_back(ValueType::DateTime); + types.emplace_back(ValueType::DateTime, is_nullable); else if (typeid_cast(type)) - types.push_back(ValueType::UUID); + types.emplace_back(ValueType::UUID, is_nullable); else throw Exception{"Unsupported type " + type->getName(), ErrorCodes::UNKNOWN_TYPE}; - - names.emplace_back(column.name); - sample_columns.emplace_back(column.column); - - /// If default value for column was not provided, use default from data type. - if (sample_columns.back()->empty()) - { - MutableColumnPtr mutable_column = (*std::move(sample_columns.back())).mutate(); - column.type->insertDefaultInto(*mutable_column); - sample_columns.back() = std::move(mutable_column); - } } } diff --git a/dbms/src/Dictionaries/ExternalResultDescription.h b/dbms/src/Dictionaries/ExternalResultDescription.h index ff9426ade57..452770fe9f6 100644 --- a/dbms/src/Dictionaries/ExternalResultDescription.h +++ b/dbms/src/Dictionaries/ExternalResultDescription.h @@ -25,13 +25,11 @@ struct ExternalResultDescription String, Date, DateTime, - UUID + UUID, }; Block sample_block; - std::vector types; - std::vector names; - Columns sample_columns; + std::vector> types; void init(const Block & sample_block_); }; diff --git a/dbms/src/Dictionaries/LibraryDictionarySource.cpp b/dbms/src/Dictionaries/LibraryDictionarySource.cpp index eaedfa71704..a0505ee79f1 100644 --- a/dbms/src/Dictionaries/LibraryDictionarySource.cpp +++ b/dbms/src/Dictionaries/LibraryDictionarySource.cpp @@ -94,9 +94,16 @@ namespace { const auto & field = columns_received->data[col_n].data[row_n]; if (!field.data) - continue; - const auto & size = field.size; - columns[row_n]->insertData(static_cast(field.data), size); + { + /// sample_block contains null_value (from config) inside corresponding column + const auto & col = sample_block.getByPosition(row_n); + columns[row_n]->insertFrom(*(col.column), 0); + } + else + { + const auto & size = field.size; + columns[row_n]->insertData(static_cast(field.data), size); + } } } diff --git a/dbms/src/Dictionaries/MongoDBBlockInputStream.cpp b/dbms/src/Dictionaries/MongoDBBlockInputStream.cpp index 8bd4a28bb6f..1f0f91a257f 100644 --- a/dbms/src/Dictionaries/MongoDBBlockInputStream.cpp +++ b/dbms/src/Dictionaries/MongoDBBlockInputStream.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -179,13 +180,22 @@ Block MongoDBBlockInputStream::readImpl() for (const auto idx : ext::range(0, size)) { - const auto & name = description.names[idx]; + const auto & name = description.sample_block.getByPosition(idx).name; const Poco::MongoDB::Element::Ptr value = document->get(name); if (value.isNull() || value->type() == Poco::MongoDB::ElementTraits::TypeId) - insertDefaultValue(*columns[idx], *description.sample_columns[idx]); + insertDefaultValue(*columns[idx], *description.sample_block.getByPosition(idx).column); else - insertValue(*columns[idx], description.types[idx], *value, name); + { + if (description.types[idx].second) + { + ColumnNullable & column_nullable = static_cast(*columns[idx]); + insertValue(column_nullable.getNestedColumn(), description.types[idx].first, *value, name); + column_nullable.getNullMapData().emplace_back(0); + } + else + insertValue(*columns[idx], description.types[idx].first, *value, name); + } } } diff --git a/dbms/src/Dictionaries/MongoDBBlockInputStream.h b/dbms/src/Dictionaries/MongoDBBlockInputStream.h index dba3cc9e9cf..3c964708c91 100644 --- a/dbms/src/Dictionaries/MongoDBBlockInputStream.h +++ b/dbms/src/Dictionaries/MongoDBBlockInputStream.h @@ -32,7 +32,7 @@ public: String getName() const override { return "MongoDB"; } - Block getHeader() const override { return description.sample_block; } + Block getHeader() const override { return description.sample_block.cloneEmpty(); } private: Block readImpl() override; diff --git a/dbms/src/Dictionaries/MySQLBlockInputStream.cpp b/dbms/src/Dictionaries/MySQLBlockInputStream.cpp index 250329eae2d..ecd6da11f4d 100644 --- a/dbms/src/Dictionaries/MySQLBlockInputStream.cpp +++ b/dbms/src/Dictionaries/MySQLBlockInputStream.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -82,9 +83,18 @@ Block MySQLBlockInputStream::readImpl() { const auto value = row[idx]; if (!value.isNull()) - insertValue(*columns[idx], description.types[idx], value); + { + if (description.types[idx].second) + { + ColumnNullable & column_nullable = static_cast(*columns[idx]); + insertValue(column_nullable.getNestedColumn(), description.types[idx].first, value); + column_nullable.getNullMapData().emplace_back(0); + } + else + insertValue(*columns[idx], description.types[idx].first, value); + } else - insertDefaultValue(*columns[idx], *description.sample_columns[idx]); + insertDefaultValue(*columns[idx], *description.sample_block.getByPosition(idx).column); } ++num_rows; diff --git a/dbms/src/Dictionaries/MySQLBlockInputStream.h b/dbms/src/Dictionaries/MySQLBlockInputStream.h index 9e760cd28f8..7e082fdc21d 100644 --- a/dbms/src/Dictionaries/MySQLBlockInputStream.h +++ b/dbms/src/Dictionaries/MySQLBlockInputStream.h @@ -21,7 +21,7 @@ public: String getName() const override { return "MySQL"; } - Block getHeader() const override { return description.sample_block; } + Block getHeader() const override { return description.sample_block.cloneEmpty(); } private: Block readImpl() override; diff --git a/dbms/src/Dictionaries/ODBCBlockInputStream.cpp b/dbms/src/Dictionaries/ODBCBlockInputStream.cpp index d22fd1b0e83..2bf1d0756eb 100644 --- a/dbms/src/Dictionaries/ODBCBlockInputStream.cpp +++ b/dbms/src/Dictionaries/ODBCBlockInputStream.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -91,9 +92,18 @@ Block ODBCBlockInputStream::readImpl() const Poco::Dynamic::Var & value = row[idx]; if (!value.isEmpty()) - insertValue(*columns[idx], description.types[idx], value); + { + if (description.types[idx].second) + { + ColumnNullable & column_nullable = static_cast(*columns[idx]); + insertValue(column_nullable.getNestedColumn(), description.types[idx].first, value); + column_nullable.getNullMapData().emplace_back(0); + } + else + insertValue(*columns[idx], description.types[idx].first, value); + } else - insertDefaultValue(*columns[idx], *description.sample_columns[idx]); + insertDefaultValue(*columns[idx], *description.sample_block.getByPosition(idx).column); } ++iterator; diff --git a/dbms/src/Dictionaries/ODBCBlockInputStream.h b/dbms/src/Dictionaries/ODBCBlockInputStream.h index 8634b59e00c..b9f5543c271 100644 --- a/dbms/src/Dictionaries/ODBCBlockInputStream.h +++ b/dbms/src/Dictionaries/ODBCBlockInputStream.h @@ -24,7 +24,7 @@ public: String getName() const override { return "ODBC"; } - Block getHeader() const override { return description.sample_block; } + Block getHeader() const override { return description.sample_block.cloneEmpty(); } private: Block readImpl() override; diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index d3ae5d86f76..27424f7355b 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -49,7 +49,7 @@ namespace ErrorCodes * but only by whole bytes. For dates and datetimes - the same as for numbers. * For example, hex(257) = '0101'. * unhex(string) - Returns a string, hex of which is equal to `string` with regard of case and discarding one leading zero. - * If such a string does not exist, could return arbitary implementation specific value. + * If such a string does not exist, could return arbitrary implementation specific value. * * bitmaskToArray(x) - Returns an array of powers of two in the binary form of x. For example, bitmaskToArray(50) = [2, 16, 32]. */ diff --git a/dbms/src/Functions/FunctionsComparison.h b/dbms/src/Functions/FunctionsComparison.h index 36132cbba4e..3f7f6f39008 100644 --- a/dbms/src/Functions/FunctionsComparison.h +++ b/dbms/src/Functions/FunctionsComparison.h @@ -725,7 +725,7 @@ private: return true; }; - if (!callOnBasicTypes(left_number, right_number, call)) + if (!callOnBasicTypes(left_number, right_number, call)) throw Exception("Wrong call for " + getName() + " with " + col_left.type->getName() + " and " + col_right.type->getName(), ErrorCodes::LOGICAL_ERROR); } diff --git a/dbms/src/Functions/FunctionsConditional.h b/dbms/src/Functions/FunctionsConditional.h index 573ae32b2a2..a469a6bb7b2 100644 --- a/dbms/src/Functions/FunctionsConditional.h +++ b/dbms/src/Functions/FunctionsConditional.h @@ -230,6 +230,11 @@ public: static FunctionPtr create(const Context &) { return std::make_shared(); } private: + template + static constexpr bool allow_arrays = + !IsDecimalNumber && !IsDecimalNumber && + !std::is_same_v && !std::is_same_v; + template static UInt32 decimalScale(Block & block [[maybe_unused]], const ColumnNumbers & arguments [[maybe_unused]]) { @@ -314,7 +319,7 @@ private: { if constexpr (std::is_same_v::Type>) return false; - else if constexpr (!IsDecimalNumber && !IsDecimalNumber) + else if constexpr (allow_arrays) { using ResultType = typename NumberTraits::ResultOfIf::Type; @@ -370,7 +375,7 @@ private: { if constexpr (std::is_same_v::Type>) return false; - else if constexpr (!IsDecimalNumber && !IsDecimalNumber) + else if constexpr (allow_arrays) { using ResultType = typename NumberTraits::ResultOfIf::Type; @@ -975,9 +980,10 @@ public: if (auto rigth_array = checkAndGetDataType(arg_else.type.get())) right_id = rigth_array->getNestedType()->getTypeId(); - bool executed_with_nums = callOnBasicTypes(left_id, right_id, call); + bool executed_with_nums = callOnBasicTypes(left_id, right_id, call); if (!( executed_with_nums + || executeTyped(cond_col, block, arguments, result, input_rows_count) || executeString(cond_col, block, arguments, result) || executeGenericArray(cond_col, block, arguments, result) || executeTuple(block, arguments, result, input_rows_count))) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 58e528a6ead..9335e5eee57 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -340,7 +340,7 @@ struct ConvertImplGenericToString ColumnString::Chars_t & data_to = col_to->getChars(); ColumnString::Offsets & offsets_to = col_to->getOffsets(); - data_to.resize(size * 2); /// Using coefficient 2 for initial size is arbitary. + data_to.resize(size * 2); /// Using coefficient 2 for initial size is arbitrary. offsets_to.resize(size); WriteBufferFromVector write_buffer(data_to); diff --git a/dbms/src/Functions/FunctionsHashing.h b/dbms/src/Functions/FunctionsHashing.h index 0bff3451711..9ea16bc09fe 100644 --- a/dbms/src/Functions/FunctionsHashing.h +++ b/dbms/src/Functions/FunctionsHashing.h @@ -565,7 +565,7 @@ public: vec_to.assign(rows, static_cast(0xe28dbde7fe22e41c)); } - /// The function supports arbitary number of arguments of arbitary types. + /// The function supports arbitrary number of arguments of arbitrary types. bool is_first_argument = true; for (size_t i = 0; i < arguments.size(); ++i) diff --git a/dbms/src/Functions/FunctionsMath.h b/dbms/src/Functions/FunctionsMath.h index 21404789ad3..5ab1bf899c2 100644 --- a/dbms/src/Functions/FunctionsMath.h +++ b/dbms/src/Functions/FunctionsMath.h @@ -162,7 +162,7 @@ private: return execute(block, col_vec, result); }; - if (!callOnBasicType(col.type->getTypeId(), call)) + if (!callOnBasicType(col.type->getTypeId(), call)) throw Exception{"Illegal column " + col.column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } @@ -385,7 +385,7 @@ private: TypeIndex left_index = col_left.type->getTypeId(); TypeIndex right_index = col_right.type->getTypeId(); - if (!callOnBasicTypes(left_index, right_index, call)) + if (!callOnBasicTypes(left_index, right_index, call)) throw Exception{"Illegal column " + col_left.column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } diff --git a/dbms/src/Functions/arrayIndex.h b/dbms/src/Functions/arrayIndex.h index 68589f56f3e..82e5f32fb67 100644 --- a/dbms/src/Functions/arrayIndex.h +++ b/dbms/src/Functions/arrayIndex.h @@ -397,7 +397,7 @@ struct ArrayIndexStringImpl } }; -/// Catch-all implementation for arrays of arbitary type. +/// Catch-all implementation for arrays of arbitrary type. /// To compare with constant value, create non-constant column with single element, /// and pass is_value_has_single_element_to_compare = true. template @@ -555,7 +555,7 @@ public: } }; -/// Catch-all implementation for arrays of arbitary type +/// Catch-all implementation for arrays of arbitrary type /// when the 2nd function argument is a NULL value. template struct ArrayIndexGenericNullImpl diff --git a/dbms/src/Functions/sleep.h b/dbms/src/Functions/sleep.h index 1dfdbf3b5f5..1a6c9024486 100644 --- a/dbms/src/Functions/sleep.h +++ b/dbms/src/Functions/sleep.h @@ -85,7 +85,7 @@ public: unsigned useconds = seconds * (variant == FunctionSleepVariant::PerBlock ? 1 : size) * 1e6; /// When sleeping, the query cannot be cancelled. For abitily to cancel query, we limit sleep time. - if (useconds > 3000000) /// The choice is arbitary + if (useconds > 3000000) /// The choice is arbitrary throw Exception("The maximum sleep time is 3000000 microseconds. Requested: " + toString(useconds), ErrorCodes::TOO_SLOW); ::usleep(useconds); diff --git a/dbms/src/Functions/timeSlot.cpp b/dbms/src/Functions/timeSlot.cpp index f9763997b92..c734bed8203 100644 --- a/dbms/src/Functions/timeSlot.cpp +++ b/dbms/src/Functions/timeSlot.cpp @@ -8,7 +8,7 @@ namespace DB { -using FunctionTimeSlot = FunctionDateOrDateTimeToSomething; +using FunctionTimeSlot = FunctionDateOrDateTimeToSomething; void registerFunctionTimeSlot(FunctionFactory & factory) { diff --git a/dbms/src/Functions/timeSlots.cpp b/dbms/src/Functions/timeSlots.cpp index 16ade60cb50..07afc2894e1 100644 --- a/dbms/src/Functions/timeSlots.cpp +++ b/dbms/src/Functions/timeSlots.cpp @@ -121,17 +121,19 @@ public: size_t getNumberOfArguments() const override { return 2; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!WhichDataType(arguments[0]).isDateTime()) - throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be DateTime.", + if (!WhichDataType(arguments[0].type).isDateTime()) + throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() + ". Must be DateTime.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!WhichDataType(arguments[1]).isUInt32()) - throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ". Must be UInt32.", + if (!WhichDataType(arguments[1].type).isUInt32()) + throw Exception("Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() + ". Must be UInt32.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return std::make_shared(std::make_shared()); + /// If time zone is specified for source data type, attach it to the resulting type. + /// Note that there is no explicit time zone argument for this function (we specify 2 as an argument number with explicit time zone). + return std::make_shared(std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0))); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 82654c30bb9..4d99f731517 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -1132,6 +1132,9 @@ void Aggregator::convertToBlockImpl( if (data.empty()) return; + if (key_columns.size() != params.keys_size) + throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; + if (final) convertToBlockImplFinal(method, data, key_columns, final_aggregate_columns); else @@ -1151,7 +1154,7 @@ void NO_INLINE Aggregator::convertToBlockImplFinal( { for (const auto & value : data) { - method.insertKeyIntoColumns(value, key_columns, params.keys_size, key_sizes); + method.insertKeyIntoColumns(value, key_columns, key_sizes); for (size_t i = 0; i < params.aggregates_size; ++i) aggregate_functions[i]->insertResultInto( @@ -1169,10 +1172,9 @@ void NO_INLINE Aggregator::convertToBlockImplNotFinal( MutableColumns & key_columns, AggregateColumnsData & aggregate_columns) const { - for (auto & value : data) { - method.insertKeyIntoColumns(value, key_columns, params.keys_size, key_sizes); + method.insertKeyIntoColumns(value, key_columns, key_sizes); /// reserved, so push_back does not throw exceptions for (size_t i = 0; i < params.aggregates_size; ++i) @@ -2085,7 +2087,7 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV /** `minus one` means the absence of information about the bucket * - in the case of single-level aggregation, as well as for blocks with "overflowing" values. - * If there is at least one block with a bucket number greater than zero, then there was a two-level aggregation. + * If there is at least one block with a bucket number greater or equal than zero, then there was a two-level aggregation. */ auto max_bucket = bucket_to_blocks.rbegin()->first; size_t has_two_level = max_bucket >= 0; diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index f32c0e9940f..1d1a86e8a9e 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -166,7 +166,7 @@ struct AggregationMethodOneNumber /** Insert the key from the hash table into columns. */ - static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, size_t /*keys_size*/, const Sizes & /*key_sizes*/) + static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes & /*key_sizes*/) { static_cast *>(key_columns[0].get())->insertData(reinterpret_cast(&value.first), sizeof(value.first)); } @@ -243,7 +243,7 @@ struct AggregationMethodString return StringRef(value.first.data, value.first.size); } - static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, size_t, const Sizes &) + static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes &) { key_columns[0]->insertData(value.first.data, value.first.size); } @@ -312,7 +312,7 @@ struct AggregationMethodFixedString return StringRef(value.first.data, value.first.size); } - static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, size_t, const Sizes &) + static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes &) { key_columns[0]->insertData(value.first.data, value.first.size); } @@ -580,7 +580,7 @@ struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod static const bool no_consecutive_keys_optimization = true; static const bool low_cardinality_optimization = true; - static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, size_t /*keys_size*/, const Sizes & /*key_sizes*/) + static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes & /*key_sizes*/) { auto ref = Base::getValueRef(value); static_cast(key_columns[0].get())->insertData(ref.data, ref.size); @@ -783,8 +783,10 @@ struct AggregationMethodKeysFixed static const bool no_consecutive_keys_optimization = false; static const bool low_cardinality_optimization = false; - static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, size_t keys_size, const Sizes & key_sizes) + static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes & key_sizes) { + size_t keys_size = key_columns.size(); + static constexpr auto bitmap_size = has_nullable_keys ? std::tuple_size>::value : 0; /// In any hash key value, column values to be read start just after the bitmap, if it exists. size_t pos = bitmap_size; @@ -891,10 +893,10 @@ struct AggregationMethodSerialized static const bool no_consecutive_keys_optimization = true; static const bool low_cardinality_optimization = false; - static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, size_t keys_size, const Sizes &) + static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes &) { auto pos = value.first.data; - for (size_t i = 0; i < keys_size; ++i) + for (size_t i = 0; i < key_columns.size(); ++i) pos = key_columns[i]->deserializeAndInsertFromArena(pos); } @@ -1284,10 +1286,10 @@ public: Block intermediate_header; /// What to count. - ColumnNumbers keys; - AggregateDescriptions aggregates; - size_t keys_size; - size_t aggregates_size; + const ColumnNumbers keys; + const AggregateDescriptions aggregates; + const size_t keys_size; + const size_t aggregates_size; /// The settings of approximate calculation of GROUP BY. const bool overflow_row; /// Do we need to put into AggregatedDataVariants::without_key aggregates for keys that are not in max_rows_to_group_by. @@ -1344,9 +1346,6 @@ public: { intermediate_header = intermediate_header_; } - - /// Calculate the column numbers in `keys` and `aggregates`. - void calculateColumnNumbers(const Block & block); }; Aggregator(const Params & params_); diff --git a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp new file mode 100644 index 00000000000..4716ac09517 --- /dev/null +++ b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -0,0 +1,130 @@ +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_RESULT_OF_SCALAR_SUBQUERY; + extern const int TOO_MANY_ROWS; +} + + +static ASTPtr addTypeConversion(std::unique_ptr && ast, const String & type_name) +{ + auto func = std::make_shared(); + ASTPtr res = func; + func->alias = ast->alias; + func->prefer_alias_to_column_name = ast->prefer_alias_to_column_name; + ast->alias.clear(); + func->name = "CAST"; + auto exp_list = std::make_shared(); + func->arguments = exp_list; + func->children.push_back(func->arguments); + exp_list->children.emplace_back(ast.release()); + exp_list->children.emplace_back(std::make_shared(type_name)); + return res; +} + +void ExecuteScalarSubqueriesVisitor::visit(ASTSubquery * subquery, ASTPtr & ast, const DumpASTNode &) const +{ + Context subquery_context = context; + Settings subquery_settings = context.getSettings(); + subquery_settings.max_result_rows = 1; + subquery_settings.extremes = 0; + subquery_context.setSettings(subquery_settings); + + ASTPtr subquery_select = subquery->children.at(0); + BlockIO res = InterpreterSelectWithUnionQuery( + subquery_select, subquery_context, {}, QueryProcessingStage::Complete, subquery_depth + 1).execute(); + + Block block; + try + { + block = res.in->read(); + + if (!block) + { + /// Interpret subquery with empty result as Null literal + auto ast_new = std::make_unique(Null()); + ast_new->setAlias(ast->tryGetAlias()); + ast = std::move(ast_new); + return; + } + + if (block.rows() != 1 || res.in->read()) + throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); + } + catch (const Exception & e) + { + if (e.code() == ErrorCodes::TOO_MANY_ROWS) + throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); + else + throw; + } + + size_t columns = block.columns(); + if (columns == 1) + { + auto lit = std::make_unique((*block.safeGetByPosition(0).column)[0]); + lit->alias = subquery->alias; + lit->prefer_alias_to_column_name = subquery->prefer_alias_to_column_name; + ast = addTypeConversion(std::move(lit), block.safeGetByPosition(0).type->getName()); + } + else + { + auto tuple = std::make_shared(); + tuple->alias = subquery->alias; + ast = tuple; + tuple->name = "tuple"; + auto exp_list = std::make_shared(); + tuple->arguments = exp_list; + tuple->children.push_back(tuple->arguments); + + exp_list->children.resize(columns); + for (size_t i = 0; i < columns; ++i) + { + exp_list->children[i] = addTypeConversion( + std::make_unique((*block.safeGetByPosition(i).column)[0]), + block.safeGetByPosition(i).type->getName()); + } + } +} + + +void ExecuteScalarSubqueriesVisitor::visit(ASTTableExpression *, ASTPtr &, const DumpASTNode &) const +{ + /// Don't descend into subqueries in FROM section. +} + +void ExecuteScalarSubqueriesVisitor::visit(ASTFunction * func, ASTPtr & ast, const DumpASTNode &) const +{ + /// Don't descend into subqueries in arguments of IN operator. + /// But if an argument is not subquery, than deeper may be scalar subqueries and we need to descend in them. + + if (functionIsInOrGlobalInOperator(func->name)) + { + for (auto & child : ast->children) + { + if (child != func->arguments) + visit(child); + else + for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i) + if (i != 1 || !typeid_cast(func->arguments->children[i].get())) + visit(func->arguments->children[i]); + } + } + else + visitChildren(ast); +} + +} diff --git a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.h b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.h new file mode 100644 index 00000000000..9688ac253cc --- /dev/null +++ b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.h @@ -0,0 +1,79 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class Context; +class ASTSubquery; +class ASTFunction; +struct ASTTableExpression; + + +/** Replace subqueries that return exactly one row + * ("scalar" subqueries) to the corresponding constants. + * + * If the subquery returns more than one column, it is replaced by a tuple of constants. + * + * Features + * + * A replacement occurs during query analysis, and not during the main runtime. + * This means that the progress indicator will not work during the execution of these requests, + * and also such queries can not be aborted. + * + * But the query result can be used for the index in the table. + * + * Scalar subqueries are executed on the request-initializer server. + * The request is sent to remote servers with already substituted constants. + */ +class ExecuteScalarSubqueriesVisitor +{ +public: + ExecuteScalarSubqueriesVisitor(const Context & context_, size_t subquery_depth_, std::ostream * ostr_ = nullptr) + : context(context_), + subquery_depth(subquery_depth_), + visit_depth(0), + ostr(ostr_) + {} + + void visit(ASTPtr & ast) const + { + DumpASTNode dump(*ast, ostr, visit_depth, "executeScalarSubqueries"); + + if (!tryVisit(ast, dump) && + !tryVisit(ast, dump) && + !tryVisit(ast, dump)) + visitChildren(ast); + } + +private: + const Context & context; + size_t subquery_depth; + mutable size_t visit_depth; + std::ostream * ostr; + + void visit(ASTSubquery * subquery, ASTPtr & ast, const DumpASTNode & dump) const; + void visit(ASTFunction * func, ASTPtr & ast, const DumpASTNode &) const; + void visit(ASTTableExpression *, ASTPtr &, const DumpASTNode &) const; + + void visitChildren(ASTPtr & ast) const + { + for (auto & child : ast->children) + visit(child); + } + + template + bool tryVisit(ASTPtr & ast, const DumpASTNode & dump) const + { + if (T * t = typeid_cast(ast.get())) + { + visit(t, ast, dump); + return true; + } + return false; + } +}; + +} diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 7285fabb26d..7aa946cad39 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -37,6 +37,7 @@ #include #include #include +#include #include #include @@ -86,8 +87,6 @@ namespace ErrorCodes extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS; extern const int UNKNOWN_IDENTIFIER; extern const int CYCLIC_ALIASES; - extern const int INCORRECT_RESULT_OF_SCALAR_SUBQUERY; - extern const int TOO_MANY_ROWS; extern const int NOT_FOUND_COLUMN_IN_BLOCK; extern const int INCORRECT_ELEMENT_OF_SET; extern const int ALIAS_REQUIRED; @@ -150,16 +149,6 @@ const std::unordered_set possibly_injective_function_names namespace { -bool functionIsInOperator(const String & name) -{ - return name == "in" || name == "notIn"; -} - -bool functionIsInOrGlobalInOperator(const String & name) -{ - return name == "in" || name == "notIn" || name == "globalIn" || name == "globalNotIn"; -} - void removeDuplicateColumns(NamesAndTypesList & columns) { std::set names; @@ -904,8 +893,13 @@ void ExpressionAnalyzer::addAliasColumns() void ExpressionAnalyzer::executeScalarSubqueries() { + LogAST log; + if (!select_query) - executeScalarSubqueriesImpl(query); + { + ExecuteScalarSubqueriesVisitor execute_scalar_subqueries_visitor(context, subquery_depth, log.stream()); + execute_scalar_subqueries_visitor.visit(query); + } else { for (auto & child : query->children) @@ -914,143 +908,14 @@ void ExpressionAnalyzer::executeScalarSubqueries() if (!typeid_cast(child.get()) && !typeid_cast(child.get())) { - executeScalarSubqueriesImpl(child); + ExecuteScalarSubqueriesVisitor execute_scalar_subqueries_visitor(context, subquery_depth, log.stream()); + execute_scalar_subqueries_visitor.visit(child); } } } } -static ASTPtr addTypeConversion(std::unique_ptr && ast, const String & type_name) -{ - auto func = std::make_shared(); - ASTPtr res = func; - func->alias = ast->alias; - func->prefer_alias_to_column_name = ast->prefer_alias_to_column_name; - ast->alias.clear(); - func->name = "CAST"; - auto exp_list = std::make_shared(); - func->arguments = exp_list; - func->children.push_back(func->arguments); - exp_list->children.emplace_back(ast.release()); - exp_list->children.emplace_back(std::make_shared(type_name)); - return res; -} - - -void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast) -{ - /** Replace subqueries that return exactly one row - * ("scalar" subqueries) to the corresponding constants. - * - * If the subquery returns more than one column, it is replaced by a tuple of constants. - * - * Features - * - * A replacement occurs during query analysis, and not during the main runtime. - * This means that the progress indicator will not work during the execution of these requests, - * and also such queries can not be aborted. - * - * But the query result can be used for the index in the table. - * - * Scalar subqueries are executed on the request-initializer server. - * The request is sent to remote servers with already substituted constants. - */ - - if (ASTSubquery * subquery = typeid_cast(ast.get())) - { - Context subquery_context = context; - Settings subquery_settings = context.getSettings(); - subquery_settings.max_result_rows = 1; - subquery_settings.extremes = 0; - subquery_context.setSettings(subquery_settings); - - ASTPtr subquery_select = subquery->children.at(0); - BlockIO res = InterpreterSelectWithUnionQuery(subquery_select, subquery_context, {}, QueryProcessingStage::Complete, subquery_depth + 1).execute(); - - Block block; - try - { - block = res.in->read(); - - if (!block) - { - /// Interpret subquery with empty result as Null literal - auto ast_new = std::make_unique(Null()); - ast_new->setAlias(ast->tryGetAlias()); - ast = std::move(ast_new); - return; - } - - if (block.rows() != 1 || res.in->read()) - throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); - } - catch (const Exception & e) - { - if (e.code() == ErrorCodes::TOO_MANY_ROWS) - throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); - else - throw; - } - - size_t columns = block.columns(); - if (columns == 1) - { - auto lit = std::make_unique((*block.safeGetByPosition(0).column)[0]); - lit->alias = subquery->alias; - lit->prefer_alias_to_column_name = subquery->prefer_alias_to_column_name; - ast = addTypeConversion(std::move(lit), block.safeGetByPosition(0).type->getName()); - } - else - { - auto tuple = std::make_shared(); - tuple->alias = subquery->alias; - ast = tuple; - tuple->name = "tuple"; - auto exp_list = std::make_shared(); - tuple->arguments = exp_list; - tuple->children.push_back(tuple->arguments); - - exp_list->children.resize(columns); - for (size_t i = 0; i < columns; ++i) - { - exp_list->children[i] = addTypeConversion( - std::make_unique((*block.safeGetByPosition(i).column)[0]), - block.safeGetByPosition(i).type->getName()); - } - } - } - else - { - /** Don't descend into subqueries in FROM section. - */ - if (!typeid_cast(ast.get())) - { - /** Don't descend into subqueries in arguments of IN operator. - * But if an argument is not subquery, than deeper may be scalar subqueries and we need to descend in them. - */ - ASTFunction * func = typeid_cast(ast.get()); - - if (func && functionIsInOrGlobalInOperator(func->name)) - { - for (auto & child : ast->children) - { - if (child != func->arguments) - executeScalarSubqueriesImpl(child); - else - for (size_t i = 0, size = func->arguments->children.size(); i < size; ++i) - if (i != 1 || !typeid_cast(func->arguments->children[i].get())) - executeScalarSubqueriesImpl(func->arguments->children[i]); - } - } - else - for (auto & child : ast->children) - executeScalarSubqueriesImpl(child); - } - } -} - - void ExpressionAnalyzer::optimizeGroupBy() { if (!(select_query && select_query->group_expression_list)) diff --git a/dbms/src/Interpreters/ExpressionJIT.cpp b/dbms/src/Interpreters/ExpressionJIT.cpp index 71e4f6bce72..a88faf13ca4 100644 --- a/dbms/src/Interpreters/ExpressionJIT.cpp +++ b/dbms/src/Interpreters/ExpressionJIT.cpp @@ -322,8 +322,13 @@ class LLVMPreparedFunction : public PreparedFunctionImpl public: LLVMPreparedFunction(std::string name_, std::shared_ptr context) - : name(std::move(name_)), context(context), function(context->symbols.at(name)) - {} + : name(std::move(name_)), context(context) + { + auto it = context->symbols.find(name); + if (context->symbols.end() == it) + throw Exception("Cannot find symbol " + name + " in LLVMContext", ErrorCodes::LOGICAL_ERROR); + function = it->second; + } String getName() const override { return name; } @@ -510,6 +515,15 @@ LLVMFunction::LLVMFunction(const ExpressionActions::Actions & actions, std::shar compileFunctionToLLVMByteCode(context, *this); } +llvm::Value * LLVMFunction::compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const +{ + auto it = subexpressions.find(name); + if (subexpressions.end() == it) + throw Exception("Cannot find subexpression " + name + " in LLVMFunction", ErrorCodes::LOGICAL_ERROR); + return it->second(builder, values); +} + + PreparedFunctionPtr LLVMFunction::prepare(const Block &, const ColumnNumbers &, size_t) const { return std::make_shared(name, context); } bool LLVMFunction::isDeterministic() const @@ -691,8 +705,6 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output static LLVMTargetInitializer initializer; auto dependents = getActionsDependents(actions, output_columns); - /// Initialize context as late as possible and only if needed - std::shared_ptr context; std::vector fused(actions.size()); for (size_t i = 0; i < actions.size(); ++i) { @@ -708,7 +720,7 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output auto hash_key = ExpressionActions::ActionsHash{}(fused[i]); { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); if (counter[hash_key]++ < min_count_to_compile) continue; } @@ -716,26 +728,24 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output std::shared_ptr fn; if (compilation_cache) { - /// Lock here, to be sure, that all functions will be compiled - std::lock_guard lock(mutex); - /// Don't use getOrSet here, because sometimes we need to initialize context - fn = compilation_cache->get(hash_key); - if (!fn) + std::tie(fn, std::ignore) = compilation_cache->getOrSet(hash_key, [&inlined_func=std::as_const(fused[i]), &sample_block] () { - if (!context) - context = std::make_shared(); Stopwatch watch; - fn = std::make_shared(fused[i], context, sample_block); + std::shared_ptr context = std::make_shared(); + auto result_fn = std::make_shared(inlined_func, context, sample_block); + size_t used_memory = context->compileAllFunctionsToNativeCode(); + ProfileEvents::increment(ProfileEvents::CompileExpressionsBytes, used_memory); ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds()); - compilation_cache->set(hash_key, fn); - } + return result_fn; + }); } else { - if (!context) - context = std::make_shared(); + std::shared_ptr context = std::make_shared(); Stopwatch watch; fn = std::make_shared(fused[i], context, sample_block); + size_t used_memory = context->compileAllFunctionsToNativeCode(); + ProfileEvents::increment(ProfileEvents::CompileExpressionsBytes, used_memory); ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds()); } @@ -751,20 +761,10 @@ void compileFunctions(ExpressionActions::Actions & actions, const Names & output fused[*dep].insert(fused[*dep].end(), fused[i].begin(), fused[i].end()); } - if (context) - { - /// Lock here, because other threads can get uncompilted functions from cache - std::lock_guard lock(mutex); - size_t used_memory = context->compileAllFunctionsToNativeCode(); - ProfileEvents::increment(ProfileEvents::CompileExpressionsBytes, used_memory); - } - for (size_t i = 0; i < actions.size(); ++i) { if (actions[i].type == ExpressionAction::APPLY_FUNCTION && actions[i].is_function_compiled) - { actions[i].function = actions[i].function_base->prepare({}, {}, 0); /// Arguments are not used for LLVMFunction. - } } } diff --git a/dbms/src/Interpreters/ExpressionJIT.h b/dbms/src/Interpreters/ExpressionJIT.h index 756ca5581ce..f5ec420f37c 100644 --- a/dbms/src/Interpreters/ExpressionJIT.h +++ b/dbms/src/Interpreters/ExpressionJIT.h @@ -30,7 +30,7 @@ public: bool isCompilable() const override { return true; } - llvm::Value * compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const override { return subexpressions.at(name)(builder, values); } + llvm::Value * compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const override; String getName() const override { return name; } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 71f39e9092d..ddac4a07611 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -417,6 +417,22 @@ InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpression res.subqueries_for_sets = query_analyzer->getSubqueriesForSets(); + /// Check that PREWHERE doesn't contain unusual actions. Unusual actions are that can change number of rows. + if (res.prewhere_info) + { + auto check_actions = [](const ExpressionActionsPtr & actions) + { + if (actions) + for (const auto & action : actions->getActions()) + if (action.type == ExpressionAction::Type::JOIN || action.type == ExpressionAction::Type::ARRAY_JOIN) + throw Exception("PREWHERE cannot contain ARRAY JOIN or JOIN action", ErrorCodes::ILLEGAL_PREWHERE); + }; + + check_actions(res.prewhere_info->prewhere_actions); + check_actions(res.prewhere_info->alias_actions); + check_actions(res.prewhere_info->remove_columns_actions); + } + return res; } @@ -675,10 +691,10 @@ static void getLimitLengthAndOffset(ASTSelectQuery & query, size_t & length, siz } } + void InterpreterSelectQuery::executeFetchColumns( QueryProcessingStage::Enum processing_stage, Pipeline & pipeline, const PrewhereInfoPtr & prewhere_info) { - const Settings & settings = context.getSettingsRef(); /// Actions to calculate ALIAS if required. @@ -806,7 +822,6 @@ void InterpreterSelectQuery::executeFetchColumns( } } - /// Limitation on the number of columns to read. /// It's not applied in 'only_analyze' mode, because the query could be analyzed without removal of unnecessary columns. if (!only_analyze && settings.max_columns_to_read && required_columns.size() > settings.max_columns_to_read) diff --git a/dbms/src/Interpreters/QueryNormalizer.cpp b/dbms/src/Interpreters/QueryNormalizer.cpp index c7665528632..a1c6997368c 100644 --- a/dbms/src/Interpreters/QueryNormalizer.cpp +++ b/dbms/src/Interpreters/QueryNormalizer.cpp @@ -21,16 +21,6 @@ namespace ErrorCodes } -namespace -{ - -bool functionIsInOrGlobalInOperator(const String & name) -{ - return name == "in" || name == "notIn" || name == "globalIn" || name == "globalNotIn"; -} - -} - QueryNormalizer::QueryNormalizer(ASTPtr & query, const QueryNormalizer::Aliases & aliases, const Settings & settings, const Names & all_column_names, const TableNamesAndColumnNames & table_names_and_column_names) diff --git a/dbms/src/Interpreters/QueryNormalizer.h b/dbms/src/Interpreters/QueryNormalizer.h index ba0acd9ce0a..f5192dadd52 100644 --- a/dbms/src/Interpreters/QueryNormalizer.h +++ b/dbms/src/Interpreters/QueryNormalizer.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -7,6 +8,17 @@ namespace DB { +inline bool functionIsInOperator(const String & name) +{ + return name == "in" || name == "notIn"; +} + +inline bool functionIsInOrGlobalInOperator(const String & name) +{ + return functionIsInOperator(name) || name == "globalIn" || name == "globalNotIn"; +} + + using TableNameAndColumnNames = std::pair; using TableNamesAndColumnNames = std::vector; diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 1cc816e245e..6eb85e9c4df 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -277,7 +277,7 @@ struct Settings M(SettingBool, log_query_settings, true, "Log query settings into the query_log.") \ M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.") \ M(SettingString, send_logs_level, "none", "Send server text logs with specified minumum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'none'") \ - M(SettingBool, enable_optimize_predicate_expression, 1, "If it is set to true, optimize predicates to subqueries.") \ + M(SettingBool, enable_optimize_predicate_expression, 0, "If it is set to true, optimize predicates to subqueries.") \ \ M(SettingUInt64, low_cardinality_max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for LowCardinality type.") \ M(SettingBool, low_cardinality_use_single_dictionary_for_part, false, "LowCardinality type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.") \ diff --git a/dbms/src/Parsers/ASTLiteral.cpp b/dbms/src/Parsers/ASTLiteral.cpp index 18a16d0f630..44c7662ba43 100644 --- a/dbms/src/Parsers/ASTLiteral.cpp +++ b/dbms/src/Parsers/ASTLiteral.cpp @@ -12,7 +12,7 @@ void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const /// Special case for very large arrays. Instead of listing all elements, will use hash of them. /// (Otherwise column name will be too long, that will lead to significant slowdown of expression analysis.) if (value.getType() == Field::Types::Array - && value.get().size() > 100) /// 100 - just arbitary value. + && value.get().size() > 100) /// 100 - just arbitrary value. { SipHash hash; applyVisitor(FieldVisitorHash(hash), value); diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp index 82edb7bafba..303fce24c53 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes MergeTreeBaseBlockInputStream::MergeTreeBaseBlockInputStream( - MergeTreeData & storage, + const MergeTreeData & storage, const PrewhereInfoPtr & prewhere_info, UInt64 max_block_size_rows, UInt64 preferred_block_size_bytes, diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.h index e2763b08e19..510408622e8 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.h @@ -18,7 +18,7 @@ class MergeTreeBaseBlockInputStream : public IProfilingBlockInputStream { public: MergeTreeBaseBlockInputStream( - MergeTreeData & storage, + const MergeTreeData & storage, const PrewhereInfoPtr & prewhere_info, UInt64 max_block_size_rows, UInt64 preferred_block_size_bytes, @@ -47,7 +47,7 @@ protected: void injectVirtualColumns(Block & block) const; protected: - MergeTreeData & storage; + const MergeTreeData & storage; PrewhereInfoPtr prewhere_info; diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp index 01548d0971d..16492b83275 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp @@ -16,7 +16,7 @@ namespace ErrorCodes MergeTreeBlockInputStream::MergeTreeBlockInputStream( - MergeTreeData & storage_, + const MergeTreeData & storage_, const MergeTreeData::DataPartPtr & owned_data_part_, size_t max_block_size_rows_, size_t preferred_block_size_bytes_, diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.h index df63eaea5dd..7411a7ff04e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.h @@ -17,7 +17,7 @@ class MergeTreeBlockInputStream : public MergeTreeBaseBlockInputStream { public: MergeTreeBlockInputStream( - MergeTreeData & storage, + const MergeTreeData & storage, const MergeTreeData::DataPartPtr & owned_data_part, size_t max_block_size_rows, size_t preferred_block_size_bytes, diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 0a55bfa97e5..a31d12d932f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1223,7 +1223,9 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( * temporary column name ('converting_column_name') created in 'createConvertExpression' method * will have old name of shared offsets for arrays. */ - MergedColumnOnlyOutputStream out(*this, in.getHeader(), full_path + part->name + '/', true /* sync */, compression_settings, true /* skip_offsets */); + IMergedBlockOutputStream::WrittenOffsetColumns unused_written_offsets; + MergedColumnOnlyOutputStream out( + *this, in.getHeader(), full_path + part->name + '/', true /* sync */, compression_settings, true /* skip_offsets */, unused_written_offsets); in.readPrefix(); out.writePrefix(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 55b09f43bd4..5ad413f21f8 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -481,7 +481,7 @@ public: bool skip_sanity_checks); /// Should be called if part data is suspected to be corrupted. - void reportBrokenPart(const String & name) + void reportBrokenPart(const String & name) const { broken_part_callback(name); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 7c923a24968..2244fb28af2 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -22,8 +22,6 @@ #include #include #include -#include -#include #include #include #include @@ -750,7 +748,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor merge_entry->progress.store(column_sizes.keyColumnsProgress(sum_input_rows_exact, sum_input_rows_exact), std::memory_order_relaxed); BlockInputStreams column_part_streams(parts.size()); - NameSet offset_columns_written; auto it_name_and_type = gathering_columns.cbegin(); @@ -767,22 +764,20 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor + "). It is a bug.", ErrorCodes::LOGICAL_ERROR); CompressedReadBufferFromFile rows_sources_read_buf(rows_sources_file_path, 0, 0); + IMergedBlockOutputStream::WrittenOffsetColumns written_offset_columns; for (size_t column_num = 0, gathering_column_names_size = gathering_column_names.size(); column_num < gathering_column_names_size; ++column_num, ++it_name_and_type) { const String & column_name = it_name_and_type->name; - const DataTypePtr & column_type = it_name_and_type->type; - const String offset_column_name = Nested::extractTableName(column_name); - Names column_name_{column_name}; + Names column_names{column_name}; Float64 progress_before = merge_entry->progress.load(std::memory_order_relaxed); - bool offset_written = offset_columns_written.count(offset_column_name); for (size_t part_num = 0; part_num < parts.size(); ++part_num) { auto column_part_stream = std::make_shared( - data, parts[part_num], DEFAULT_MERGE_BLOCK_SIZE, 0, 0, column_name_, MarkRanges{MarkRange(0, parts[part_num]->marks_count)}, + data, parts[part_num], DEFAULT_MERGE_BLOCK_SIZE, 0, 0, column_names, MarkRanges{MarkRange(0, parts[part_num]->marks_count)}, false, nullptr, true, min_bytes_when_use_direct_io, DBMS_DEFAULT_BUFFER_SIZE, false, Names{}, 0, true); column_part_stream->setProgressCallback(MergeProgressCallbackVerticalStep( @@ -793,7 +788,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor rows_sources_read_buf.seek(0, 0); ColumnGathererStream column_gathered_stream(column_name, column_part_streams, rows_sources_read_buf); - MergedColumnOnlyOutputStream column_to(data, column_gathered_stream.getHeader(), new_part_tmp_path, false, compression_settings, offset_written); + MergedColumnOnlyOutputStream column_to( + data, column_gathered_stream.getHeader(), new_part_tmp_path, false, compression_settings, false, written_offset_columns); size_t column_elems_written = 0; column_to.writePrefix(); @@ -811,9 +807,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor ", but " + toString(rows_written) + " rows of PK columns", ErrorCodes::LOGICAL_ERROR); } - if (typeid_cast(column_type.get())) - offset_columns_written.emplace(offset_column_name); - /// NOTE: 'progress' is modified by single thread, but it may be concurrently read from MergeListElement::getInfo() (StorageSystemMerges). merge_entry->columns_written = merging_column_names.size() + column_num; @@ -971,7 +964,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor createHardLink(dir_it.path().toString(), destination.toString()); } - MergedColumnOnlyOutputStream out(data, in_header, new_part_tmp_path, /* sync = */ false, compression_settings, /* skip_offsets = */ false); + IMergedBlockOutputStream::WrittenOffsetColumns unused_written_offsets; + MergedColumnOnlyOutputStream out( + data, in_header, new_part_tmp_path, /* sync = */ false, compression_settings, /* skip_offsets = */ false, unused_written_offsets); in->readPrefix(); out.writePrefix(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.h b/dbms/src/Storages/MergeTree/MergeTreeDataPart.h index 03b6756bd81..d714cf1b037 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.h @@ -27,7 +27,7 @@ struct MergeTreeDataPart using Checksums = MergeTreeDataPartChecksums; using Checksum = MergeTreeDataPartChecksums::Checksum; - MergeTreeDataPart(MergeTreeData & storage_, const String & name_, const MergeTreePartInfo & info_) + MergeTreeDataPart(const MergeTreeData & storage_, const String & name_, const MergeTreePartInfo & info_) : storage(storage_), name(name_), info(info_) { } @@ -77,7 +77,7 @@ struct MergeTreeDataPart bool isEmpty() const { return rows_count == 0; } - MergeTreeData & storage; + const MergeTreeData & storage; String name; MergeTreePartInfo info; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 438a660af6e..20042b528fd 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -67,7 +67,7 @@ namespace ErrorCodes } -MergeTreeDataSelectExecutor::MergeTreeDataSelectExecutor(MergeTreeData & data_) +MergeTreeDataSelectExecutor::MergeTreeDataSelectExecutor(const MergeTreeData & data_) : data(data_), log(&Logger::get(data.getLogName() + " (SelectExecutor)")) { } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index f854480b356..589fc6e8583 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -17,7 +17,7 @@ class KeyCondition; class MergeTreeDataSelectExecutor { public: - MergeTreeDataSelectExecutor(MergeTreeData & data_); + MergeTreeDataSelectExecutor(const MergeTreeData & data_); /** When reading, selects a set of parts that covers the desired range of the index. * max_block_number_to_read - if not zero, do not read all the parts whose right border is greater than this threshold. @@ -40,7 +40,7 @@ public: Int64 max_block_number_to_read) const; private: - MergeTreeData & data; + const MergeTreeData & data; Logger * log; diff --git a/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp b/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp index e48ac859103..33bdb8d2aaa 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -16,7 +16,7 @@ namespace DB MergeTreeReadPool::MergeTreeReadPool( const size_t threads, const size_t sum_marks, const size_t min_marks_for_concurrent_read, - RangesInDataParts parts, MergeTreeData & data, const PrewhereInfoPtr & prewhere_info, + RangesInDataParts parts, const MergeTreeData & data, const PrewhereInfoPtr & prewhere_info, const bool check_columns, const Names & column_names, const BackoffSettings & backoff_settings, size_t preferred_block_size_bytes, const bool do_not_steal_tasks) diff --git a/dbms/src/Storages/MergeTree/MergeTreeReadPool.h b/dbms/src/Storages/MergeTree/MergeTreeReadPool.h index 8b19c6b2a1a..c0d10d01be4 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReadPool.h @@ -67,7 +67,7 @@ private: public: MergeTreeReadPool( const size_t threads, const size_t sum_marks, const size_t min_marks_for_concurrent_read, - RangesInDataParts parts, MergeTreeData & data, const PrewhereInfoPtr & prewhere_info, + RangesInDataParts parts, const MergeTreeData & data, const PrewhereInfoPtr & prewhere_info, const bool check_columns, const Names & column_names, const BackoffSettings & backoff_settings, size_t preferred_block_size_bytes, const bool do_not_steal_tasks = false); @@ -91,7 +91,7 @@ private: RangesInDataParts & parts, const size_t min_marks_for_concurrent_read); std::vector> per_part_columns_lock; - MergeTreeData & data; + const MergeTreeData & data; Names column_names; Names ordered_names; bool do_not_steal_tasks; diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index da65d38de7b..c9818c4ce7f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -36,7 +36,7 @@ MergeTreeReader::~MergeTreeReader() = default; MergeTreeReader::MergeTreeReader(const String & path, const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, bool save_marks_in_cache, - MergeTreeData & storage, const MarkRanges & all_mark_ranges, + const MergeTreeData & storage, const MarkRanges & all_mark_ranges, size_t aio_threshold, size_t max_read_buffer_size, const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type) diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.h b/dbms/src/Storages/MergeTree/MergeTreeReader.h index 32a71fe76da..8b165e607b1 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.h @@ -30,7 +30,7 @@ public: UncompressedCache * uncompressed_cache, MarkCache * mark_cache, bool save_marks_in_cache, - MergeTreeData & storage, const MarkRanges & all_mark_ranges, + const MergeTreeData & storage, const MarkRanges & all_mark_ranges, size_t aio_threshold, size_t max_read_buffer_size, const ValueSizeMap & avg_value_size_hints = ValueSizeMap{}, const ReadBufferFromFileBase::ProfileCallback & profile_callback = ReadBufferFromFileBase::ProfileCallback{}, @@ -111,7 +111,7 @@ private: /// If save_marks_in_cache is false, then, if marks are not in cache, we will load them but won't save in the cache, to avoid evicting other data. bool save_marks_in_cache; - MergeTreeData & storage; + const MergeTreeData & storage; MarkRanges all_mark_ranges; size_t aio_threshold; size_t max_read_buffer_size; diff --git a/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp index e9f1fa26cbc..b3e697b9b5a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp @@ -14,7 +14,7 @@ MergeTreeThreadBlockInputStream::MergeTreeThreadBlockInputStream( const size_t max_block_size_rows, size_t preferred_block_size_bytes, size_t preferred_max_column_in_block_size_bytes, - MergeTreeData & storage, + const MergeTreeData & storage, const bool use_uncompressed_cache, const PrewhereInfoPtr & prewhere_info, const Settings & settings, diff --git a/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.h index 06f76ccd20c..be47ba3cac9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.h @@ -21,7 +21,7 @@ public: const size_t max_block_size, size_t preferred_block_size_bytes, size_t preferred_max_column_in_block_size_bytes, - MergeTreeData & storage, + const MergeTreeData & storage, const bool use_uncompressed_cache, const PrewhereInfoPtr & prewhere_info, const Settings & settings, diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 98de7b0399c..c12ddc51381 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -19,6 +19,7 @@ constexpr auto MARKS_FILE_EXTENSION = ".mrk"; } + /// Implementation of IMergedBlockOutputStream. IMergedBlockOutputStream::IMergedBlockOutputStream( @@ -70,7 +71,7 @@ void IMergedBlockOutputStream::addStreams( IDataType::OutputStreamGetter IMergedBlockOutputStream::createStreamGetter( - const String & name, OffsetColumns & offset_columns, bool skip_offsets) + const String & name, WrittenOffsetColumns & offset_columns, bool skip_offsets) { return [&, skip_offsets] (const IDataType::SubstreamPath & substream_path) -> WriteBuffer * { @@ -93,7 +94,7 @@ void IMergedBlockOutputStream::writeData( const String & name, const IDataType & type, const IColumn & column, - OffsetColumns & offset_columns, + WrittenOffsetColumns & offset_columns, bool skip_offsets, IDataType::SerializeBinaryBulkStatePtr & serialization_state) { @@ -304,7 +305,7 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( IDataType::SerializeBinaryBulkSettings serialize_settings; serialize_settings.low_cardinality_max_dictionary_size = settings.low_cardinality_max_dictionary_size; serialize_settings.low_cardinality_use_single_dictionary_for_part = settings.low_cardinality_use_single_dictionary_for_part != 0; - OffsetColumns offset_columns; + WrittenOffsetColumns offset_columns; auto it = columns_list.begin(); for (size_t i = 0; i < columns_list.size(); ++i, ++it) { @@ -395,7 +396,7 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm size_t rows = block.rows(); /// The set of written offset columns so that you do not write shared offsets of nested structures columns several times - OffsetColumns offset_columns; + WrittenOffsetColumns offset_columns; auto sort_columns = storage.getPrimarySortColumns(); @@ -427,7 +428,7 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm if (serialization_states.empty()) { serialization_states.reserve(columns_list.size()); - OffsetColumns tmp_offset_columns; + WrittenOffsetColumns tmp_offset_columns; IDataType::SerializeBinaryBulkSettings settings; for (const auto & col : columns_list) @@ -501,12 +502,15 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm /// Implementation of MergedColumnOnlyOutputStream. MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( - MergeTreeData & storage_, const Block & header_, String part_path_, bool sync_, CompressionSettings compression_settings, bool skip_offsets_) + MergeTreeData & storage_, const Block & header_, String part_path_, bool sync_, + CompressionSettings compression_settings, bool skip_offsets_, + WrittenOffsetColumns & already_written_offset_columns) : IMergedBlockOutputStream( storage_, storage_.context.getSettings().min_compress_block_size, storage_.context.getSettings().max_compress_block_size, compression_settings, storage_.context.getSettings().min_bytes_to_use_direct_io), - header(header_), part_path(part_path_), sync(sync_), skip_offsets(skip_offsets_) + header(header_), part_path(part_path_), sync(sync_), skip_offsets(skip_offsets_), + already_written_offset_columns(already_written_offset_columns) { } @@ -517,7 +521,7 @@ void MergedColumnOnlyOutputStream::write(const Block & block) column_streams.clear(); serialization_states.clear(); serialization_states.reserve(block.columns()); - OffsetColumns tmp_offset_columns; + WrittenOffsetColumns tmp_offset_columns; IDataType::SerializeBinaryBulkSettings settings; for (size_t i = 0; i < block.columns(); ++i) @@ -535,7 +539,7 @@ void MergedColumnOnlyOutputStream::write(const Block & block) size_t rows = block.rows(); - OffsetColumns offset_columns; + WrittenOffsetColumns offset_columns = already_written_offset_columns; for (size_t i = 0; i < block.columns(); ++i) { const ColumnWithTypeAndName & column = block.safeGetByPosition(i); @@ -558,11 +562,11 @@ MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndG IDataType::SerializeBinaryBulkSettings serialize_settings; serialize_settings.low_cardinality_max_dictionary_size = settings.low_cardinality_max_dictionary_size; serialize_settings.low_cardinality_use_single_dictionary_for_part = settings.low_cardinality_use_single_dictionary_for_part != 0; - OffsetColumns offset_columns; - for (size_t i = 0; i < header.columns(); ++i) + + for (size_t i = 0, size = header.columns(); i < size; ++i) { - auto & column = header.safeGetByPosition(i); - serialize_settings.getter = createStreamGetter(column.name, offset_columns, skip_offsets); + auto & column = header.getByPosition(i); + serialize_settings.getter = createStreamGetter(column.name, already_written_offset_columns, skip_offsets); column.type->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[i]); } diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h index 947b982d7a5..a3f6a025c31 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -23,8 +23,9 @@ public: CompressionSettings compression_settings_, size_t aio_threshold_); + using WrittenOffsetColumns = std::set; + protected: - using OffsetColumns = std::set; using SerializationState = IDataType::SerializeBinaryBulkStatePtr; using SerializationStates = std::vector; @@ -67,10 +68,10 @@ protected: void addStreams(const String & path, const String & name, const IDataType & type, size_t estimated_size, bool skip_offsets); - IDataType::OutputStreamGetter createStreamGetter(const String & name, OffsetColumns & offset_columns, bool skip_offsets); + IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenOffsetColumns & offset_columns, bool skip_offsets); /// Write data of one column. - void writeData(const String & name, const IDataType & type, const IColumn & column, OffsetColumns & offset_columns, + void writeData(const String & name, const IDataType & type, const IColumn & column, WrittenOffsetColumns & offset_columns, bool skip_offsets, IDataType::SerializeBinaryBulkStatePtr & serialization_state); MergeTreeData & storage; @@ -150,13 +151,17 @@ private: }; -/// Writes only those columns that are in `block` +/// Writes only those columns that are in `header` class MergedColumnOnlyOutputStream final : public IMergedBlockOutputStream { public: /// skip_offsets: used when ALTERing columns if we know that array offsets are not altered. + /// Pass empty 'already_written_offset_columns' first time then and pass the same object to subsequent instances of MergedColumnOnlyOutputStream + /// if you want to serialize elements of Nested data structure in different instances of MergedColumnOnlyOutputStream. MergedColumnOnlyOutputStream( - MergeTreeData & storage_, const Block & header_, String part_path_, bool sync_, CompressionSettings compression_settings, bool skip_offsets_); + MergeTreeData & storage_, const Block & header_, String part_path_, bool sync_, + CompressionSettings compression_settings, bool skip_offsets_, + WrittenOffsetColumns & already_written_offset_columns); Block getHeader() const override { return header; } void write(const Block & block) override; @@ -171,6 +176,9 @@ private: bool initialized = false; bool sync; bool skip_offsets; + + /// To correctly write Nested elements column-by-column. + WrittenOffsetColumns & already_written_offset_columns; }; } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index ddb5d20599e..bee9f627d5e 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -15,6 +15,7 @@ #include #include +#include #include #include @@ -3826,6 +3827,11 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query optimize->database = leader_address.database; optimize->table = leader_address.table; } + else if (auto * drop = typeid_cast(new_query.get()); drop->kind == ASTDropQuery::Kind::Truncate) + { + drop->database = leader_address.database; + drop->table = leader_address.table; + } else throw Exception("Can't proxy this query. Unsupported query type", ErrorCodes::NOT_IMPLEMENTED); diff --git a/dbms/src/Storages/System/StorageSystemNumbers.h b/dbms/src/Storages/System/StorageSystemNumbers.h index a23137fa976..926e336e162 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.h +++ b/dbms/src/Storages/System/StorageSystemNumbers.h @@ -17,7 +17,7 @@ class Context; * You could also specify a limit (how many numbers to give). * If multithreaded is specified, numbers will be generated in several streams * (and result could be out of order). If both multithreaded and limit are specified, - * the table could give you not exactly 1..limit range, but some arbitary 'limit' numbers. + * the table could give you not exactly 1..limit range, but some arbitrary 'limit' numbers. */ class StorageSystemNumbers : public ext::shared_ptr_helper, public IStorage { diff --git a/dbms/tests/instructions/syntax.txt b/dbms/tests/instructions/syntax.txt new file mode 100644 index 00000000000..8ec0df48dcc --- /dev/null +++ b/dbms/tests/instructions/syntax.txt @@ -0,0 +1,6 @@ +# Quick syntax check (2 minutes on 16-core server) + +mkdir build && cd build +CC=clang-8 CXX=clang++-8 cmake -D ENABLE_EMBEDDED_COMPILER=0 -D CMAKE_BUILD_TYPE=Debug .. +ninja re2_st +time jq --raw-output '.[] | .command' compile_commands.json | grep -P -- ' -o [^ ]+\.o' | grep -v -P -- '-c .+/contrib/' | grep -vP '\.s$' | sed -r -e 's/ -o [^ ]+\.o/ -fsyntax-only/' | sort -R | xargs -I{} -P$(nproc) sh -c '{}' diff --git a/dbms/tests/integration/README.md b/dbms/tests/integration/README.md index 86808503180..822e37e0aa2 100644 --- a/dbms/tests/integration/README.md +++ b/dbms/tests/integration/README.md @@ -14,9 +14,9 @@ Don't use Docker from your system repository. * [pip](https://pypi.python.org/pypi/pip). To install: `sudo apt-get install python-pip` * [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` +* [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: `sudo -H pip install docker-compose docker dicttoxml kazoo PyMySQL psycopg2` -(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-kazoo` +(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-kazoo python-psycopg2` 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)) diff --git a/dbms/tests/integration/helpers/cluster.py b/dbms/tests/integration/helpers/cluster.py index 6c4fdcbc7f7..e124b15e010 100644 --- a/dbms/tests/integration/helpers/cluster.py +++ b/dbms/tests/integration/helpers/cluster.py @@ -13,6 +13,7 @@ import pymysql import xml.dom.minidom from kazoo.client import KazooClient from kazoo.exceptions import KazooException +import psycopg2 import docker from docker.errors import ContainerError @@ -79,6 +80,7 @@ class ClickHouseCluster: self.instances = {} self.with_zookeeper = False self.with_mysql = False + self.with_postgres = False self.with_kafka = False self.with_odbc_drivers = False @@ -92,7 +94,7 @@ class ClickHouseCluster: cmd += " client" 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, hostname=None, env_variables={}, image="ubuntu:14.04"): + 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, hostname=None, env_variables={}, image="ubuntu:14.04"): """Add an instance to the cluster. name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. @@ -127,6 +129,12 @@ class ClickHouseCluster: self.base_mysql_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_mysql.yml')] + if with_postgres and not self.with_postgres: + self.with_postgres = True + 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.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')] + if with_odbc_drivers and not self.with_odbc_drivers: self.with_odbc_drivers = True if not self.with_mysql: @@ -134,6 +142,12 @@ class ClickHouseCluster: self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_mysql.yml')]) self.base_mysql_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_mysql.yml')] + if not self.with_postgres: + self.with_postgres = True + 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.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')] + if with_kafka and not self.with_kafka: self.with_kafka = True @@ -168,6 +182,21 @@ class ClickHouseCluster: raise Exception("Cannot wait MySQL container") + def wait_postgres_to_start(self, timeout=60): + start = time.time() + while time.time() - start < timeout: + try: + conn_string = "host='localhost' user='postgres' password='mysecretpassword'" + conn = psycopg2.connect(conn_string) + conn.close() + print "Postgres Started" + return + except Exception as ex: + print "Can't connect to Postgres " + str(ex) + time.sleep(0.5) + + raise Exception("Cannot wait Postgres container") + def wait_zookeeper_to_start(self, timeout=60): start = time.time() while time.time() - start < timeout: @@ -204,20 +233,24 @@ class ClickHouseCluster: self.docker_client = docker.from_env(version=self.docker_api_version) if self.with_zookeeper and self.base_zookeeper_cmd: - subprocess_check_call(self.base_zookeeper_cmd + ['up', '-d', '--force-recreate', '--remove-orphans']) + subprocess_check_call(self.base_zookeeper_cmd + ['up', '-d', '--force-recreate']) for command in self.pre_zookeeper_commands: self.run_kazoo_commands_with_retries(command, repeats=5) self.wait_zookeeper_to_start(120) if self.with_mysql and self.base_mysql_cmd: - subprocess_check_call(self.base_mysql_cmd + ['up', '-d', '--force-recreate', '--remove-orphans']) + subprocess_check_call(self.base_mysql_cmd + ['up', '-d', '--force-recreate']) self.wait_mysql_to_start(120) + if self.with_postgres and self.base_postgres_cmd: + subprocess_check_call(self.base_postgres_cmd + ['up', '-d', '--force-recreate']) + self.wait_postgres_to_start(120) + if self.with_kafka and self.base_kafka_cmd: - subprocess_check_call(self.base_kafka_cmd + ['up', '-d', '--force-recreate', '--remove-orphans']) + subprocess_check_call(self.base_kafka_cmd + ['up', '-d', '--force-recreate']) self.kafka_docker_id = self.get_instance_docker_id('kafka1') - subprocess_check_call(self.base_cmd + ['up', '-d', '--force-recreate', '--remove-orphans']) + subprocess_check_call(self.base_cmd + ['up', '-d', '--force-recreate']) start_deadline = time.time() + 20.0 # seconds for instance in self.instances.itervalues(): @@ -281,7 +314,7 @@ services: - {logs_dir}:/var/log/clickhouse-server/ {odbc_ini_path} entrypoint: - - /usr/bin/clickhouse + - clickhouse - server - --config-file=/etc/clickhouse-server/config.xml - --log-file=/var/log/clickhouse-server/clickhouse-server.log @@ -444,8 +477,18 @@ class ClickHouseInstance: }, "PostgreSQL": { "DSN": "postgresql_odbc", + "Database": "postgres", + "UserName": "postgres", + "Password": "mysecretpassword", + "Port": "5432", + "Servername": "postgres1", + "Protocol": "9.3", + "ReadOnly": "No", + "RowVersioning": "No", + "ShowSystemTables": "No", "Driver": "/usr/lib/x86_64-linux-gnu/odbc/psqlodbca.so", "Setup": "/usr/lib/x86_64-linux-gnu/odbc/libodbcpsqlS.so", + "ConnSettings": "", } } else: diff --git a/dbms/tests/integration/helpers/docker_compose_postgres.yml b/dbms/tests/integration/helpers/docker_compose_postgres.yml new file mode 100644 index 00000000000..74049d9f0b3 --- /dev/null +++ b/dbms/tests/integration/helpers/docker_compose_postgres.yml @@ -0,0 +1,9 @@ +version: '2' +services: + postgres1: + image: postgres + restart: always + environment: + POSTGRES_PASSWORD: mysecretpassword + ports: + - 5432:5432 diff --git a/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml b/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml new file mode 100644 index 00000000000..1c293f66761 --- /dev/null +++ b/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml @@ -0,0 +1,38 @@ + + + postgres_odbc_hashed + + + clickhouse.test_table
+ DSN=postgresql_odbc; + postgres +
+ + + 5 + 5 + + + + + + + + column1 + + + + column1 + Int64 + 1 + + + + column2 + String + '' + + + +
+
diff --git a/dbms/tests/integration/test_odbc_interaction/test.py b/dbms/tests/integration/test_odbc_interaction/test.py index ec9a3ec32db..7b82d4a4281 100644 --- a/dbms/tests/integration/test_odbc_interaction/test.py +++ b/dbms/tests/integration/test_odbc_interaction/test.py @@ -3,12 +3,14 @@ import pytest import os import pymysql.cursors +import psycopg2 +from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT from helpers.cluster import ClickHouseCluster SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) -node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, image='alesapin/ubuntu_with_odbc:14.04', main_configs=['configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml', 'configs/dictionaries/sqlite3_odbc_cached_dictionary.xml']) +node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, image='alesapin/ubuntu_with_odbc:14.04', main_configs=['configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml', 'configs/dictionaries/sqlite3_odbc_cached_dictionary.xml', 'configs/dictionaries/postgres_odbc_hashed_dictionary.xml']) create_table_sql_template = """ CREATE TABLE `clickhouse`.`{}` ( @@ -31,24 +33,49 @@ def create_mysql_table(conn, table_name): with conn.cursor() as cursor: cursor.execute(create_table_sql_template.format(table_name)) +def get_postgres_conn(): + conn_string = "host='localhost' user='postgres' password='mysecretpassword'" + conn = psycopg2.connect(conn_string) + conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) + conn.autocommit = True + return conn + +def create_postgres_db(conn, name): + cursor = conn.cursor() + cursor.execute("CREATE SCHEMA {}".format(name)) + @pytest.fixture(scope="module") def started_cluster(): try: cluster.start() - sqlite_db = node1.odbc_drivers["SQLite3"]["Database"] + sqlite_db = node1.odbc_drivers["SQLite3"]["Database"] + print "sqlite data received" node1.exec_in_container(["bash", "-c", "echo 'CREATE TABLE t1(x INTEGER PRIMARY KEY ASC, y, z);' | sqlite3 {}".format(sqlite_db)], privileged=True, user='root') node1.exec_in_container(["bash", "-c", "echo 'CREATE TABLE t2(X INTEGER PRIMARY KEY ASC, Y, Z);' | sqlite3 {}".format(sqlite_db)], privileged=True, user='root') node1.exec_in_container(["bash", "-c", "echo 'CREATE TABLE t3(X INTEGER PRIMARY KEY ASC, Y, Z);' | sqlite3 {}".format(sqlite_db)], privileged=True, user='root') node1.exec_in_container(["bash", "-c", "echo 'CREATE TABLE t4(X INTEGER PRIMARY KEY ASC, Y, Z);' | sqlite3 {}".format(sqlite_db)], privileged=True, user='root') - conn = get_mysql_conn() + print "sqlite tables created" + mysql_conn = get_mysql_conn() + print "mysql connection received" ## create mysql db and table - create_mysql_db(conn, 'clickhouse') + create_mysql_db(mysql_conn, 'clickhouse') + print "mysql database created" + + postgres_conn = get_postgres_conn() + print "postgres connection received" + + create_postgres_db(postgres_conn, 'clickhouse') + print "postgres db created" + + cursor = postgres_conn.cursor() + cursor.execute("create table if not exists clickhouse.test_table (column1 int primary key, column2 varchar(40) not null)") yield cluster except Exception as ex: print(ex) + raise ex finally: cluster.shutdown() @@ -141,3 +168,11 @@ def test_sqlite_odbc_cached_dictionary(started_cluster): time.sleep(5) assert node1.query("select dictGetUInt8('sqlite3_odbc_cached', 'Z', toUInt64(1))") == "12\n" + +def test_postgres_odbc_hached_dictionary_with_schema(started_cluster): + conn = get_postgres_conn() + cursor = conn.cursor() + cursor.execute("insert into clickhouse.test_table values(1, 'hello'),(2, 'world')") + time.sleep(5) + assert node1.query("select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(1))") == "hello\n" + assert node1.query("select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(2))") == "world\n" diff --git a/dbms/tests/queries/0_stateless/00642_cast.reference b/dbms/tests/queries/0_stateless/00642_cast.reference index fcfb7325442..296f1351c16 100644 --- a/dbms/tests/queries/0_stateless/00642_cast.reference +++ b/dbms/tests/queries/0_stateless/00642_cast.reference @@ -6,6 +6,7 @@ hello hello hello hello +1970-01-01 00:00:01 CREATE TABLE test.cast ( x UInt8, e Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')) ENGINE = MergeTree ORDER BY e SETTINGS index_granularity = 8192 x UInt8 e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\') diff --git a/dbms/tests/queries/0_stateless/00642_cast.sql b/dbms/tests/queries/0_stateless/00642_cast.sql index 0e50d4c7e53..4f0c1e7f699 100644 --- a/dbms/tests/queries/0_stateless/00642_cast.sql +++ b/dbms/tests/queries/0_stateless/00642_cast.sql @@ -15,6 +15,8 @@ SELECT cast(1 AS Enum8( SELECT CAST(1, 'Enum8(\'hello\' = 1,\n\t\'world\' = 2)'); SELECT cast(1, 'Enum8(\'hello\' = 1,\n\t\'world\' = 2)'); +SELECT toTimeZone(CAST(1 AS TIMESTAMP), 'UTC'); + DROP TABLE IF EXISTS test.cast; CREATE TABLE test.cast ( diff --git a/dbms/tests/queries/0_stateless/00698_validate_array_sizes_for_nested_kshvakov.reference b/dbms/tests/queries/0_stateless/00698_validate_array_sizes_for_nested_kshvakov.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00698_validate_array_sizes_for_nested_kshvakov.sql b/dbms/tests/queries/0_stateless/00698_validate_array_sizes_for_nested_kshvakov.sql new file mode 100644 index 00000000000..0d15c7f9953 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00698_validate_array_sizes_for_nested_kshvakov.sql @@ -0,0 +1,17 @@ +SET send_logs_level = 'none'; + +DROP TABLE IF EXISTS test.Issue_2231_Invalid_Nested_Columns_Size; +CREATE TABLE test.Issue_2231_Invalid_Nested_Columns_Size ( + Date Date, + NestedColumn Nested( + ID Int32, + Count Int64 + ) +) Engine = MergeTree + PARTITION BY tuple() + ORDER BY Date; + +INSERT INTO test.Issue_2231_Invalid_Nested_Columns_Size VALUES (today(), [2,2], [1]), (today(), [2,2], [1, 1]); -- { serverError 190 } + +SELECT * FROM test.Issue_2231_Invalid_Nested_Columns_Size; +DROP TABLE test.Issue_2231_Invalid_Nested_Columns_Size; diff --git a/dbms/tests/queries/0_stateless/00729_prewhere_array_join.reference b/dbms/tests/queries/0_stateless/00729_prewhere_array_join.reference new file mode 100644 index 00000000000..60f4d20e07d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00729_prewhere_array_join.reference @@ -0,0 +1,8 @@ +background 1 +foreground 1 +heading 1 +image 1 +background 1 +foreground 1 +heading 1 +image 1 diff --git a/dbms/tests/queries/0_stateless/00729_prewhere_array_join.sql b/dbms/tests/queries/0_stateless/00729_prewhere_array_join.sql new file mode 100644 index 00000000000..8c0c86b1931 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00729_prewhere_array_join.sql @@ -0,0 +1,33 @@ +SET send_logs_level = 'none'; +USE test; + +drop table if exists t1; +create table t1 (id UInt64, val Array(String),nid UInt64, eDate Date)ENGINE = MergeTree(eDate, (id, eDate), 8192); + +insert into t1 (id,val,nid,eDate) values (1,['background','foreground','heading','image'],1,'2018-09-27'); +insert into t1 (id,val,nid,eDate) values (1,['background','foreground','heading','image'],1,'2018-09-27'); +insert into t1 (id,val,nid,eDate) values (2,['background','foreground','heading','image'],1,'2018-09-27'); +insert into t1 (id,val,nid,eDate) values (2,[],2,'2018-09-27'); +insert into t1 (id,val,nid,eDate) values (3,[],4,'2018-09-27'); +insert into t1 (id,val,nid,eDate) values (3,[],5,'2018-09-27'); +insert into t1 (id,val,nid,eDate) values (3,[],6,'2018-09-27'); +insert into t1 (id,val,nid,eDate) values (3,[],7,'2018-09-27'); +insert into t1 (id,val,nid,eDate) values (3,[],8,'2018-09-27'); + +select arrayJoin(val) as nameGroup6 from t1 prewhere notEmpty(toString(nameGroup6)) group by nameGroup6 order by nameGroup6; -- { serverError 182 } +select arrayJoin(val) as nameGroup6, countDistinct(nid) as rowids from t1 where notEmpty(toString(nameGroup6)) group by nameGroup6 order by nameGroup6; +select arrayJoin(val) as nameGroup6, countDistinct(nid) as rowids from t1 prewhere notEmpty(toString(nameGroup6)) group by nameGroup6 order by nameGroup6; -- { serverError 182 } + +drop table t1; +create table t1 (id UInt64, val Array(String),nid UInt64, eDate Date) ENGINE = MergeTree(eDate, (id, eDate), 8192); + +insert into t1 (id,val,nid,eDate) values (1,['background','foreground','heading','image'],1,'2018-09-27'); +insert into t1 (id,val,nid,eDate) values (1,['background','foreground','heading','image'],1,'2018-09-27'); +insert into t1 (id,val,nid,eDate) values (2,['background','foreground','heading','image'],1,'2018-09-27'); +insert into t1 (id,val,nid,eDate) values (2,[],2,'2018-09-27'); + +select arrayJoin(val) as nameGroup6 from t1 prewhere notEmpty(toString(nameGroup6)) group by nameGroup6 order by nameGroup6; -- { serverError 182 } +select arrayJoin(val) as nameGroup6, countDistinct(nid) as rowids from t1 where notEmpty(toString(nameGroup6)) group by nameGroup6 order by nameGroup6; +select arrayJoin(val) as nameGroup6, countDistinct(nid) as rowids from t1 prewhere notEmpty(toString(nameGroup6)) group by nameGroup6 order by nameGroup6; -- { serverError 182 } + +drop table t1; diff --git a/dbms/tests/queries/0_stateless/00732_decimal_summing_merge_tree.reference b/dbms/tests/queries/0_stateless/00732_decimal_summing_merge_tree.reference new file mode 100644 index 00000000000..1644f50993c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_decimal_summing_merge_tree.reference @@ -0,0 +1,2 @@ +2001-01-01 2.0000 0.00000000 -2.0000000000 +2001-01-01 0.0000 1.00000000 0.0000000000 diff --git a/dbms/tests/queries/0_stateless/00732_decimal_summing_merge_tree.sql b/dbms/tests/queries/0_stateless/00732_decimal_summing_merge_tree.sql new file mode 100644 index 00000000000..f560283434f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_decimal_summing_merge_tree.sql @@ -0,0 +1,27 @@ +CREATE DATABASE IF NOT EXISTS test; +DROP TABLE IF EXISTS test.decimal_sum; +CREATE TABLE test.decimal_sum +( + date Date, + sum32 Decimal32(4), + sum64 Decimal64(8), + sum128 Decimal128(10) +) Engine = SummingMergeTree(date, (date), 8192); + +INSERT INTO test.decimal_sum VALUES ('2001-01-01', 1, 1, -1); +INSERT INTO test.decimal_sum VALUES ('2001-01-01', 1, -1, -1); + +OPTIMIZE TABLE test.decimal_sum; +SELECT * FROM test.decimal_sum; + +INSERT INTO test.decimal_sum VALUES ('2001-01-01', -2, 1, 2); + +OPTIMIZE TABLE test.decimal_sum; +SELECT * FROM test.decimal_sum; + +INSERT INTO test.decimal_sum VALUES ('2001-01-01', 0, -1, 0); + +OPTIMIZE TABLE test.decimal_sum; +SELECT * FROM test.decimal_sum; + +drop table test.decimal_sum; diff --git a/dbms/tests/queries/0_stateless/00733_if_datetime.reference b/dbms/tests/queries/0_stateless/00733_if_datetime.reference new file mode 100644 index 00000000000..de6934f87b2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00733_if_datetime.reference @@ -0,0 +1,16 @@ +2001-02-03 04:05:06 +2000-01-01 00:00:00 +2001-02-03 04:05:06 +2000-01-01 00:00:00 +2001-02-03 04:05:06 +2000-01-01 00:00:00 +2001-02-03 04:05:06 +2000-01-01 00:00:00 +2001-02-03 +2000-01-01 +2001-02-03 +2000-01-01 +2001-02-03 +2000-01-01 +2001-02-03 +2000-01-01 diff --git a/dbms/tests/queries/0_stateless/00733_if_datetime.sql b/dbms/tests/queries/0_stateless/00733_if_datetime.sql new file mode 100644 index 00000000000..23ebc3d8a43 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00733_if_datetime.sql @@ -0,0 +1,9 @@ +SELECT number % 2 ? toDateTime('2000-01-01 00:00:00') : toDateTime('2001-02-03 04:05:06') FROM numbers(2); +SELECT number % 2 ? toDateTime('2000-01-01 00:00:00') : materialize(toDateTime('2001-02-03 04:05:06')) FROM numbers(2); +SELECT number % 2 ? materialize(toDateTime('2000-01-01 00:00:00')) : toDateTime('2001-02-03 04:05:06') FROM numbers(2); +SELECT number % 2 ? materialize(toDateTime('2000-01-01 00:00:00')) : materialize(toDateTime('2001-02-03 04:05:06')) FROM numbers(2); + +SELECT number % 2 ? toDate('2000-01-01') : toDate('2001-02-03') FROM numbers(2); +SELECT number % 2 ? toDate('2000-01-01') : materialize(toDate('2001-02-03')) FROM numbers(2); +SELECT number % 2 ? materialize(toDate('2000-01-01')) : toDate('2001-02-03') FROM numbers(2); +SELECT number % 2 ? materialize(toDate('2000-01-01')) : materialize(toDate('2001-02-03')) FROM numbers(2); diff --git a/dbms/tests/queries/0_stateless/00734_timeslot.reference b/dbms/tests/queries/0_stateless/00734_timeslot.reference new file mode 100644 index 00000000000..5acfa2b537c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00734_timeslot.reference @@ -0,0 +1,2 @@ +2000-01-02 03:00:00 +['2000-01-02 03:00:00','2000-01-02 03:30:00','2000-01-02 04:00:00','2000-01-02 04:30:00','2000-01-02 05:00:00','2000-01-02 05:30:00'] diff --git a/dbms/tests/queries/0_stateless/00734_timeslot.sql b/dbms/tests/queries/0_stateless/00734_timeslot.sql new file mode 100644 index 00000000000..7362074ccaf --- /dev/null +++ b/dbms/tests/queries/0_stateless/00734_timeslot.sql @@ -0,0 +1,2 @@ +SELECT timeSlot(toDateTime('2000-01-02 03:04:05', 'UTC')); +SELECT timeSlots(toDateTime('2000-01-02 03:04:05', 'UTC'), toUInt32(10000)); diff --git a/dbms/tests/queries/0_stateless/00735_conditional.reference b/dbms/tests/queries/0_stateless/00735_conditional.reference new file mode 100644 index 00000000000..bdea753dc4f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00735_conditional.reference @@ -0,0 +1,148 @@ +value vs value +0 1 1 Int8 Int8 Int8 +0 1 1 Int8 Int16 Int16 +0 1 1 Int8 Int32 Int32 +0 1 1 Int8 Int64 Int64 +0 1 1 Int8 UInt8 Int16 +0 1 1 Int8 UInt16 Int32 +0 1 1 Int8 UInt32 Int64 +0 1 1 Int8 Float32 Float32 +0 1 1 Int8 Float64 Float64 +0 1 1 Int16 Int8 Int16 +0 1 1 Int16 Int16 Int16 +0 1 1 Int16 Int32 Int32 +0 1 1 Int16 Int64 Int64 +0 1 1 Int16 UInt8 Int16 +0 1 1 Int16 UInt16 Int32 +0 1 1 Int16 UInt32 Int64 +0 1 1 Int16 Float32 Float32 +0 1 1 Int16 Float64 Float64 +0 1 1 Int32 Int8 Int32 +0 1 1 Int32 Int16 Int32 +0 1 1 Int32 Int32 Int32 +0 1 1 Int32 Int64 Int64 +0 1 1 Int32 UInt8 Int32 +0 1 1 Int32 UInt16 Int32 +0 1 1 Int32 UInt32 Int64 +0 1 1 Int32 Float32 Float64 +0 1 1 Int32 Float64 Float64 +0 1 1 Int64 Int8 Int64 +0 1 1 Int64 Int16 Int64 +0 1 1 Int64 Int32 Int64 +0 1 1 Int64 Int64 Int64 +0 1 1 Int64 UInt8 Int64 +0 1 1 Int64 UInt16 Int64 +0 1 1 Int64 UInt32 Int64 +0 1 1 UInt8 Int8 Int16 +0 1 1 UInt8 Int16 Int16 +0 1 1 UInt8 Int32 Int32 +0 1 1 UInt8 Int64 Int64 +0 1 1 UInt8 UInt8 UInt8 +0 1 1 UInt8 UInt16 UInt16 +0 1 1 UInt8 UInt32 UInt32 +0 1 1 UInt8 UInt64 UInt64 +0 1 1 UInt8 Float32 Float32 +0 1 1 UInt8 Float64 Float64 +0 1 1 UInt16 Int8 Int32 +0 1 1 UInt16 Int16 Int32 +0 1 1 UInt16 Int32 Int32 +0 1 1 UInt16 Int64 Int64 +0 1 1 UInt16 UInt8 UInt16 +0 1 1 UInt16 UInt16 UInt16 +0 1 1 UInt16 UInt32 UInt32 +0 1 1 UInt16 UInt64 UInt64 +0 1 1 UInt16 Float32 Float32 +0 1 1 UInt16 Float64 Float64 +0 1 1 UInt32 Int8 Int64 +0 1 1 UInt32 Int16 Int64 +0 1 1 UInt32 Int32 Int64 +0 1 1 UInt32 Int64 Int64 +0 1 1 UInt32 UInt8 UInt32 +0 1 1 UInt32 UInt16 UInt32 +0 1 1 UInt32 UInt32 UInt32 +0 1 1 UInt32 UInt64 UInt64 +0 1 1 UInt32 Float32 Float64 +0 1 1 UInt32 Float64 Float64 +0 1 1 UInt64 UInt8 UInt64 +0 1 1 UInt64 UInt16 UInt64 +0 1 1 UInt64 UInt32 UInt64 +0 1 1 UInt64 UInt64 UInt64 +0000-00-00 1970-01-02 1970-01-02 Date Date Date +0000-00-00 1970-01-01 03:00:01 1970-01-01 03:00:01 Date DateTime DateTime +0000-00-00 00:00:00 1970-01-02 1970-01-01 03:00:01 DateTime Date DateTime +0000-00-00 00:00:00 1970-01-01 03:00:01 1970-01-01 03:00:01 DateTime DateTime DateTime +00000000-0000-0000-0000-000000000000 00000000-0000-0001-0000-000000000000 00000000-0000-0001-0000-000000000000 UUID UUID UUID +column vs value +0 1 1 Int8 Int8 Int8 +0 1 1 Int8 Int16 Int16 +0 1 1 Int8 Int32 Int32 +0 1 1 Int8 Int64 Int64 +0 1 1 Int8 UInt8 Int16 +0 1 1 Int8 UInt16 Int32 +0 1 1 Int8 UInt32 Int64 +0 1 1 Int8 Float32 Float32 +0 1 1 Int8 Float64 Float64 +0 1 1 Int16 Int8 Int16 +0 1 1 Int16 Int16 Int16 +0 1 1 Int16 Int32 Int32 +0 1 1 Int16 Int64 Int64 +0 1 1 Int16 UInt8 Int16 +0 1 1 Int16 UInt16 Int32 +0 1 1 Int16 UInt32 Int64 +0 1 1 Int16 Float32 Float32 +0 1 1 Int16 Float64 Float64 +0 1 1 Int32 Int8 Int32 +0 1 1 Int32 Int16 Int32 +0 1 1 Int32 Int32 Int32 +0 1 1 Int32 Int64 Int64 +0 1 1 Int32 UInt8 Int32 +0 1 1 Int32 UInt16 Int32 +0 1 1 Int32 UInt32 Int64 +0 1 1 Int32 Float32 Float64 +0 1 1 Int32 Float64 Float64 +0 1 1 Int64 Int8 Int64 +0 1 1 Int64 Int16 Int64 +0 1 1 Int64 Int32 Int64 +0 1 1 Int64 Int64 Int64 +0 1 1 Int64 UInt8 Int64 +0 1 1 Int64 UInt16 Int64 +0 1 1 Int64 UInt32 Int64 +0 1 1 UInt8 Int8 Int16 +0 1 1 UInt8 Int16 Int16 +0 1 1 UInt8 Int32 Int32 +0 1 1 UInt8 Int64 Int64 +0 1 1 UInt8 UInt8 UInt8 +0 1 1 UInt8 UInt16 UInt16 +0 1 1 UInt8 UInt32 UInt32 +0 1 1 UInt8 UInt64 UInt64 +0 1 1 UInt8 Float32 Float32 +0 1 1 UInt8 Float64 Float64 +0 1 1 UInt16 Int8 Int32 +0 1 1 UInt16 Int16 Int32 +0 1 1 UInt16 Int32 Int32 +0 1 1 UInt16 Int64 Int64 +0 1 1 UInt16 UInt8 UInt16 +0 1 1 UInt16 UInt16 UInt16 +0 1 1 UInt16 UInt32 UInt32 +0 1 1 UInt16 UInt64 UInt64 +0 1 1 UInt16 Float32 Float32 +0 1 1 UInt16 Float64 Float64 +0 1 1 UInt32 Int8 Int64 +0 1 1 UInt32 Int16 Int64 +0 1 1 UInt32 Int32 Int64 +0 1 1 UInt32 Int64 Int64 +0 1 1 UInt32 UInt8 UInt32 +0 1 1 UInt32 UInt16 UInt32 +0 1 1 UInt32 UInt32 UInt32 +0 1 1 UInt32 UInt64 UInt64 +0 1 1 UInt32 Float32 Float64 +0 1 1 UInt32 Float64 Float64 +0 1 1 UInt64 UInt8 UInt64 +0 1 1 UInt64 UInt16 UInt64 +0 1 1 UInt64 UInt32 UInt64 +0 1 1 UInt64 UInt64 UInt64 +0000-00-00 1970-01-02 1970-01-02 Date Date Date +0000-00-00 1970-01-01 03:00:01 1970-01-01 03:00:01 Date DateTime DateTime +0000-00-00 00:00:00 1970-01-02 1970-01-01 03:00:01 DateTime Date DateTime +0000-00-00 00:00:00 1970-01-01 03:00:01 1970-01-01 03:00:01 DateTime DateTime DateTime +00000000-0000-0000-0000-000000000000 00000000-0000-0001-0000-000000000000 00000000-0000-0001-0000-000000000000 UUID UUID UUID diff --git a/dbms/tests/queries/0_stateless/00735_conditional.sql b/dbms/tests/queries/0_stateless/00735_conditional.sql new file mode 100644 index 00000000000..aeb90dca7f7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00735_conditional.sql @@ -0,0 +1,379 @@ +SET send_logs_level = 'none'; + +SELECT 'value vs value'; + +SELECT toInt8(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt8(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt8(0) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt8(0) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt8(0) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt8(0) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt8(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt8(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt8(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt8(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt8(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt8(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt8(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt8(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT toInt8(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT toInt8(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } + +SELECT toInt16(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt16(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt16(0) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt16(0) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt16(0) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt16(0) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt16(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt16(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt16(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt16(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt16(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt16(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt16(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt16(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT toInt16(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT toInt16(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } + +SELECT toInt32(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt32(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt32(0) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt32(0) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt32(0) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt32(0) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt32(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt32(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt32(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt32(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt32(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt32(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt32(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt32(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT toInt32(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT toInt32(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } + +SELECT toInt64(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt64(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt64(0) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt64(0) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt64(0) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt64(0) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt64(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toInt64(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt64(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt64(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt64(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt64(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt64(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt64(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT toInt64(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT toInt64(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } + +SELECT toUInt8(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt8(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt8(0) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt8(0) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt8(0) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt8(0) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt8(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt8(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt8(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt8(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt8(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt8(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt8(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt8(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT toUInt8(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT toUInt8(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } + +SELECT toUInt16(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt16(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt16(0) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt16(0) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt16(0) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt16(0) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt16(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt16(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt16(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt16(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt16(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt16(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt16(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt16(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT toUInt16(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT toUInt16(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } + +SELECT toUInt32(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt32(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt32(0) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt32(0) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt32(0) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt32(0) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt32(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt32(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt32(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt32(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt32(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt32(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt32(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt32(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT toUInt32(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT toUInt32(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } + +SELECT toUInt64(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt64(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt64(0) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt64(0) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt64(0) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt64(0) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt64(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt64(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUInt64(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt64(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt64(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt64(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt64(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt64(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT toUInt64(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT toUInt64(0) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } + +SELECT toDate(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDate(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDate(0) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDate(0) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDate(0) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDate(0) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDate(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDate(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDate(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDate(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDate(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toDate(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toDate(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDate(0) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDate(0) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDate(0) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } + +SELECT toDateTime(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toDateTime(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toDateTime(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } + +SELECT toUUID(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUUID(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUUID(0) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUUID(0) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUUID(0) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUUID(0) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUUID(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUUID(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUUID(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUUID(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUUID(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUUID(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUUID(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toUUID(0) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUUID(0) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUUID(0) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } + +SELECT 'column vs value'; + +SELECT materialize(toInt8(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt8(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt8(0)) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt8(0)) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt8(0)) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt8(0)) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt8(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt8(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt8(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt8(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt8(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt8(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt8(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt8(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT materialize(toInt8(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT materialize(toInt8(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } + +SELECT materialize(toInt16(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt16(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt16(0)) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt16(0)) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt16(0)) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt16(0)) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt16(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt16(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt16(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt16(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt16(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt16(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt16(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt16(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT materialize(toInt16(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT materialize(toInt16(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } + +SELECT materialize(toInt32(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt32(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt32(0)) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt32(0)) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt32(0)) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt32(0)) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt32(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt32(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt32(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt32(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt32(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt32(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt32(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt32(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT materialize(toInt32(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT materialize(toInt32(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } + +SELECT materialize(toInt64(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt64(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt64(0)) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt64(0)) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt64(0)) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt64(0)) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt64(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toInt64(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt64(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt64(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt64(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt64(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt64(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt64(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT materialize(toInt64(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT materialize(toInt64(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } + +SELECT materialize(toUInt8(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt8(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt8(0)) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt8(0)) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt8(0)) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt8(0)) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt8(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt8(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt8(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt8(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt8(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt8(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt8(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt8(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT materialize(toUInt8(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT materialize(toUInt8(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } + +SELECT materialize(toUInt16(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt16(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt16(0)) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt16(0)) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt16(0)) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt16(0)) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt16(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt16(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt16(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt16(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt16(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt16(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt16(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt16(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT materialize(toUInt16(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT materialize(toUInt16(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } + +SELECT materialize(toUInt32(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt32(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt32(0)) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt32(0)) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt32(0)) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt32(0)) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt32(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt32(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt32(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt32(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt32(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt32(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt32(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt32(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT materialize(toUInt32(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT materialize(toUInt32(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } + +SELECT materialize(toUInt64(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt64(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt64(0)) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt64(0)) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt64(0)) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt64(0)) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt64(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt64(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUInt64(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt64(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt64(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt64(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt64(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt64(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT materialize(toUInt64(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } +SELECT materialize(toUInt64(0)) AS x, toDecimal128(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } + +SELECT materialize(toDate(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDate(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDate(0)) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDate(0)) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDate(0)) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDate(0)) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDate(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDate(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDate(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDate(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDate(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toDate(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toDate(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDate(0)) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDate(0)) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDate(0)) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } + +SELECT materialize(toDateTime(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0)) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0)) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0)) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0)) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toDateTime(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toDateTime(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0)) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0)) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0)) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } + +SELECT materialize(toUUID(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUUID(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUUID(0)) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUUID(0)) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUUID(0)) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUUID(0)) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUUID(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUUID(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUUID(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUUID(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUUID(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUUID(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUUID(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toUUID(0)) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUUID(0)) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUUID(0)) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } diff --git a/dbms/tests/queries/0_stateless/00735_or_expr_optimize_bug.reference b/dbms/tests/queries/0_stateless/00735_or_expr_optimize_bug.reference new file mode 100644 index 00000000000..ec635144f60 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00735_or_expr_optimize_bug.reference @@ -0,0 +1 @@ +9 diff --git a/dbms/tests/queries/0_stateless/00735_or_expr_optimize_bug.sql b/dbms/tests/queries/0_stateless/00735_or_expr_optimize_bug.sql new file mode 100644 index 00000000000..422f69878fd --- /dev/null +++ b/dbms/tests/queries/0_stateless/00735_or_expr_optimize_bug.sql @@ -0,0 +1,7 @@ +CREATE DATABASE IF NOT EXISTS test; +DROP TABLE IF EXISTS test.or_expr_bug; +CREATE TABLE test.or_expr_bug (a UInt64, b UInt64) ENGINE = Memory; + +INSERT INTO test.or_expr_bug VALUES(1,21),(1,22),(1,23),(2,21),(2,22),(2,23),(3,21),(3,22),(3,23); + +SELECT count(*) FROM test.or_expr_bug WHERE (a=1 OR a=2 OR a=3) AND (b=21 OR b=22 OR b=23); diff --git a/dbms/tests/queries/0_stateless/00736_disjunction_optimisation.reference b/dbms/tests/queries/0_stateless/00736_disjunction_optimisation.reference new file mode 100644 index 00000000000..afd698b425e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00736_disjunction_optimisation.reference @@ -0,0 +1,90 @@ +1 21 +1 22 +1 23 +2 21 +2 22 +2 23 +3 21 +3 22 +3 23 +1 21 +1 22 +1 23 +2 21 +2 22 +2 23 +3 21 +3 22 +3 23 +1 21 +1 22 +1 23 +2 21 +2 22 +2 23 +3 21 +3 22 +3 23 +1 1 21 1 1 1 +1 1 22 0 1 1 +1 1 23 0 0 1 +2 1 21 1 1 1 +2 1 22 0 1 1 +2 1 23 0 0 1 +3 1 21 1 1 1 +3 1 22 0 1 1 +3 1 23 0 0 1 +21 1 +22 1 +23 1 +21 1 +22 1 +23 1 +21 1 +22 1 +23 1 +1 21 +1 22 +1 23 +2 21 +2 22 +2 23 +3 21 +3 22 +3 23 +1 21 +1 22 +1 23 +2 21 +2 22 +2 23 +3 21 +3 22 +3 23 +1 21 +1 22 +1 23 +2 21 +2 22 +2 23 +3 21 +3 22 +3 23 +1 1 21 1 1 1 +1 1 22 0 1 1 +1 1 23 0 0 1 +2 1 21 1 1 1 +2 1 22 0 1 1 +2 1 23 0 0 1 +3 1 21 1 1 1 +3 1 22 0 1 1 +3 1 23 0 0 1 +21 1 +22 1 +23 1 +21 1 +22 1 +23 1 +21 1 +22 1 +23 1 diff --git a/dbms/tests/queries/0_stateless/00736_disjunction_optimisation.sql b/dbms/tests/queries/0_stateless/00736_disjunction_optimisation.sql new file mode 100644 index 00000000000..caf2af7a037 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00736_disjunction_optimisation.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS test.bug; +CREATE TABLE IF NOT EXISTS test.bug(k UInt64, s UInt64) ENGINE = Memory; +insert into test.bug values(1,21),(1,22),(1,23),(2,21),(2,22),(2,23),(3,21),(3,22),(3,23); + +set optimize_min_equality_disjunction_chain_length = 2; + +select * from test.bug; +select * from test.bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23); +select * from (select * from test.bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23); +select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from test.bug; +select s, (s=21 or s=22 or s=23) from test.bug; + +set optimize_min_equality_disjunction_chain_length = 3; + +select * from test.bug; +select * from test.bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23); +select * from (select * from test.bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23); +select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from test.bug; +select s, (s=21 or s=22 or s=23) from test.bug; + +DROP TABLE test.bug; diff --git a/dbms/tests/queries/0_stateless/00737_decimal_group_by.reference b/dbms/tests/queries/0_stateless/00737_decimal_group_by.reference new file mode 100644 index 00000000000..2f838f4bcdd --- /dev/null +++ b/dbms/tests/queries/0_stateless/00737_decimal_group_by.reference @@ -0,0 +1,11 @@ +1.10 +2.1000 +3.100000000000 +1.20 +2.2000 +3.200000000000 +1.30 +2.3000 +3.300000000000 +1 1.000000000000000000 10.000000000000000000 +1 1.000000000000000000 10.000000000000000000 diff --git a/dbms/tests/queries/0_stateless/00737_decimal_group_by.sql b/dbms/tests/queries/0_stateless/00737_decimal_group_by.sql new file mode 100644 index 00000000000..0904bb1d601 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00737_decimal_group_by.sql @@ -0,0 +1,26 @@ +select toDecimal32(1.1, 2) as x group by x; +select toDecimal64(2.1, 4) as x group by x; +select toDecimal128(3.1, 12) as x group by x; + +select materialize(toDecimal32(1.2, 2)) as x group by x; +select materialize(toDecimal64(2.2, 4)) as x group by x; +select materialize(toDecimal128(3.2, 12)) as x group by x; + +select x from (select toDecimal32(1.3, 2) x) group by x; +select x from (select toDecimal64(2.3, 4) x) group by x; +select x from (select toDecimal128(3.3, 12) x) group by x; + +DROP TABLE IF EXISTS test.decimal; +CREATE TABLE IF NOT EXISTS test.decimal +( + A UInt64, + B Decimal128(18), + C Decimal128(18) +) Engine = Memory; + +INSERT INTO test.decimal VALUES (1,1,1), (1,1,2), (1,1,3), (1,1,4); + +SELECT A, toString(B) AS B_str, toString(SUM(C)) AS c_str FROM test.decimal GROUP BY A, B_str; +SELECT A, B_str, toString(cc) FROM (SELECT A, toString(B) AS B_str, SUM(C) AS cc FROM test.decimal GROUP BY A, B_str); + +DROP TABLE test.decimal; diff --git a/dbms/tests/queries/0_stateless/00738_nested_merge_multidimensional_array.reference b/dbms/tests/queries/0_stateless/00738_nested_merge_multidimensional_array.reference new file mode 100644 index 00000000000..f2a3a5abff5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00738_nested_merge_multidimensional_array.reference @@ -0,0 +1,2 @@ +2 2 1 +2 2 1 diff --git a/dbms/tests/queries/0_stateless/00738_nested_merge_multidimensional_array.sql b/dbms/tests/queries/0_stateless/00738_nested_merge_multidimensional_array.sql new file mode 100644 index 00000000000..41b13e32a99 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00738_nested_merge_multidimensional_array.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS test.sites; +CREATE TABLE test.sites (Domain UInt8, `Users.UserID` Array(UInt64), `Users.Dates` Array(Array(Date))) ENGINE = MergeTree ORDER BY Domain SETTINGS vertical_merge_algorithm_min_rows_to_activate = 0, vertical_merge_algorithm_min_columns_to_activate = 0; + +SYSTEM STOP MERGES; + +INSERT INTO test.sites VALUES (1,[1],[[]]); +INSERT INTO test.sites VALUES (2,[1],[['2018-06-22']]); + +SELECT count(), countArray(Users.Dates), countArrayArray(Users.Dates) FROM test.sites; +SYSTEM START MERGES; +OPTIMIZE TABLE test.sites FINAL; +SELECT count(), countArray(Users.Dates), countArrayArray(Users.Dates) FROM test.sites; + +DROP TABLE test.sites; diff --git a/dbms/tests/queries/0_stateless/00739_array_element_nullable_string_mattrobenolt.reference b/dbms/tests/queries/0_stateless/00739_array_element_nullable_string_mattrobenolt.reference new file mode 100644 index 00000000000..945f0c2deb4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00739_array_element_nullable_string_mattrobenolt.reference @@ -0,0 +1,4 @@ +1 foo +1 foo +1 \N +\N diff --git a/dbms/tests/queries/0_stateless/00739_array_element_nullable_string_mattrobenolt.sql b/dbms/tests/queries/0_stateless/00739_array_element_nullable_string_mattrobenolt.sql new file mode 100644 index 00000000000..a5b33c55d90 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00739_array_element_nullable_string_mattrobenolt.sql @@ -0,0 +1,12 @@ +create temporary table wups (a Array(Nullable(String))); +select count(), a[1] from wups group by a[1]; +insert into wups (a) values(['foo']); +select count(), a[1] from wups group by a[1]; +insert into wups (a) values([]); +select count(), a[1] from wups group by a[1] order by a[1]; + +drop temporary table wups; + +create temporary table wups (a Array(Nullable(String))); +insert into wups (a) values([]); +select a[1] from wups; diff --git a/dbms/tests/queries/0_stateless/00740_optimize_predicate_expression.reference b/dbms/tests/queries/0_stateless/00740_optimize_predicate_expression.reference new file mode 100644 index 00000000000..6db331af725 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00740_optimize_predicate_expression.reference @@ -0,0 +1 @@ +nan diff --git a/dbms/tests/queries/0_stateless/00740_optimize_predicate_expression.sql b/dbms/tests/queries/0_stateless/00740_optimize_predicate_expression.sql new file mode 100644 index 00000000000..57a7aa81ade --- /dev/null +++ b/dbms/tests/queries/0_stateless/00740_optimize_predicate_expression.sql @@ -0,0 +1,35 @@ +DROP TABLE IF EXISTS test.perf; +CREATE TABLE test.perf (site String, user_id UInt64, z Float64) ENGINE = Log; + +SELECT * FROM (SELECT perf_1.z AS z_1 FROM test.perf AS perf_1); + +SELECT sum(mul)/sqrt(sum(sqr_dif_1) * sum(sqr_dif_2)) AS z_r +FROM( +SELECT + (SELECT avg(z_1) AS z_1_avg, + avg(z_2) AS z_2_avg + FROM ( + SELECT perf_1.site, perf_1.z AS z_1 + FROM test.perf AS perf_1 + WHERE user_id = 000 + ) ALL INNER JOIN ( + SELECT perf_2.site, perf_2.z AS z_2 + FROM test.perf AS perf_2 + WHERE user_id = 999 + ) USING site) as avg_values, + z_1 - avg_values.1 AS dif_1, + z_2 - avg_values.2 AS dif_2, + dif_1 * dif_2 AS mul, + dif_1*dif_1 AS sqr_dif_1, + dif_2*dif_2 AS sqr_dif_2 +FROM ( + SELECT perf_1.site, perf_1.z AS z_1 + FROM test.perf AS perf_1 + WHERE user_id = 000 +) ALL INNER JOIN ( + SELECT perf_2.site, perf_2.z AS z_2 + FROM test.perf AS perf_2 + WHERE user_id = 999 +) USING site); + +DROP TABLE test.perf; diff --git a/dbms/tests/queries/0_stateless/00741_client_comment_multiline.reference b/dbms/tests/queries/0_stateless/00741_client_comment_multiline.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00741_client_comment_multiline.sql b/dbms/tests/queries/0_stateless/00741_client_comment_multiline.sql new file mode 100644 index 00000000000..5ca79f1c227 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00741_client_comment_multiline.sql @@ -0,0 +1 @@ +CREATE DATABASE IF NOT EXISTS test; -- foo diff --git a/dbms/tests/queries/0_stateless/00742_require_join_strictness.reference b/dbms/tests/queries/0_stateless/00742_require_join_strictness.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00742_require_join_strictness.sql b/dbms/tests/queries/0_stateless/00742_require_join_strictness.sql new file mode 100644 index 00000000000..63d81b653c5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00742_require_join_strictness.sql @@ -0,0 +1,3 @@ +SET send_logs_level = 'none'; +SET join_default_strictness = ''; +SELECT * FROM system.one INNER JOIN (SELECT number AS k FROM system.numbers) ON dummy = k; -- { serverError 417 } diff --git a/dbms/tests/queries/0_stateless/00743_limit_by_not_found_column.reference b/dbms/tests/queries/0_stateless/00743_limit_by_not_found_column.reference new file mode 100644 index 00000000000..de72e35bfe4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00743_limit_by_not_found_column.reference @@ -0,0 +1,2 @@ +0 +http://reddit.com/r/cpp/comments/xyz diff --git a/dbms/tests/queries/0_stateless/00743_limit_by_not_found_column.sql b/dbms/tests/queries/0_stateless/00743_limit_by_not_found_column.sql new file mode 100644 index 00000000000..2401e1713fd --- /dev/null +++ b/dbms/tests/queries/0_stateless/00743_limit_by_not_found_column.sql @@ -0,0 +1,65 @@ +USE test; +DROP TABLE IF EXISTS installation_stats; +CREATE TABLE installation_stats (message String, info String, message_type String) ENGINE = Log; + +SELECT count(*) AS total +FROM +( + SELECT + message, + info, + count() AS cnt + FROM installation_stats + WHERE message_type LIKE 'fail' + GROUP BY + message, + info + ORDER BY cnt DESC + LIMIT 5 BY message +); + +DROP TABLE installation_stats; + +CREATE TEMPORARY TABLE Accounts (AccountID UInt64, Currency String); + +SELECT AccountID +FROM +( + SELECT + AccountID, + Currency + FROM Accounts + LIMIT 2 BY Currency +); + +CREATE TEMPORARY TABLE commententry1 (created_date Date, link_id String, subreddit String); +INSERT INTO commententry1 VALUES ('2016-01-01', 'xyz', 'cpp'); + +SELECT concat('http://reddit.com/r/', subreddit, '/comments/', replaceRegexpOne(link_id, 't[0-9]_', '')) +FROM +( + SELECT + y, + subreddit, + link_id, + cnt + FROM + ( + SELECT + created_date AS y, + link_id, + subreddit, + count(*) AS cnt + FROM commententry1 + WHERE toYear(created_date) = 2016 + GROUP BY + y, + link_id, + subreddit + ORDER BY y ASC + ) + ORDER BY + y ASC, + cnt DESC + LIMIT 1 BY y +); diff --git a/dbms/tests/queries/0_stateless/00744_join_not_found_column.reference b/dbms/tests/queries/0_stateless/00744_join_not_found_column.reference new file mode 100644 index 00000000000..cd121fd3feb --- /dev/null +++ b/dbms/tests/queries/0_stateless/00744_join_not_found_column.reference @@ -0,0 +1,2 @@ +1 +1 1 diff --git a/dbms/tests/queries/0_stateless/00744_join_not_found_column.sql b/dbms/tests/queries/0_stateless/00744_join_not_found_column.sql new file mode 100644 index 00000000000..c942b081964 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00744_join_not_found_column.sql @@ -0,0 +1,36 @@ +CREATE TEMPORARY TABLE test +( + x Int32 +); + +INSERT INTO test VALUES (1); + +SELECT x +FROM +( + SELECT + x, + 1 + FROM test + ALL INNER JOIN + ( + SELECT + count(), + 1 + FROM test + ) USING (1) + LIMIT 10 +); + +SELECT + x, + 1 +FROM test +ALL INNER JOIN +( + SELECT + count(), + 1 + FROM test +) USING (1) +LIMIT 10; diff --git a/dbms/tests/queries/bugs/database_in_view.sql b/dbms/tests/queries/bugs/database_in_view.sql new file mode 100644 index 00000000000..0b7149e4e88 --- /dev/null +++ b/dbms/tests/queries/bugs/database_in_view.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test.whoami; +DROP TABLE IF EXISTS test.tellme; +DROP TABLE IF EXISTS test.tellme_nested; + +use test; +create view whoami as select 1 as n; +create view tellme as select * from whoami; +create view tellme_nested as select * from (select * from whoami); +select * from tellme; +select * from tellme_nested; + +use default; +select * from test.tellme; +select * from test.tellme_nested; + +DROP TABLE test.whoami; +DROP TABLE test.tellme; +DROP TABLE test.tellme_nested; diff --git a/dbms/tests/queries/bugs/prewhere_alias_array.sql b/dbms/tests/queries/bugs/prewhere_alias_array.sql new file mode 100644 index 00000000000..3281c6ac0c4 --- /dev/null +++ b/dbms/tests/queries/bugs/prewhere_alias_array.sql @@ -0,0 +1,4 @@ +DROP TABLE IF EXISTS test.prewhere; +CREATE TABLE test.prewhere (x Array(UInt64), y ALIAS x, s String) ENGINE = MergeTree ORDER BY tuple() +SELECT count() FROM test.prewhere PREWHERE (length(s) >= 1) = 0 WHERE NOT ignore(y) +DROP TABLE test.prewhere; diff --git a/debian/changelog b/debian/changelog index 6fb0d845d77..8c8faa45d4f 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (18.14.6) unstable; urgency=low +clickhouse (18.14.9) unstable; urgency=low * Modified source code - -- Thu, 11 Oct 2018 01:25:18 +0300 + -- Tue, 16 Oct 2018 15:58:16 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index 33c41b7e475..e1780db7c6f 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.14.6 +ARG version=18.14.9 RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 9563107ad84..fc86c17ad73 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.14.6 +ARG version=18.14.9 ARG gosu_ver=1.10 RUN apt-get update \ @@ -39,4 +39,4 @@ ENV CLICKHOUSE_CONFIG /etc/clickhouse-server/config.xml ENTRYPOINT ["/entrypoint.sh"] -CMD ["/usr/bin/clickhouse-server", "--config=${CLICKHOUSE_CONFIG}"] +CMD /usr/bin/clickhouse-server --config=${CLICKHOUSE_CONFIG} diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index 5fba7d947bb..32f0143d678 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=18.14.6 +ARG version=18.14.9 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ diff --git a/docs/README.md b/docs/README.md index d733c1a232d..fbc371d0d26 100644 --- a/docs/README.md +++ b/docs/README.md @@ -2,12 +2,14 @@ Basically ClickHouse uses "documentation as code" approach, so you can edit Markdown files in this folder from GitHub web interface or fork ClickHouse repository, edit, commit, push and open pull request. -At the moment documentation is bilingual in English and Russian, so it's better to try keeping languages in sync if you can, but it's not strictly required as there are people watching over this. If you add new article, you should also add it to `toc_{en,ru}.yaml` file with pages index. +At the moment documentation is bilingual in English and Russian, so it's better to try keeping languages in sync if you can, but it's not strictly required as there are people watching over this. If you add new article, you should also add it to `toc_{en,ru,zh,fa}.yaml` files with pages index. Master branch is then asynchronously published to ClickHouse official website: * In English: https://clickhouse.yandex/docs/en/ * In Russian: https://clickhouse.yandex/docs/ru/ +* In Chinese: https://clickhouse.yandex/docs/zh/ +* In Farsi: https://clickhouse.yandex/docs/fa/ Infrastructure to build Markdown to documentation website resides in [tools](tools) folder, it has it's own [README.md](tools/README.md) with more details. @@ -29,6 +31,14 @@ ClickHouse can be directly used by all sorts of either analysts and engineers, s * People tend to get temporary stuck with some specific words or phrases, usually auxiliary, for a shord period of time. So they get repeated over and over in small part of content, which looks weird when reading. It is easy to fix this by reading your text again before publishing, also you can use this opportunity to fix mistypes and lost punctuation. * Try to avoid naming the reader in text, it is not strictly prohibited though. +# How to start translation to new language + +1. Create new docs subfolder named with [ISO-639-1 language code](https://en.wikipedia.org/wiki/List_of_ISO_639-1_codes) +2. Add Markdown files with some translation, mirroring the folder structure of other languages +3. Commit and open pull request with new content + +Some additional configuration has to be done to actually make new language live on official website, but it's not automated/documented yet, so we'll do it on our own after pull request with content is merged. + # Quick cheatsheet on used Markdown dialect * Headers on separate line starting with `# `, `## ` or `### `. @@ -38,7 +48,7 @@ ClickHouse can be directly used by all sorts of either analysts and engineers, s * Inline piece of code is `in backticks`. * Multiline code block are ```in triple backtick quotes ```. * Brightly highlighted block of text starts with `!!! info "Header"`, on next line 4 spaces and content. Instead of `info` can be `warning`. -* Hide block to be opened by click: `
Header hidden content
`. +* Hide block to be opened by click: `
Header hidden content
`. * Colored text: `text`. * Additional anchor to be linked to: ``, for headers fully in English they are created automatically like `"FoO Bar" -> "foo-bar"`. * Table: diff --git a/docs/en/data_types/array.md b/docs/en/data_types/array.md index 8bb8d2200c7..f8d909cfae5 100644 --- a/docs/en/data_types/array.md +++ b/docs/en/data_types/array.md @@ -83,3 +83,5 @@ Code: 386. DB::Exception: Received from localhost:9000, 127.0.0.1. DB::Exception 0 rows in set. Elapsed: 0.246 sec. ``` + +[Original article](https://clickhouse.yandex/docs/en/data_types/array/) diff --git a/docs/en/data_types/boolean.md b/docs/en/data_types/boolean.md index 14cde15910c..52d2c8e320c 100644 --- a/docs/en/data_types/boolean.md +++ b/docs/en/data_types/boolean.md @@ -2,3 +2,5 @@ There isn't a separate type for boolean values. They use the UInt8 type, restricted to the values 0 or 1. + +[Original article](https://clickhouse.yandex/docs/en/data_types/boolean/) diff --git a/docs/en/data_types/date.md b/docs/en/data_types/date.md index cb179c0d8c4..b5bac572a73 100644 --- a/docs/en/data_types/date.md +++ b/docs/en/data_types/date.md @@ -5,3 +5,5 @@ The minimum value is output as 0000-00-00. The date is stored without the time zone. + +[Original article](https://clickhouse.yandex/docs/en/data_types/date/) diff --git a/docs/en/data_types/datetime.md b/docs/en/data_types/datetime.md index 584a0048554..c87eacbc644 100644 --- a/docs/en/data_types/datetime.md +++ b/docs/en/data_types/datetime.md @@ -13,3 +13,5 @@ By default, the client switches to the timezone of the server when it connects. So when working with a textual date (for example, when saving text dumps), keep in mind that there may be ambiguity during changes for daylight savings time, and there may be problems matching data if the time zone changed. + +[Original article](https://clickhouse.yandex/docs/en/data_types/datetime/) diff --git a/docs/en/data_types/decimal.md b/docs/en/data_types/decimal.md index fe321ee99a4..101b59adaec 100644 --- a/docs/en/data_types/decimal.md +++ b/docs/en/data_types/decimal.md @@ -95,3 +95,5 @@ SELECT toDecimal32(1, 8) < 100 ``` DB::Exception: Can't compare. ``` + +[Original article](https://clickhouse.yandex/docs/en/data_types/decimal/) diff --git a/docs/en/data_types/enum.md b/docs/en/data_types/enum.md index a3f44ca99bb..7faeecf5b4c 100644 --- a/docs/en/data_types/enum.md +++ b/docs/en/data_types/enum.md @@ -113,3 +113,5 @@ The Enum type can be changed without cost using ALTER, if only the set of values Using ALTER, it is possible to change an Enum8 to an Enum16 or vice versa, just like changing an Int8 to Int16. + +[Original article](https://clickhouse.yandex/docs/en/data_types/enum/) diff --git a/docs/en/data_types/fixedstring.md b/docs/en/data_types/fixedstring.md index d83b0087057..5e158ccd2e0 100644 --- a/docs/en/data_types/fixedstring.md +++ b/docs/en/data_types/fixedstring.md @@ -8,3 +8,5 @@ Note that this behavior differs from MySQL behavior for the CHAR type (where str Fewer functions can work with the FixedString(N) type than with String, so it is less convenient to use. + +[Original article](https://clickhouse.yandex/docs/en/data_types/fixedstring/) diff --git a/docs/en/data_types/float.md b/docs/en/data_types/float.md index d05758b0e8a..c106640e85a 100644 --- a/docs/en/data_types/float.md +++ b/docs/en/data_types/float.md @@ -13,7 +13,7 @@ We recommend that you store data in integer form whenever possible. For example, - Computations with floating-point numbers might produce a rounding error. -```sql +``` sql SELECT 1 - 0.9 ``` @@ -33,7 +33,7 @@ In contrast to standard SQL, ClickHouse supports the following categories of flo - `Inf` – Infinity. -```sql +``` sql SELECT 0.5 / 0 ``` @@ -45,7 +45,7 @@ SELECT 0.5 / 0 - `-Inf` – Negative infinity. -```sql +``` sql SELECT -0.5 / 0 ``` @@ -69,3 +69,5 @@ SELECT 0 / 0 See the rules for `NaN` sorting in the section [ORDER BY clause](../query_language/select.md#query_language-queries-order_by). + +[Original article](https://clickhouse.yandex/docs/en/data_types/float/) diff --git a/docs/en/data_types/index.md b/docs/en/data_types/index.md index ecab86c25e0..9acd30e11f2 100644 --- a/docs/en/data_types/index.md +++ b/docs/en/data_types/index.md @@ -6,3 +6,5 @@ ClickHouse can store various types of data in table cells. This section describes the supported data types and special considerations when using and/or implementing them, if any. + +[Original article](https://clickhouse.yandex/docs/en/data_types/) diff --git a/docs/en/data_types/int_uint.md b/docs/en/data_types/int_uint.md index cd940f28742..75a0d7bbde9 100644 --- a/docs/en/data_types/int_uint.md +++ b/docs/en/data_types/int_uint.md @@ -18,3 +18,5 @@ Fixed-length integers, with or without a sign. - UInt32 - [0 : 4294967295] - UInt64 - [0 : 18446744073709551615] + +[Original article](https://clickhouse.yandex/docs/en/data_types/int_uint/) diff --git a/docs/en/data_types/nested_data_structures/aggregatefunction.md b/docs/en/data_types/nested_data_structures/aggregatefunction.md index 243777c9831..377c165a103 100644 --- a/docs/en/data_types/nested_data_structures/aggregatefunction.md +++ b/docs/en/data_types/nested_data_structures/aggregatefunction.md @@ -62,3 +62,5 @@ SELECT uniqMerge(state) FROM (SELECT uniqState(UserID) AS state FROM table GROUP See [AggregatingMergeTree](../../operations/table_engines/aggregatingmergetree.md#table_engine-aggregatingmergetree) engine description. + +[Original article](https://clickhouse.yandex/docs/en/data_types/nested_data_structures/aggregatefunction/) diff --git a/docs/en/data_types/nested_data_structures/index.md b/docs/en/data_types/nested_data_structures/index.md index 53acd5b558d..1c4d2c902a8 100644 --- a/docs/en/data_types/nested_data_structures/index.md +++ b/docs/en/data_types/nested_data_structures/index.md @@ -1,2 +1,4 @@ # Nested Data Structures + +[Original article](https://clickhouse.yandex/docs/en/data_types/nested_data_structures/) diff --git a/docs/en/data_types/nested_data_structures/nested.md b/docs/en/data_types/nested_data_structures/nested.md index 8a4bd4297fb..1d90048d626 100644 --- a/docs/en/data_types/nested_data_structures/nested.md +++ b/docs/en/data_types/nested_data_structures/nested.md @@ -4,7 +4,7 @@ A nested data structure is like a nested table. The parameters of a nested data Example: -```sql +``` sql CREATE TABLE test.visits ( CounterID UInt32, @@ -35,7 +35,7 @@ In most cases, when working with a nested data structure, its individual columns Example: -```sql +``` sql SELECT Goals.ID, Goals.EventTime @@ -44,7 +44,7 @@ WHERE CounterID = 101500 AND length(Goals.ID) < 5 LIMIT 10 ``` -```text +``` ┌─Goals.ID───────────────────────┬─Goals.EventTime───────────────────────────────────────────────────────────────────────────┐ │ [1073752,591325,591325] │ ['2014-03-17 16:38:10','2014-03-17 16:38:48','2014-03-17 16:42:27'] │ │ [1073752] │ ['2014-03-17 00:28:25'] │ @@ -63,7 +63,7 @@ It is easiest to think of a nested data structure as a set of multiple column ar The only place where a SELECT query can specify the name of an entire nested data structure instead of individual columns is the ARRAY JOIN clause. For more information, see "ARRAY JOIN clause". Example: -```sql +``` sql SELECT Goal.ID, Goal.EventTime @@ -73,7 +73,7 @@ WHERE CounterID = 101500 AND length(Goals.ID) < 5 LIMIT 10 ``` -```text +``` ┌─Goal.ID─┬──────Goal.EventTime─┐ │ 1073752 │ 2014-03-17 16:38:10 │ │ 591325 │ 2014-03-17 16:38:48 │ @@ -96,3 +96,5 @@ For a DESCRIBE query, the columns in a nested data structure are listed separate The ALTER query is very limited for elements in a nested data structure. + +[Original article](https://clickhouse.yandex/docs/en/data_types/nested_data_structures/nested/) diff --git a/docs/en/data_types/nullable.md b/docs/en/data_types/nullable.md index 5cddb4c5173..87d0ac0c337 100644 --- a/docs/en/data_types/nullable.md +++ b/docs/en/data_types/nullable.md @@ -53,3 +53,5 @@ FROM t_null 2 rows in set. Elapsed: 0.144 sec. ``` + +[Original article](https://clickhouse.yandex/docs/en/data_types/nullable/) diff --git a/docs/en/data_types/special_data_types/expression.md b/docs/en/data_types/special_data_types/expression.md index 0fec4b71578..0bc7f635a0b 100644 --- a/docs/en/data_types/special_data_types/expression.md +++ b/docs/en/data_types/special_data_types/expression.md @@ -2,3 +2,5 @@ Used for representing lambda expressions in high-order functions. + +[Original article](https://clickhouse.yandex/docs/en/data_types/special_data_types/expression/) diff --git a/docs/en/data_types/special_data_types/index.md b/docs/en/data_types/special_data_types/index.md index 2c4b62deaa8..c66857cf5df 100644 --- a/docs/en/data_types/special_data_types/index.md +++ b/docs/en/data_types/special_data_types/index.md @@ -2,3 +2,5 @@ Special data type values can't be saved to a table or output in results, but are used as the intermediate result of running a query. + +[Original article](https://clickhouse.yandex/docs/en/data_types/special_data_types/) diff --git a/docs/en/data_types/special_data_types/nothing.md b/docs/en/data_types/special_data_types/nothing.md index 525142fd3ac..5b916b05557 100644 --- a/docs/en/data_types/special_data_types/nothing.md +++ b/docs/en/data_types/special_data_types/nothing.md @@ -20,3 +20,5 @@ SELECT toTypeName([]) 1 rows in set. Elapsed: 0.062 sec. ``` + +[Original article](https://clickhouse.yandex/docs/en/data_types/special_data_types/nothing/) diff --git a/docs/en/data_types/special_data_types/set.md b/docs/en/data_types/special_data_types/set.md index 346fe51e2bb..d2d61e7ac62 100644 --- a/docs/en/data_types/special_data_types/set.md +++ b/docs/en/data_types/special_data_types/set.md @@ -2,3 +2,5 @@ Used for the right half of an IN expression. + +[Original article](https://clickhouse.yandex/docs/en/data_types/special_data_types/set/) diff --git a/docs/en/data_types/string.md b/docs/en/data_types/string.md index 5386be28c96..5e2fe8e1807 100644 --- a/docs/en/data_types/string.md +++ b/docs/en/data_types/string.md @@ -12,3 +12,5 @@ If you need to store texts, we recommend using UTF-8 encoding. At the very least Similarly, certain functions for working with strings have separate variations that work under the assumption that the string contains a set of bytes representing a UTF-8 encoded text. For example, the 'length' function calculates the string length in bytes, while the 'lengthUTF8' function calculates the string length in Unicode code points, assuming that the value is UTF-8 encoded. + +[Original article](https://clickhouse.yandex/docs/en/data_types/string/) diff --git a/docs/en/data_types/tuple.md b/docs/en/data_types/tuple.md index 98e548fe837..592e5537bdb 100644 --- a/docs/en/data_types/tuple.md +++ b/docs/en/data_types/tuple.md @@ -52,3 +52,5 @@ SELECT 1 rows in set. Elapsed: 0.002 sec. ``` + +[Original article](https://clickhouse.yandex/docs/en/data_types/tuple/) diff --git a/docs/en/development/architecture.md b/docs/en/development/architecture.md index 0f443538292..e20dc2eeb6d 100644 --- a/docs/en/development/architecture.md +++ b/docs/en/development/architecture.md @@ -193,3 +193,5 @@ In addition, each replica stores its state in ZooKeeper as the set of parts and > The ClickHouse cluster consists of independent shards, and each shard consists of replicas. The cluster is not elastic, so after adding a new shard, data is not rebalanced between shards automatically. Instead, the cluster load will be uneven. This implementation gives you more control, and it is fine for relatively small clusters such as tens of nodes. But for clusters with hundreds of nodes that we are using in production, this approach becomes a significant drawback. We should implement a table engine that will span its data across the cluster with dynamically replicated regions that could be split and balanced between clusters automatically. + +[Original article](https://clickhouse.yandex/docs/en/development/architecture/) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index d2470cf2f79..bb434ae5b54 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -95,3 +95,5 @@ cd .. To create an executable, run `ninja clickhouse`. This will create the `dbms/programs/clickhouse` executable, which can be used with `client` or `server` arguments. + +[Original article](https://clickhouse.yandex/docs/en/development/build/) diff --git a/docs/en/development/build_osx.md b/docs/en/development/build_osx.md index 4e2aa2a6ff1..7de659e7ccd 100644 --- a/docs/en/development/build_osx.md +++ b/docs/en/development/build_osx.md @@ -79,3 +79,5 @@ Reboot. To check if it's working, you can use `ulimit -n` command. + +[Original article](https://clickhouse.yandex/docs/en/development/build_osx/) diff --git a/docs/en/development/index.md b/docs/en/development/index.md index a7b046fd22d..d9095f383b3 100644 --- a/docs/en/development/index.md +++ b/docs/en/development/index.md @@ -1,2 +1,4 @@ # ClickHouse Development + +[Original article](https://clickhouse.yandex/docs/en/development/) diff --git a/docs/en/development/style.md b/docs/en/development/style.md index d80f535e125..edfda73a7eb 100644 --- a/docs/en/development/style.md +++ b/docs/en/development/style.md @@ -834,3 +834,5 @@ function( const & RangesInDataParts ranges, size_t limit) ``` + +[Original article](https://clickhouse.yandex/docs/en/development/style/) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index ee2402aedc7..5455a234ae3 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -249,3 +249,5 @@ In Travis CI due to limit on time and computational power we can afford only sub In Jenkins we run functional tests for each commit and for each pull request from trusted users; the same under ASan; we also run quorum tests, dictionary tests, Metrica B2B tests. We use Jenkins to prepare and publish releases. Worth to note that we are not happy with Jenkins at all. One of our goals is to provide reliable testing infrastructure that will be available to community. + +[Original article](https://clickhouse.yandex/docs/en/development/tests/) diff --git a/docs/en/faq/general.md b/docs/en/faq/general.md index 2b19d6f35bf..f5b9d29b5bd 100644 --- a/docs/en/faq/general.md +++ b/docs/en/faq/general.md @@ -11,3 +11,5 @@ Distributed sorting is one of the main causes of reduced performance when runnin Most MapReduce implementations allow you to execute arbitrary code on a cluster. But a declarative query language is better suited to OLAP in order to run experiments quickly. For example, Hadoop has Hive and Pig. Also consider Cloudera Impala or Shark (outdated) for Spark, as well as Spark SQL, Presto, and Apache Drill. Performance when running such tasks is highly sub-optimal compared to specialized systems, but relatively high latency makes it unrealistic to use these systems as the backend for a web interface. + +[Original article](https://clickhouse.yandex/docs/en/faq/general/) diff --git a/docs/en/getting_started/example_datasets/amplab_benchmark.md b/docs/en/getting_started/example_datasets/amplab_benchmark.md index 49265d5da85..e5d9812a9c3 100644 --- a/docs/en/getting_started/example_datasets/amplab_benchmark.md +++ b/docs/en/getting_started/example_datasets/amplab_benchmark.md @@ -21,7 +21,7 @@ cd .. Run the following ClickHouse queries: -```sql +``` sql CREATE TABLE rankings_tiny ( pageURL String, @@ -96,7 +96,7 @@ for i in 5nodes/uservisits/*.deflate; do echo $i; zlib-flate -uncompress < $i | Queries for obtaining data samples: -```sql +``` sql SELECT pageURL, pageRank FROM rankings_1node WHERE pageRank > 1000 SELECT substring(sourceIP, 1, 8), sum(adRevenue) FROM uservisits_1node GROUP BY substring(sourceIP, 1, 8) @@ -119,3 +119,5 @@ ORDER BY totalRevenue DESC LIMIT 1 ``` + +[Original article](https://clickhouse.yandex/docs/en/getting_started/example_datasets/amplab_benchmark/) diff --git a/docs/en/getting_started/example_datasets/criteo.md b/docs/en/getting_started/example_datasets/criteo.md index 8544c0da9c0..268e8346d6c 100644 --- a/docs/en/getting_started/example_datasets/criteo.md +++ b/docs/en/getting_started/example_datasets/criteo.md @@ -4,7 +4,7 @@ Download the data from diff --git a/docs/en/getting_started/example_datasets/nyc_taxi.md b/docs/en/getting_started/example_datasets/nyc_taxi.md index 19cf5e242ee..0f5f1cb3cb4 100644 --- a/docs/en/getting_started/example_datasets/nyc_taxi.md +++ b/docs/en/getting_started/example_datasets/nyc_taxi.md @@ -24,7 +24,7 @@ It takes about 20-30 minutes to process each month's worth of data in PostgreSQL You can check the number of downloaded rows as follows: -```text +``` time psql nyc-taxi-data -c "SELECT count(*) FROM trips;" ## Count 1298979494 @@ -39,7 +39,7 @@ The data in PostgreSQL uses 370 GB of space. Exporting the data from PostgreSQL: -```sql +``` sql COPY ( SELECT trips.id, @@ -114,7 +114,7 @@ This takes about 5 hours. The resulting TSV file is 590612904969 bytes. Create a temporary table in ClickHouse: -```sql +``` sql CREATE TABLE trips ( trip_id UInt32, @@ -173,7 +173,7 @@ dropoff_puma Nullable(String) It is needed for converting fields to more correct data types and, if possible, to eliminate NULLs. -```text +``` time clickhouse-client --query="INSERT INTO trips FORMAT TabSeparated" < trips.tsv real 75m56.214s @@ -191,7 +191,7 @@ To start, we'll create a table on a single server. Later we will make the table Create and populate a summary table: -```text +``` CREATE TABLE trips_mergetree ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) AS SELECT @@ -258,7 +258,7 @@ To load it faster, you can create the table with the `Log` engine instead of `Me The table uses 126 GB of disk space. -```text +``` :) SELECT formatReadableSize(sum(bytes)) FROM system.parts WHERE table = 'trips_mergetree' AND active SELECT formatReadableSize(sum(bytes)) @@ -276,7 +276,7 @@ Among other things, you can run the OPTIMIZE query on MergeTree. But it's not re Q1: -```sql +``` sql SELECT cab_type, count(*) FROM trips_mergetree GROUP BY cab_type ``` @@ -284,7 +284,7 @@ SELECT cab_type, count(*) FROM trips_mergetree GROUP BY cab_type Q2: -```sql +``` sql SELECT passenger_count, avg(total_amount) FROM trips_mergetree GROUP BY passenger_count ``` @@ -292,7 +292,7 @@ SELECT passenger_count, avg(total_amount) FROM trips_mergetree GROUP BY passenge Q3: -```sql +``` sql SELECT passenger_count, toYear(pickup_date) AS year, count(*) FROM trips_mergetree GROUP BY passenger_count, year ``` @@ -300,7 +300,7 @@ SELECT passenger_count, toYear(pickup_date) AS year, count(*) FROM trips_mergetr Q4: -```sql +``` sql SELECT passenger_count, toYear(pickup_date) AS year, round(trip_distance) AS distance, count(*) FROM trips_mergetree GROUP BY passenger_count, year, distance @@ -319,19 +319,19 @@ Creating a table on three servers: On each server: -```text +``` CREATE TABLE default.trips_mergetree_third ( trip_id UInt32, vendor_id Enum8('1' = 1, '2' = 2, 'CMT' = 3, 'VTS' = 4, 'DDS' = 5, 'B02512' = 10, 'B02598' = 11, 'B02617' = 12, 'B02682' = 13, 'B02764' = 14), pickup_date Date, pickup_datetime DateTime, dropoff_date Date, dropoff_datetime DateTime, store_and_fwd_flag UInt8, rate_code_id UInt8, pickup_longitude Float64, pickup_latitude Float64, dropoff_longitude Float64, dropoff_latitude Float64, passenger_count UInt8, trip_distance Float64, fare_amount Float32, extra Float32, mta_tax Float32, tip_amount Float32, tolls_amount Float32, ehail_fee Float32, improvement_surcharge Float32, total_amount Float32, payment_type_ Enum8('UNK' = 0, 'CSH' = 1, 'CRE' = 2, 'NOC' = 3, 'DIS' = 4), trip_type UInt8, pickup FixedString(25), dropoff FixedString(25), cab_type Enum8('yellow' = 1, 'green' = 2, 'uber' = 3), pickup_nyct2010_gid UInt8, pickup_ctlabel Float32, pickup_borocode UInt8, pickup_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), pickup_ct2010 FixedString(6), pickup_boroct2010 FixedString(7), pickup_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), pickup_ntacode FixedString(4), pickup_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), pickup_puma UInt16, dropoff_nyct2010_gid UInt8, dropoff_ctlabel Float32, dropoff_borocode UInt8, dropoff_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), dropoff_ct2010 FixedString(6), dropoff_boroct2010 FixedString(7), dropoff_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), dropoff_ntacode FixedString(4), dropoff_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), dropoff_puma UInt16) ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) ``` On the source server: -```sql +``` sql CREATE TABLE trips_mergetree_x3 AS trips_mergetree_third ENGINE = Distributed(perftest, default, trips_mergetree_third, rand()) ``` The following query redistributes data: -```sql +``` sql INSERT INTO trips_mergetree_x3 SELECT * FROM trips_mergetree ``` @@ -364,3 +364,5 @@ We ran queries using a client located in a Yandex datacenter in Finland on a clu | 3 | 0.212 | 0.438 | 0.733 | 1.241 | | 140 | 0.028 | 0.043 | 0.051 | 0.072 | + +[Original article](https://clickhouse.yandex/docs/en/getting_started/example_datasets/nyc_taxi/) diff --git a/docs/en/getting_started/example_datasets/ontime.md b/docs/en/getting_started/example_datasets/ontime.md index bceab083d43..f319325d687 100644 --- a/docs/en/getting_started/example_datasets/ontime.md +++ b/docs/en/getting_started/example_datasets/ontime.md @@ -18,7 +18,7 @@ done Creating a table: -```sql +``` sql CREATE TABLE `ontime` ( `Year` UInt16, `Quarter` UInt8, @@ -142,37 +142,37 @@ Queries: Q0. -```sql +``` sql select avg(c1) from (select Year, Month, count(*) as c1 from ontime group by Year, Month); ``` Q1. The number of flights per day from the year 2000 to 2008 -```sql +``` sql SELECT DayOfWeek, count(*) AS c FROM ontime WHERE Year >= 2000 AND Year <= 2008 GROUP BY DayOfWeek ORDER BY c DESC; ``` Q2. The number of flights delayed by more than 10 minutes, grouped by the day of the week, for 2000-2008 -```sql +``` sql SELECT DayOfWeek, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Year <= 2008 GROUP BY DayOfWeek ORDER BY c DESC ``` Q3. The number of delays by airport for 2000-2008 -```sql +``` sql SELECT Origin, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Year <= 2008 GROUP BY Origin ORDER BY c DESC LIMIT 10 ``` Q4. The number of delays by carrier for 2007 -```sql +``` sql SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC ``` Q5. The percentage of delays by carrier for 2007 -```sql +``` sql SELECT Carrier, c, c2, c*1000/c2 as c3 FROM ( @@ -198,13 +198,13 @@ ORDER BY c3 DESC; Better version of the same query: -```sql +``` sql SELECT Carrier, avg(DepDelay > 10) * 1000 AS c3 FROM ontime WHERE Year = 2007 GROUP BY Carrier ORDER BY Carrier ``` Q6. The previous request for a broader range of years, 2000-2008 -```sql +``` sql SELECT Carrier, c, c2, c*1000/c2 as c3 FROM ( @@ -230,13 +230,13 @@ ORDER BY c3 DESC; Better version of the same query: -```sql +``` sql SELECT Carrier, avg(DepDelay > 10) * 1000 AS c3 FROM ontime WHERE Year >= 2000 AND Year <= 2008 GROUP BY Carrier ORDER BY Carrier ``` Q7. Percentage of flights delayed for more than 10 minutes, by year -```sql +``` sql SELECT Year, c1/c2 FROM ( @@ -260,25 +260,25 @@ ORDER BY Year Better version of the same query: -```sql +``` sql SELECT Year, avg(DepDelay > 10) FROM ontime GROUP BY Year ORDER BY Year ``` Q8. The most popular destinations by the number of directly connected cities for various year ranges -```sql +``` sql SELECT DestCityName, uniqExact(OriginCityName) AS u FROM ontime WHERE Year >= 2000 and Year <= 2010 GROUP BY DestCityName ORDER BY u DESC LIMIT 10; ``` Q9. -```sql +``` sql select Year, count(*) as c1 from ontime group by Year; ``` Q10. -```sql +``` sql select min(Year), max(Year), Carrier, count(*) as cnt, sum(ArrDelayMinutes>30) as flights_delayed, @@ -296,7 +296,7 @@ LIMIT 1000; Bonus: -```sql +``` sql SELECT avg(cnt) FROM (SELECT Year,Month,count(*) AS cnt FROM ontime WHERE DepDel15=1 GROUP BY Year,Month) select avg(c1) from (select Year,Month,count(*) as c1 from ontime group by Year,Month) @@ -317,3 +317,5 @@ This performance test was created by Vadim Tkachenko. See: - - + +[Original article](https://clickhouse.yandex/docs/en/getting_started/example_datasets/ontime/) diff --git a/docs/en/getting_started/example_datasets/star_schema.md b/docs/en/getting_started/example_datasets/star_schema.md index 6c1585bd6b0..5606cf71dd1 100644 --- a/docs/en/getting_started/example_datasets/star_schema.md +++ b/docs/en/getting_started/example_datasets/star_schema.md @@ -21,7 +21,7 @@ Generating data: Creating tables in ClickHouse: -```sql +``` sql CREATE TABLE lineorder ( LO_ORDERKEY UInt32, LO_LINENUMBER UInt8, @@ -83,3 +83,5 @@ cat customer.tbl | sed 's/$/2000-01-01/' | clickhouse-client --query "INSERT INT cat lineorder.tbl | clickhouse-client --query "INSERT INTO lineorder FORMAT CSV" ``` + +[Original article](https://clickhouse.yandex/docs/en/getting_started/example_datasets/star_schema/) diff --git a/docs/en/getting_started/example_datasets/wikistat.md b/docs/en/getting_started/example_datasets/wikistat.md index fee0a56b52c..f81d0525367 100644 --- a/docs/en/getting_started/example_datasets/wikistat.md +++ b/docs/en/getting_started/example_datasets/wikistat.md @@ -4,7 +4,7 @@ See: Creating a table: -```sql +``` sql CREATE TABLE wikistat ( date Date, @@ -25,3 +25,5 @@ cat links.txt | while read link; do wget http://dumps.wikimedia.org/other/pageco ls -1 /opt/wikistat/ | grep gz | while read i; do echo $i; gzip -cd /opt/wikistat/$i | ./wikistat-loader --time="$(echo -n $i | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})([0-9]{2})-([0-9]{2})([0-9]{2})([0-9]{2})\.gz/\1-\2-\3 \4-00-00/')" | clickhouse-client --query="INSERT INTO wikistat FORMAT TabSeparated"; done ``` + +[Original article](https://clickhouse.yandex/docs/en/getting_started/example_datasets/wikistat/) diff --git a/docs/en/getting_started/index.md b/docs/en/getting_started/index.md index a4a18e5ab12..53a8fa495fc 100644 --- a/docs/en/getting_started/index.md +++ b/docs/en/getting_started/index.md @@ -24,7 +24,7 @@ For testing and development, the system can be installed on a single server or o In `/etc/apt/sources.list` (or in a separate `/etc/apt/sources.list.d/clickhouse.list` file), add the repository: -```text +``` deb http://repo.yandex.ru/clickhouse/deb/stable/ main/ ``` @@ -51,14 +51,14 @@ To compile, follow the instructions: build.md You can compile packages and install them. You can also use programs without installing packages. -```text +``` Client: dbms/programs/clickhouse-client Server: dbms/programs/clickhouse-server ``` For the server, create a catalog with data, such as: -```text +``` /opt/clickhouse/data/default/ /opt/clickhouse/metadata/default/ ``` @@ -137,3 +137,5 @@ SELECT 1 To continue experimenting, you can try to download from the test data sets. + +[Original article](https://clickhouse.yandex/docs/en/getting_started/) diff --git a/docs/en/index.md b/docs/en/index.md index 9794746e9b7..450b14181c1 100644 --- a/docs/en/index.md +++ b/docs/en/index.md @@ -77,9 +77,8 @@ See the difference? For example, the query "count the number of records for each advertising platform" requires reading one "advertising platform ID" column, which takes up 1 byte uncompressed. If most of the traffic was not from advertising platforms, you can expect at least 10-fold compression of this column. When using a quick compression algorithm, data decompression is possible at a speed of at least several gigabytes of uncompressed data per second. In other words, this query can be processed at a speed of approximately several billion rows per second on a single server. This speed is actually achieved in practice. -
Example -

-

+
Example +``` $ clickhouse-client ClickHouse client version 0.0.52053. Connecting to localhost:9000. @@ -120,9 +119,10 @@ LIMIT 20 20 rows in set. Elapsed: 0.153 sec. Processed 1.00 billion rows, 4.00 GB (6.53 billion rows/s., 26.10 GB/s.) -:)
+:) +``` -

+ ### CPU @@ -138,3 +138,5 @@ There are two ways to do this: This is not done in "normal" databases, because it doesn't make sense when running simple queries. However, there are exceptions. For example, MemSQL uses code generation to reduce latency when processing SQL queries. (For comparison, analytical DBMSs require optimization of throughput, not latency.) Note that for CPU efficiency, the query language must be declarative (SQL or MDX), or at least a vector (J, K). The query should only contain implicit loops, allowing for optimization. + +[Original article](https://clickhouse.yandex/docs/en/) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index f3824e3951e..c042a87ee00 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -113,3 +113,5 @@ Example of a config file: ``` + +[Original article](https://clickhouse.yandex/docs/en/interfaces/cli/) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 2d08acf9cca..d26b8f54ab9 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -32,31 +32,131 @@ The table below lists supported formats and how they can be used in `INSERT` and | [XML](#xml) | ✗ | ✔ | | [CapnProto](#capnproto) | ✔ | ✔ | - + -## CapnProto +## TabSeparated -Cap'n Proto is a binary message format similar to Protocol Buffers and Thrift, but not like JSON or MessagePack. +In TabSeparated format, data is written by row. Each row contains values separated by tabs. Each value is follow by a tab, except the last value in the row, which is followed by a line feed. Strictly Unix line feeds are assumed everywhere. The last row also must contain a line feed at the end. Values are written in text format, without enclosing quotation marks, and with special characters escaped. -Cap'n Proto messages are strictly typed and not self-describing, meaning they need an external schema description. The schema is applied on the fly and cached for each query. +This format is also available under the name `TSV`. -```sql -SELECT SearchPhrase, count() AS c FROM test.hits - GROUP BY SearchPhrase FORMAT CapnProto SETTINGS schema = 'schema:Message' +The `TabSeparated` format is convenient for processing data using custom programs and scripts. It is used by default in the HTTP interface, and in the command-line client's batch mode. This format also allows transferring data between different DBMSs. For example, you can get a dump from MySQL and upload it to ClickHouse, or vice versa. + +The `TabSeparated` format supports outputting total values (when using WITH TOTALS) and extreme values (when 'extremes' is set to 1). In these cases, the total values and extremes are output after the main data. The main result, total values, and extremes are separated from each other by an empty line. Example: + +``` sql +SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT TabSeparated`` ``` -Where `schema.capnp` looks like this: - ``` -struct Message { - SearchPhrase @0 :Text; - c @1 :Uint64; -} +2014-03-17 1406958 +2014-03-18 1383658 +2014-03-19 1405797 +2014-03-20 1353623 +2014-03-21 1245779 +2014-03-22 1031592 +2014-03-23 1046491 + +0000-00-00 8873898 + +2014-03-17 1031592 +2014-03-23 1406958 ``` -Schema files are in the file that is located in the directory specified in [ format_schema_path](../operations/server_settings/settings.md#server_settings-format_schema_path) in the server configuration. +### Data formatting + +Integer numbers are written in decimal form. Numbers can contain an extra "+" character at the beginning (ignored when parsing, and not recorded when formatting). Non-negative numbers can't contain the negative sign. When reading, it is allowed to parse an empty string as a zero, or (for signed types) a string consisting of just a minus sign as a zero. Numbers that do not fit into the corresponding data type may be parsed as a different number, without an error message. + +Floating-point numbers are written in decimal form. The dot is used as the decimal separator. Exponential entries are supported, as are 'inf', '+inf', '-inf', and 'nan'. An entry of floating-point numbers may begin or end with a decimal point. +During formatting, accuracy may be lost on floating-point numbers. +During parsing, it is not strictly required to read the nearest machine-representable number. + +Dates are written in YYYY-MM-DD format and parsed in the same format, but with any characters as separators. +Dates with times are written in the format YYYY-MM-DD hh:mm:ss and parsed in the same format, but with any characters as separators. +This all occurs in the system time zone at the time the client or server starts (depending on which one formats data). For dates with times, daylight saving time is not specified. So if a dump has times during daylight saving time, the dump does not unequivocally match the data, and parsing will select one of the two times. +During a read operation, incorrect dates and dates with times can be parsed with natural overflow or as null dates and times, without an error message. + +As an exception, parsing dates with times is also supported in Unix timestamp format, if it consists of exactly 10 decimal digits. The result is not time zone-dependent. The formats YYYY-MM-DD hh:mm:ss and NNNNNNNNNN are differentiated automatically. + +Strings are output with backslash-escaped special characters. The following escape sequences are used for output: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\'`, `\\`. Parsing also supports the sequences `\a`, `\v`, and `\xHH` (hex escape sequences) and any `\c` sequences, where `c` is any character (these sequences are converted to `c`). Thus, reading data supports formats where a line feed can be written as `\n` or `\`, or as a line feed. For example, the string `Hello world` with a line feed between the words instead of a space can be parsed in any of the following variations: + +``` +Hello\nworld + +Hello\ +world +``` + +The second variant is supported because MySQL uses it when writing tab-separated dumps. + +The minimum set of characters that you need to escape when passing data in TabSeparated format: tab, line feed (LF) and backslash. + +Only a small set of symbols are escaped. You can easily stumble onto a string value that your terminal will ruin in output. + +Arrays are written as a list of comma-separated values in square brackets. Number items in the array are fomratted as normally, but dates, dates with times, and strings are written in single quotes with the same escaping rules as above. + +[NULL](../query_language/syntax.md#null-literal) is formatted as `\N`. + + + +## TabSeparatedRaw + +Differs from `TabSeparated` format in that the rows are written without escaping. +This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). + +This format is also available under the name `TSVRaw`. + + +## TabSeparatedWithNames + +Differs from the `TabSeparated` format in that the column names are written in the first row. +During parsing, the first row is completely ignored. You can't use column names to determine their position or to check their correctness. +(Support for parsing the header row may be added in the future.) + +This format is also available under the name `TSVWithNames`. + + +## TabSeparatedWithNamesAndTypes + +Differs from the `TabSeparated` format in that the column names are written to the first row, while the column types are in the second row. +During parsing, the first and second rows are completely ignored. + +This format is also available under the name `TSVWithNamesAndTypes`. + + +## TSKV + +Similar to TabSeparated, but outputs a value in name=value format. Names are escaped the same way as in TabSeparated format, and the = symbol is also escaped. + +``` +SearchPhrase= count()=8267016 +SearchPhrase=bathroom interior design count()=2166 +SearchPhrase=yandex count()=1655 +SearchPhrase=2014 spring fashion count()=1549 +SearchPhrase=freeform photos count()=1480 +SearchPhrase=angelina jolie count()=1245 +SearchPhrase=omsk count()=1112 +SearchPhrase=photos of dog breeds count()=1091 +SearchPhrase=curtain designs count()=1064 +SearchPhrase=baku count()=1000 +``` + +[NULL](../query_language/syntax.md#null-literal) is formatted as `\N`. + +``` sql +SELECT * FROM t_null FORMAT TSKV +``` + +``` +x=1 y=\N +``` + +When there is a large number of small columns, this format is ineffective, and there is generally no reason to use it. It is used in some departments of Yandex. + +Both data output and parsing are supported in this format. For parsing, any order is supported for the values of different columns. It is acceptable for some values to be omitted – they are treated as equal to their default values. In this case, zeros and blank rows are used as default values. Complex values that could be specified in the table are not supported as defaults. + +Parsing allows the presence of the additional field `tskv` without the equal sign or a value. This field is ignored. -Deserialization is effective and usually doesn't increase the system load. ## CSV @@ -86,7 +186,7 @@ Also prints the header row, similar to `TabSeparatedWithNames`. Outputs data in JSON format. Besides data tables, it also outputs column names and types, along with some additional information: the total number of output rows, and the number of rows that could have been output if there weren't a LIMIT. Example: -```sql +``` sql SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTALS ORDER BY c DESC LIMIT 5 FORMAT JSON ``` @@ -263,7 +363,7 @@ Each result block is output as a separate table. This is necessary so that block [NULL](../query_language/syntax.md#null-literal) is output as `ᴺᵁᴸᴸ`. -```sql +``` sql SELECT * FROM t_null ``` @@ -278,11 +378,11 @@ This format is only appropriate for outputting a query result, but not for parsi The Pretty format supports outputting total values (when using WITH TOTALS) and extremes (when 'extremes' is set to 1). In these cases, total values and extreme values are output after the main data, in separate tables. Example (shown for the PrettyCompact format): -```sql +``` sql SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT PrettyCompact ``` -```text +``` ┌──EventDate─┬───────c─┐ │ 2014-03-17 │ 1406958 │ │ 2014-03-18 │ 1383658 │ @@ -359,131 +459,6 @@ Array is represented as a varint length (unsigned [LEB128](https://en.wikipedia. For [NULL](../query_language/syntax.md#null-literal) support, an additional byte containing 1 or 0 is added before each [Nullable](../data_types/nullable.md#data_type-nullable) value. If 1, then the value is `NULL` and this byte is interpreted as a separate value. If 0, the value after the byte is not `NULL`. - - -## TabSeparated - -In TabSeparated format, data is written by row. Each row contains values separated by tabs. Each value is follow by a tab, except the last value in the row, which is followed by a line feed. Strictly Unix line feeds are assumed everywhere. The last row also must contain a line feed at the end. Values are written in text format, without enclosing quotation marks, and with special characters escaped. - -This format is also available under the name `TSV`. - -The `TabSeparated` format is convenient for processing data using custom programs and scripts. It is used by default in the HTTP interface, and in the command-line client's batch mode. This format also allows transferring data between different DBMSs. For example, you can get a dump from MySQL and upload it to ClickHouse, or vice versa. - -The `TabSeparated` format supports outputting total values (when using WITH TOTALS) and extreme values (when 'extremes' is set to 1). In these cases, the total values and extremes are output after the main data. The main result, total values, and extremes are separated from each other by an empty line. Example: - -```sql -SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT TabSeparated`` -``` - -```text -2014-03-17 1406958 -2014-03-18 1383658 -2014-03-19 1405797 -2014-03-20 1353623 -2014-03-21 1245779 -2014-03-22 1031592 -2014-03-23 1046491 - -0000-00-00 8873898 - -2014-03-17 1031592 -2014-03-23 1406958 -``` - -## Data formatting - -Integer numbers are written in decimal form. Numbers can contain an extra "+" character at the beginning (ignored when parsing, and not recorded when formatting). Non-negative numbers can't contain the negative sign. When reading, it is allowed to parse an empty string as a zero, or (for signed types) a string consisting of just a minus sign as a zero. Numbers that do not fit into the corresponding data type may be parsed as a different number, without an error message. - -Floating-point numbers are written in decimal form. The dot is used as the decimal separator. Exponential entries are supported, as are 'inf', '+inf', '-inf', and 'nan'. An entry of floating-point numbers may begin or end with a decimal point. -During formatting, accuracy may be lost on floating-point numbers. -During parsing, it is not strictly required to read the nearest machine-representable number. - -Dates are written in YYYY-MM-DD format and parsed in the same format, but with any characters as separators. -Dates with times are written in the format YYYY-MM-DD hh:mm:ss and parsed in the same format, but with any characters as separators. -This all occurs in the system time zone at the time the client or server starts (depending on which one formats data). For dates with times, daylight saving time is not specified. So if a dump has times during daylight saving time, the dump does not unequivocally match the data, and parsing will select one of the two times. -During a read operation, incorrect dates and dates with times can be parsed with natural overflow or as null dates and times, without an error message. - -As an exception, parsing dates with times is also supported in Unix timestamp format, if it consists of exactly 10 decimal digits. The result is not time zone-dependent. The formats YYYY-MM-DD hh:mm:ss and NNNNNNNNNN are differentiated automatically. - -Strings are output with backslash-escaped special characters. The following escape sequences are used for output: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\'`, `\\`. Parsing also supports the sequences `\a`, `\v`, and `\xHH` (hex escape sequences) and any `\c` sequences, where `c` is any character (these sequences are converted to `c`). Thus, reading data supports formats where a line feed can be written as `\n` or `\`, or as a line feed. For example, the string `Hello world` with a line feed between the words instead of a space can be parsed in any of the following variations: - -```text -Hello\nworld - -Hello\ -world -``` - -The second variant is supported because MySQL uses it when writing tab-separated dumps. - -The minimum set of characters that you need to escape when passing data in TabSeparated format: tab, line feed (LF) and backslash. - -Only a small set of symbols are escaped. You can easily stumble onto a string value that your terminal will ruin in output. - -Arrays are written as a list of comma-separated values in square brackets. Number items in the array are fomratted as normally, but dates, dates with times, and strings are written in single quotes with the same escaping rules as above. - -[NULL](../query_language/syntax.md#null-literal) is formatted as `\N`. - - - -## TabSeparatedRaw - -Differs from `TabSeparated` format in that the rows are written without escaping. -This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). - -This format is also available under the name `TSVRaw`. - - -## TabSeparatedWithNames - -Differs from the `TabSeparated` format in that the column names are written in the first row. -During parsing, the first row is completely ignored. You can't use column names to determine their position or to check their correctness. -(Support for parsing the header row may be added in the future.) - -This format is also available under the name `TSVWithNames`. - - -## TabSeparatedWithNamesAndTypes - -Differs from the `TabSeparated` format in that the column names are written to the first row, while the column types are in the second row. -During parsing, the first and second rows are completely ignored. - -This format is also available under the name `TSVWithNamesAndTypes`. - - -## TSKV - -Similar to TabSeparated, but outputs a value in name=value format. Names are escaped the same way as in TabSeparated format, and the = symbol is also escaped. - -```text -SearchPhrase= count()=8267016 -SearchPhrase=bathroom interior design count()=2166 -SearchPhrase=yandex count()=1655 -SearchPhrase=2014 spring fashion count()=1549 -SearchPhrase=freeform photos count()=1480 -SearchPhrase=angelina jolie count()=1245 -SearchPhrase=omsk count()=1112 -SearchPhrase=photos of dog breeds count()=1091 -SearchPhrase=curtain designs count()=1064 -SearchPhrase=baku count()=1000 -``` - -[NULL](../query_language/syntax.md#null-literal) is formatted as `\N`. - -```sql -SELECT * FROM t_null FORMAT TSKV -``` - -``` -x=1 y=\N -``` - -When there is a large number of small columns, this format is ineffective, and there is generally no reason to use it. It is used in some departments of Yandex. - -Both data output and parsing are supported in this format. For parsing, any order is supported for the values of different columns. It is acceptable for some values to be omitted – they are treated as equal to their default values. In this case, zeros and blank rows are used as default values. Complex values that could be specified in the table are not supported as defaults. - -Parsing allows the presence of the additional field `tskv` without the equal sign or a value. This field is ignored. - ## Values Prints every row in brackets. Rows are separated by commas. There is no comma after the last row. The values inside the brackets are also comma-separated. Numbers are output in decimal format without quotes. Arrays are output in square brackets. Strings, dates, and dates with times are output in quotes. Escaping rules and parsing are similar to the [TabSeparated](#tabseparated) format. During formatting, extra spaces aren't inserted, but during parsing, they are allowed and skipped (except for spaces inside array values, which are not allowed). [NULL](../query_language/syntax.md#null-literal) is represented as `NULL`. @@ -502,7 +477,7 @@ Prints each value on a separate line with the column name specified. This format Example: -```sql +``` sql SELECT * FROM t_null FORMAT Vertical ``` @@ -620,3 +595,31 @@ Just as for JSON, invalid UTF-8 sequences are changed to the replacement charact In string values, the characters `<` and `&` are escaped as `<` and `&`. Arrays are output as `HelloWorld...`,and tuples as `HelloWorld...`. + + + +## CapnProto + +Cap'n Proto is a binary message format similar to Protocol Buffers and Thrift, but not like JSON or MessagePack. + +Cap'n Proto messages are strictly typed and not self-describing, meaning they need an external schema description. The schema is applied on the fly and cached for each query. + +``` sql +SELECT SearchPhrase, count() AS c FROM test.hits + GROUP BY SearchPhrase FORMAT CapnProto SETTINGS schema = 'schema:Message' +``` + +Where `schema.capnp` looks like this: + +``` +struct Message { + SearchPhrase @0 :Text; + c @1 :Uint64; +} +``` + +Schema files are in the file that is located in the directory specified in [ format_schema_path](../operations/server_settings/settings.md#server_settings-format_schema_path) in the server configuration. + +Deserialization is effective and usually doesn't increase the system load. + +[Original article](https://clickhouse.yandex/docs/en/interfaces/formats/) diff --git a/docs/en/interfaces/http_interface.md b/docs/en/interfaces/http_interface.md index 4b20f2a0b65..e514f0c4969 100644 --- a/docs/en/interfaces/http_interface.md +++ b/docs/en/interfaces/http_interface.md @@ -218,3 +218,5 @@ curl -sS 'http://localhost:8123/?max_result_bytes=4000000&buffer_size=3000000&wa Use buffering to avoid situations where a query processing error occurred after the response code and HTTP headers were sent to the client. In this situation, an error message is written at the end of the response body, and on the client side, the error can only be detected at the parsing stage. + +[Original article](https://clickhouse.yandex/docs/en/interfaces/http_interface/) diff --git a/docs/en/interfaces/index.md b/docs/en/interfaces/index.md index e43f4474271..9f445d45229 100644 --- a/docs/en/interfaces/index.md +++ b/docs/en/interfaces/index.md @@ -4,3 +4,5 @@ To explore the system's capabilities, download data to tables, or make manual queries, use the clickhouse-client program. + +[Original article](https://clickhouse.yandex/docs/en/interfaces/) diff --git a/docs/en/interfaces/jdbc.md b/docs/en/interfaces/jdbc.md index a8808770fa9..8454881dfb7 100644 --- a/docs/en/interfaces/jdbc.md +++ b/docs/en/interfaces/jdbc.md @@ -3,3 +3,5 @@ - [Official driver](https://github.com/yandex/clickhouse-jdbc). - Third-party driver from [ClickHouse-Native-JDBC](https://github.com/housepower/ClickHouse-Native-JDBC). + +[Original article](https://clickhouse.yandex/docs/en/interfaces/jdbc/) diff --git a/docs/en/interfaces/tcp.md b/docs/en/interfaces/tcp.md index c9813823bc8..86ef118e882 100644 --- a/docs/en/interfaces/tcp.md +++ b/docs/en/interfaces/tcp.md @@ -2,3 +2,5 @@ The native interface is used in the "clickhouse-client" command-line client for interaction between servers with distributed query processing, and also in C++ programs. We will only cover the command-line client. + +[Original article](https://clickhouse.yandex/docs/en/interfaces/tcp/) diff --git a/docs/en/interfaces/third-party_client_libraries.md b/docs/en/interfaces/third-party_client_libraries.md index a46d9efc696..3ae40e829dc 100644 --- a/docs/en/interfaces/third-party_client_libraries.md +++ b/docs/en/interfaces/third-party_client_libraries.md @@ -7,6 +7,7 @@ We have not tested the libraries listed below. - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) - [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-client](https://github.com/yurial/clickhouse-client) + - [aiochclient](https://github.com/maximdanilchenko/aiochclient) - PHP - [phpClickHouse](https://github.com/smi2/phpClickHouse) - [clickhouse-php-client](https://github.com/8bitov/clickhouse-php-client) @@ -42,6 +43,10 @@ We have not tested the libraries listed below. - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) - Java - [clickhouse-client-java](https://github.com/VirtusAI/clickhouse-client-java) +- Kotlin + - [AORM](https://github.com/TanVD/AORM) - Nim - [nim-clickhouse](https://github.com/leonardoce/nim-clickhouse) + +[Original article](https://clickhouse.yandex/docs/en/interfaces/third-party_client_libraries/) diff --git a/docs/en/interfaces/third-party_gui.md b/docs/en/interfaces/third-party_gui.md index eee127f4f0d..280b0bee218 100644 --- a/docs/en/interfaces/third-party_gui.md +++ b/docs/en/interfaces/third-party_gui.md @@ -36,3 +36,14 @@ The following features are planned for development: - Cluster management. - Monitoring replicated and Kafka tables. +## DBeaver + +[DBeaver](https://dbeaver.io/) - universal desktop database client with ClickHouse support. + +Key features: + +- Query development with syntax highlight. +- Table preview. +- Autocompletion. + +[Original article](https://clickhouse.yandex/docs/en/interfaces/third-party_gui/) diff --git a/docs/en/introduction/distinctive_features.md b/docs/en/introduction/distinctive_features.md index eff79191bbc..3354a75c54b 100644 --- a/docs/en/introduction/distinctive_features.md +++ b/docs/en/introduction/distinctive_features.md @@ -60,3 +60,5 @@ ClickHouse provides various ways to trade accuracy for performance: Uses asynchronous multimaster replication. After being written to any available replica, data is distributed to all the remaining replicas in the background. The system maintains identical data on different replicas. Recovery after most failures is performed automatically, and in complex cases — semi-automatically. For more information, see the section [Data replication](../operations/table_engines/replication.md#table_engines-replication). + +[Original article](https://clickhouse.yandex/docs/en/introduction/distinctive_features/) diff --git a/docs/en/introduction/features_considered_disadvantages.md b/docs/en/introduction/features_considered_disadvantages.md index 55fecaf123a..5071c9b9b8d 100644 --- a/docs/en/introduction/features_considered_disadvantages.md +++ b/docs/en/introduction/features_considered_disadvantages.md @@ -3,3 +3,5 @@ 1. No full-fledged transactions. 2. Lack of ability to modify or delete already inserted data with high rate and low latency. There are batch deletes and updates available to clean up or modify data, for example to comply with [GDPR](https://gdpr-info.eu). 3. The sparse index makes ClickHouse not really suitable for point queries retrieving single rows by their keys. + +[Original article](https://clickhouse.yandex/docs/en/introduction/features_considered_disadvantages/) diff --git a/docs/en/introduction/performance.md b/docs/en/introduction/performance.md index d9796d26388..47d4a3b946d 100644 --- a/docs/en/introduction/performance.md +++ b/docs/en/introduction/performance.md @@ -21,3 +21,5 @@ Under the same conditions, ClickHouse can handle several hundred queries per sec ## Performance When Inserting Data We recommend inserting data in packets of at least 1000 rows, or no more than a single request per second. When inserting to a MergeTree table from a tab-separated dump, the insertion speed will be from 50 to 200 MB/s. If the inserted rows are around 1 Kb in size, the speed will be from 50,000 to 200,000 rows per second. If the rows are small, the performance will be higher in rows per second (on Banner System data -`>` 500,000 rows per second; on Graphite data -`>` 1,000,000 rows per second). To improve performance, you can make multiple INSERT queries in parallel, and performance will increase linearly. + +[Original article](https://clickhouse.yandex/docs/en/introduction/performance/) diff --git a/docs/en/introduction/ya_metrika_task.md b/docs/en/introduction/ya_metrika_task.md index db173e17817..9acbdc0989e 100644 --- a/docs/en/introduction/ya_metrika_task.md +++ b/docs/en/introduction/ya_metrika_task.md @@ -46,3 +46,5 @@ OLAPServer worked well for non-aggregated data, but it had many restrictions tha To remove the limitations of OLAPServer and solve the problem of working with non-aggregated data for all reports, we developed the ClickHouse DBMS. + +[Original article](https://clickhouse.yandex/docs/en/introduction/ya_metrika_task/) diff --git a/docs/en/operations/access_rights.md b/docs/en/operations/access_rights.md index 3064da75108..451be2c7322 100644 --- a/docs/en/operations/access_rights.md +++ b/docs/en/operations/access_rights.md @@ -99,3 +99,5 @@ The user can get a list of all databases and tables in them by using `SHOW` quer Database access is not related to the [readonly](settings/query_complexity.md#query_complexity_readonly) setting. You can't grant full access to one database and `readonly` access to another one. + +[Original article](https://clickhouse.yandex/docs/en/operations/access_rights/) diff --git a/docs/en/operations/configuration_files.md b/docs/en/operations/configuration_files.md index d55cf7c2001..a7cdfb124ee 100644 --- a/docs/en/operations/configuration_files.md +++ b/docs/en/operations/configuration_files.md @@ -40,3 +40,5 @@ $ cat /etc/clickhouse-server/users.d/alice.xml For each config file, the server also generates `file-preprocessed.xml` files when starting. These files contain all the completed substitutions and overrides, and they are intended for informational use. If ZooKeeper substitutions were used in the config files but ZooKeeper is not available on the server start, the server loads the configuration from the preprocessed file. The server tracks changes in config files, as well as files and ZooKeeper nodes that were used when performing substitutions and overrides, and reloads the settings for users and clusters on the fly. This means that you can modify the cluster, users, and their settings without restarting the server. + +[Original article](https://clickhouse.yandex/docs/en/operations/configuration_files/) diff --git a/docs/en/operations/index.md b/docs/en/operations/index.md index 9073e2a2a78..63cb19bb639 100644 --- a/docs/en/operations/index.md +++ b/docs/en/operations/index.md @@ -1,2 +1,4 @@ # Operations + +[Original article](https://clickhouse.yandex/docs/en/operations/) diff --git a/docs/en/operations/quotas.md b/docs/en/operations/quotas.md index e8b603ff11a..e8cde130828 100644 --- a/docs/en/operations/quotas.md +++ b/docs/en/operations/quotas.md @@ -104,3 +104,5 @@ For distributed query processing, the accumulated amounts are stored on the requ When the server is restarted, quotas are reset. + +[Original article](https://clickhouse.yandex/docs/en/operations/quotas/) diff --git a/docs/en/operations/server_settings/index.md b/docs/en/operations/server_settings/index.md index 5631d131a43..88f11c48f4b 100644 --- a/docs/en/operations/server_settings/index.md +++ b/docs/en/operations/server_settings/index.md @@ -10,3 +10,5 @@ Other settings are described in the "[Settings](../settings/index.md#settings)" Before studying the settings, read the [Configuration files](../configuration_files.md#configuration_files) section and note the use of substitutions (the `incl` and `optional` attributes). + +[Original article](https://clickhouse.yandex/docs/en/operations/server_settings/) diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index 7f52f9bb845..8e10969ed6b 100644 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -717,3 +717,5 @@ For more information, see the section "[Replication](../../operations/table_engi ``` + +[Original article](https://clickhouse.yandex/docs/en/operations/server_settings/settings/) diff --git a/docs/en/operations/settings/index.md b/docs/en/operations/settings/index.md index 1c03340670f..5676796fd10 100644 --- a/docs/en/operations/settings/index.md +++ b/docs/en/operations/settings/index.md @@ -22,3 +22,5 @@ Similarly, you can use ClickHouse sessions in the HTTP protocol. To do this, you Settings that can only be made in the server config file are not covered in this section. + +[Original article](https://clickhouse.yandex/docs/en/operations/settings/) diff --git a/docs/en/operations/settings/query_complexity.md b/docs/en/operations/settings/query_complexity.md index 2132557d699..9e49dc58ca3 100644 --- a/docs/en/operations/settings/query_complexity.md +++ b/docs/en/operations/settings/query_complexity.md @@ -193,3 +193,5 @@ Maximum number of bytes (uncompressed data) that can be passed to a remote serve ## transfer_overflow_mode What to do when the amount of data exceeds one of the limits: 'throw' or 'break'. By default, throw. + +[Original article](https://clickhouse.yandex/docs/en/operations/settings/query_complexity/) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 67673cab042..8afea8e5d35 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -417,3 +417,5 @@ See also the following parameters: - [insert_quorum](#setting-insert_quorum) - [insert_quorum_timeout](#setting-insert_quorum_timeout) + +[Original article](https://clickhouse.yandex/docs/en/operations/settings/settings/) diff --git a/docs/en/operations/settings/settings_profiles.md b/docs/en/operations/settings/settings_profiles.md index 3e3175bc9fb..338800fbee2 100644 --- a/docs/en/operations/settings/settings_profiles.md +++ b/docs/en/operations/settings/settings_profiles.md @@ -9,7 +9,7 @@ Example: Install the `web` profile. -```sql +``` sql SET profile = 'web' ``` @@ -63,3 +63,5 @@ The example specifies two profiles: `default` and `web`. The `default` profile Settings profiles can inherit from each other. To use inheritance, indicate the `profile` setting before the other settings that are listed in the profile. + +[Original article](https://clickhouse.yandex/docs/en/operations/settings/settings_profiles/) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index abd6e819373..a67fec3f9c5 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -18,7 +18,7 @@ Example: The number of SELECT queries currently running; the amount of memory in Contains information about clusters available in the config file and the servers in them. Columns: -```text +``` cluster String — The cluster name. shard_num UInt32 — The shard number in the cluster, starting from 1. shard_weight UInt32 — The relative weight of the shard when writing data. @@ -34,7 +34,7 @@ user String — The name of the user for connecting to the server. Contains information about the columns in all tables. You can use this table to get information similar to `DESCRIBE TABLE`, but for multiple tables at once. -```text +``` database String — The name of the database the table is in. table String – Table name. name String — Column name. @@ -183,7 +183,7 @@ Formats: This system table is used for implementing the `SHOW PROCESSLIST` query. Columns: -```text +``` user String – Name of the user who made the request. For distributed query processing, this is the user who helped the requestor server send the query to this server, not the user who made the distributed request on the requestor server. address String - The IP address the request was made from. The same for distributed processing. @@ -210,14 +210,14 @@ This table can be used for monitoring. The table contains a row for every Replic Example: -```sql +``` sql SELECT * FROM system.replicas WHERE table = 'visits' FORMAT Vertical ``` -```text +``` Row 1: ────── database: merge @@ -243,7 +243,7 @@ active_replicas: 2 Columns: -```text +``` database: Database name table: Table name engine: Table engine name @@ -296,7 +296,7 @@ If you don't request the last 4 columns (log_max_index, log_pointer, total_repli For example, you can check that everything is working correctly like this: -```sql +``` sql SELECT database, table, @@ -335,7 +335,7 @@ I.e. used for executing the query you are using to read from the system.settings Columns: -```text +``` name String — Setting name. value String — Setting value. changed UInt8 — Whether the setting was explicitly defined in the config or explicitly changed. @@ -343,13 +343,13 @@ changed UInt8 — Whether the setting was explicitly defined in the config or ex Example: -```sql +``` sql SELECT * FROM system.settings WHERE changed ``` -```text +``` ┌─name───────────────────┬─value───────┬─changed─┐ │ max_threads │ 8 │ 1 │ │ use_uncompressed_cache │ 0 │ 1 │ @@ -393,14 +393,14 @@ Columns: Example: -```sql +``` sql SELECT * FROM system.zookeeper WHERE path = '/clickhouse/tables/01-08/visits/replicas' FORMAT Vertical ``` -```text +``` Row 1: ────── name: example01-08-1.yandex.ru @@ -435,3 +435,5 @@ numChildren: 7 pzxid: 987021252247 path: /clickhouse/tables/01-08/visits/replicas ``` + +[Original article](https://clickhouse.yandex/docs/en/operations/system_tables/) diff --git a/docs/en/operations/table_engines/aggregatingmergetree.md b/docs/en/operations/table_engines/aggregatingmergetree.md index 6932984797f..3a463e619a5 100644 --- a/docs/en/operations/table_engines/aggregatingmergetree.md +++ b/docs/en/operations/table_engines/aggregatingmergetree.md @@ -12,8 +12,8 @@ It is appropriate to use `AggregatingMergeTree` if it reduces the number of row ## Creating a Table -``` -CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] +``` sql +CREATE TABLE t ( name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], @@ -58,7 +58,7 @@ In the results of `SELECT` query the values of `AggregateFunction` type have im `AggregatingMergeTree` materialized view that watches the `test.visits` table: -```sql +``` sql CREATE MATERIALIZED VIEW test.basic ENGINE = AggregatingMergeTree() PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate) AS SELECT @@ -72,7 +72,7 @@ GROUP BY CounterID, StartDate; Inserting of data into the `test.visits` table. -```sql +``` sql INSERT INTO test.visits ... ``` @@ -80,7 +80,7 @@ The data are inserted in both the table and view `test.basic` that will perform To get the aggregated data, we need to execute a query such as `SELECT ... GROUP BY ...` from the view `test.basic`: -```sql +``` sql SELECT StartDate, sumMerge(Visits) AS Visits, @@ -89,3 +89,5 @@ FROM test.basic GROUP BY StartDate ORDER BY StartDate; ``` + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/aggregatingmergetree/) diff --git a/docs/en/operations/table_engines/buffer.md b/docs/en/operations/table_engines/buffer.md index 2098c5f19da..24a990bb260 100644 --- a/docs/en/operations/table_engines/buffer.md +++ b/docs/en/operations/table_engines/buffer.md @@ -2,7 +2,7 @@ Buffers the data to write in RAM, periodically flushing it to another table. During the read operation, data is read from the buffer and the other table simultaneously. -```text +``` Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes) ``` @@ -16,7 +16,7 @@ The conditions for flushing the data are calculated separately for each of the ' Example: -```sql +``` sql CREATE TABLE merge.hits_buffer AS merge.hits ENGINE = Buffer(merge, hits, 16, 10, 100, 10000, 1000000, 10000000, 100000000) ``` @@ -52,3 +52,5 @@ A Buffer table is used when too many INSERTs are received from a large number of Note that it doesn't make sense to insert data one row at a time, even for Buffer tables. This will only produce a speed of a few thousand rows per second, while inserting larger blocks of data can produce over a million rows per second (see the section "Performance"). + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/buffer/) diff --git a/docs/en/operations/table_engines/collapsingmergetree.md b/docs/en/operations/table_engines/collapsingmergetree.md index 7954ff9b359..c580b85fab3 100644 --- a/docs/en/operations/table_engines/collapsingmergetree.md +++ b/docs/en/operations/table_engines/collapsingmergetree.md @@ -215,3 +215,5 @@ SELECT * FROM UAct FINAL ``` This way of selecting the data is very inefficient. Don't use it for big tables. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/collapsingmergetree/) diff --git a/docs/en/operations/table_engines/custom_partitioning_key.md b/docs/en/operations/table_engines/custom_partitioning_key.md index bcfe8c8c410..55940db8ca9 100644 --- a/docs/en/operations/table_engines/custom_partitioning_key.md +++ b/docs/en/operations/table_engines/custom_partitioning_key.md @@ -12,7 +12,7 @@ ENGINE [=] Name(...) [PARTITION BY expr] [ORDER BY expr] [SAMPLE BY expr] [SETTI For MergeTree tables, the partition expression is specified after `PARTITION BY`, the primary key after `ORDER BY`, the sampling key after `SAMPLE BY`, and `SETTINGS` can specify `index_granularity` (optional; the default value is 8192), as well as other settings from [MergeTreeSettings.h](https://github.com/yandex/ClickHouse/blob/master/dbms/src/Storages/MergeTree/MergeTreeSettings.h). The other engine parameters are specified in parentheses after the engine name, as previously. Example: -```sql +``` sql ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/name', 'replica1', Sign) PARTITION BY (toMonday(StartDate), EventType) ORDER BY (CounterID, StartDate, intHash32(UserID)) @@ -27,7 +27,7 @@ After this table is created, merge will only work for data parts that have the s To specify a partition in ALTER PARTITION commands, specify the value of the partition expression (or a tuple). Constants and constant expressions are supported. Example: -```sql +``` sql ALTER TABLE table DROP PARTITION (toMonday(today()), 1) ``` @@ -45,3 +45,5 @@ The partition ID is its string identifier (human-readable, if possible) that is For more examples, see the tests [`00502_custom_partitioning_local`](https://github.com/yandex/ClickHouse/blob/master/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.sql) and [`00502_custom_partitioning_replicated_zookeeper`](https://github.com/yandex/ClickHouse/blob/master/dbms/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql). + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/custom_partitioning_key/) diff --git a/docs/en/operations/table_engines/dictionary.md b/docs/en/operations/table_engines/dictionary.md index 6bf606ffa12..eed7f7afaf4 100644 --- a/docs/en/operations/table_engines/dictionary.md +++ b/docs/en/operations/table_engines/dictionary.md @@ -39,7 +39,7 @@ As an example, consider a dictionary of `products` with the following configurat Query the dictionary data: -```sql +``` sql select name, type, key, attribute.names, attribute.types, bytes_allocated, element_count,source from system.dictionaries where name = 'products'; SELECT @@ -73,7 +73,7 @@ CREATE TABLE %table_name% (%fields%) engine = Dictionary(%dictionary_name%)` Usage example: -```sql +``` sql create table products (product_id UInt64, title String) Engine = Dictionary(products); CREATE TABLE products @@ -92,7 +92,7 @@ Ok. Take a look at what's in the table. -```sql +``` sql select * from products limit 1; SELECT * @@ -108,3 +108,5 @@ LIMIT 1 1 rows in set. Elapsed: 0.006 sec. ``` + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/dictionary/) diff --git a/docs/en/operations/table_engines/distributed.md b/docs/en/operations/table_engines/distributed.md index 0bb55bc2603..6bd60c1591d 100644 --- a/docs/en/operations/table_engines/distributed.md +++ b/docs/en/operations/table_engines/distributed.md @@ -7,7 +7,7 @@ Reading is automatically parallelized. During a read, the table indexes on remot The Distributed engine accepts parameters: the cluster name in the server's config file, the name of a remote database, the name of a remote table, and (optionally) a sharding key. Example: -```text +``` Distributed(logs, default, hits[, sharding_key]) ``` @@ -122,3 +122,5 @@ If the server ceased to exist or had a rough restart (for example, after a devic When the max_parallel_replicas option is enabled, query processing is parallelized across all replicas within a single shard. For more information, see the section "Settings, max_parallel_replicas". + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/distributed/) diff --git a/docs/en/operations/table_engines/external_data.md b/docs/en/operations/table_engines/external_data.md index efb1f3c4fc4..ea27c9f0e1c 100644 --- a/docs/en/operations/table_engines/external_data.md +++ b/docs/en/operations/table_engines/external_data.md @@ -60,3 +60,5 @@ curl -F 'passwd=@passwd.tsv;' 'http://localhost:8123/?query=SELECT+shell,+count( For distributed query processing, the temporary tables are sent to all the remote servers. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/external_data/) diff --git a/docs/en/operations/table_engines/file.md b/docs/en/operations/table_engines/file.md index 8e31e346031..ed49a693630 100644 --- a/docs/en/operations/table_engines/file.md +++ b/docs/en/operations/table_engines/file.md @@ -31,7 +31,7 @@ You may manually create this subfolder and file in server filesystem and then [A **1.** Set up the `file_engine_table` table: -```sql +``` sql CREATE TABLE file_engine_table (name String, value UInt32) ENGINE=File(TabSeparated) ``` @@ -47,11 +47,11 @@ two 2 **3.** Query the data: -```sql +``` sql SELECT * FROM file_engine_table ``` -```text +``` ┌─name─┬─value─┐ │ one │ 1 │ │ two │ 2 │ @@ -76,3 +76,5 @@ $ echo -e "1,2\n3,4" | clickhouse-local -q "CREATE TABLE table (a Int64, b Int64 - `SELECT ... SAMPLE` - Indices - Replication + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/file/) diff --git a/docs/en/operations/table_engines/graphitemergetree.md b/docs/en/operations/table_engines/graphitemergetree.md index d1aaeca6aed..bc557594d76 100644 --- a/docs/en/operations/table_engines/graphitemergetree.md +++ b/docs/en/operations/table_engines/graphitemergetree.md @@ -27,7 +27,7 @@ The Graphite data table must contain the following fields at minimum: Rollup pattern: -```text +``` pattern regexp function @@ -84,3 +84,5 @@ Example of settings: ``` + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/graphitemergetree/) diff --git a/docs/en/operations/table_engines/index.md b/docs/en/operations/table_engines/index.md index eba6fffe37d..21909e3ce26 100644 --- a/docs/en/operations/table_engines/index.md +++ b/docs/en/operations/table_engines/index.md @@ -14,3 +14,5 @@ The table engine (type of table) determines: When reading, the engine is only required to output the requested columns, but in some cases the engine can partially process data when responding to the request. For most serious tasks, you should use engines from the `MergeTree` family. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/) diff --git a/docs/en/operations/table_engines/join.md b/docs/en/operations/table_engines/join.md index 1aff89ca887..299071312b8 100644 --- a/docs/en/operations/table_engines/join.md +++ b/docs/en/operations/table_engines/join.md @@ -2,7 +2,7 @@ A prepared data structure for JOIN that is always located in RAM. -```text +``` Join(ANY|ALL, LEFT|INNER, k1[, k2, ...]) ``` @@ -15,3 +15,5 @@ You can use INSERT to add data to the table, similar to the Set engine. For ANY, Storing data on the disk is the same as for the Set engine. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/join/) diff --git a/docs/en/operations/table_engines/kafka.md b/docs/en/operations/table_engines/kafka.md index eb79599e214..3ee70146409 100644 --- a/docs/en/operations/table_engines/kafka.md +++ b/docs/en/operations/table_engines/kafka.md @@ -44,7 +44,7 @@ Optional parameters: Examples: -```sql +``` sql CREATE TABLE queue ( timestamp UInt64, level String, @@ -86,7 +86,7 @@ When the `MATERIALIZED VIEW` joins the engine, it starts collecting data in the Example: -```sql +``` sql CREATE TABLE queue ( timestamp UInt64, level String, @@ -136,3 +136,5 @@ Similar to GraphiteMergeTree, the Kafka engine supports extended configuration u ``` For a list of possible configuration options, see the [librdkafka configuration reference](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md). Use the underscore (`_`) instead of a dot in the ClickHouse configuration. For example, `check.crcs=true` will be `true`. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/kafka/) diff --git a/docs/en/operations/table_engines/log.md b/docs/en/operations/table_engines/log.md index 406f94bd0d0..fffc5a11aca 100644 --- a/docs/en/operations/table_engines/log.md +++ b/docs/en/operations/table_engines/log.md @@ -4,3 +4,5 @@ Log differs from TinyLog in that a small file of "marks" resides with the column For concurrent data access, the read operations can be performed simultaneously, while write operations block reads and each other. The Log engine does not support indexes. Similarly, if writing to a table failed, the table is broken, and reading from it returns an error. The Log engine is appropriate for temporary data, write-once tables, and for testing or demonstration purposes. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/log/) diff --git a/docs/en/operations/table_engines/materializedview.md b/docs/en/operations/table_engines/materializedview.md index e2eb857aca7..c13a1ac8710 100644 --- a/docs/en/operations/table_engines/materializedview.md +++ b/docs/en/operations/table_engines/materializedview.md @@ -2,3 +2,5 @@ Used for implementing materialized views (for more information, see [CREATE TABLE](../../query_language/create.md#query_language-queries-create_table)). For storing data, it uses a different engine that was specified when creating the view. When reading from a table, it just uses this engine. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/materializedview/) diff --git a/docs/en/operations/table_engines/memory.md b/docs/en/operations/table_engines/memory.md index 113e4aa9f11..d68ae923684 100644 --- a/docs/en/operations/table_engines/memory.md +++ b/docs/en/operations/table_engines/memory.md @@ -9,3 +9,5 @@ Normally, using this table engine is not justified. However, it can be used for The Memory engine is used by the system for temporary tables with external query data (see the section "External data for processing a query"), and for implementing GLOBAL IN (see the section "IN operators"). + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/memory/) diff --git a/docs/en/operations/table_engines/merge.md b/docs/en/operations/table_engines/merge.md index e04dace12e0..e8de53bc286 100644 --- a/docs/en/operations/table_engines/merge.md +++ b/docs/en/operations/table_engines/merge.md @@ -65,3 +65,5 @@ The `Merge` type table contains a virtual `_table` column of the `String` type. If the `WHERE/PREWHERE` clause contains conditions for the `_table` column that do not depend on other table columns (as one of the conjunction elements, or as an entire expression), these conditions are used as an index. The conditions are performed on a data set of table names to read data from, and the read operation will be performed from only those tables that the condition was triggered on. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/merge/) diff --git a/docs/en/operations/table_engines/mergetree.md b/docs/en/operations/table_engines/mergetree.md index bc6c337e600..2f9fab70258 100644 --- a/docs/en/operations/table_engines/mergetree.md +++ b/docs/en/operations/table_engines/mergetree.md @@ -178,7 +178,7 @@ ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDa In this case, in queries: -```sql +``` sql SELECT count() FROM table WHERE EventDate = toDate(now()) AND CounterID = 34 SELECT count() FROM table WHERE EventDate = toDate(now()) AND (CounterID = 34 OR CounterID = 42) SELECT count() FROM table WHERE ((EventDate >= toDate('2014-01-01') AND EventDate <= toDate('2014-01-31')) OR EventDate = toDate('2014-05-01')) AND CounterID IN (101500, 731962, 160656) AND (CounterID = 101500 OR EventDate != toDate('2014-05-01')) @@ -190,7 +190,7 @@ The queries above show that the index is used even for complex expressions. Read In the example below, the index can't be used. -```sql +``` sql SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' ``` @@ -203,3 +203,6 @@ The key for partitioning by month allows reading only those data blocks which co For concurrent table access, we use multi-versioning. In other words, when a table is simultaneously read and updated, data is read from a set of parts that is current at the time of the query. There are no lengthy locks. Inserts do not get in the way of read operations. Reading from a table is automatically parallelized. + + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/mergetree/) diff --git a/docs/en/operations/table_engines/mysql.md b/docs/en/operations/table_engines/mysql.md index 06c0fd6b622..b9e419b5ac0 100644 --- a/docs/en/operations/table_engines/mysql.md +++ b/docs/en/operations/table_engines/mysql.md @@ -26,3 +26,5 @@ The rest of the conditions and the `LIMIT` sampling constraint are executed in C The `MySQL` engine does not support the [Nullable](../../data_types/nullable.md#data_type-nullable) data type, so when reading data from MySQL tables, `NULL` is converted to default values for the specified column type (usually 0 or an empty string). + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/mysql/) diff --git a/docs/en/operations/table_engines/null.md b/docs/en/operations/table_engines/null.md index 49850ec9864..58d3552d19d 100644 --- a/docs/en/operations/table_engines/null.md +++ b/docs/en/operations/table_engines/null.md @@ -4,3 +4,5 @@ When writing to a Null table, data is ignored. When reading from a Null table, t However, you can create a materialized view on a Null table. So the data written to the table will end up in the view. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/null/) diff --git a/docs/en/operations/table_engines/replacingmergetree.md b/docs/en/operations/table_engines/replacingmergetree.md index e96cb74c681..5add65e3339 100644 --- a/docs/en/operations/table_engines/replacingmergetree.md +++ b/docs/en/operations/table_engines/replacingmergetree.md @@ -51,4 +51,7 @@ CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] All of the parameters excepting `ver` have the same meaning as in `MergeTree`. + - `ver` - column with the version. Optional parameter. For a description, see the text above. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/replacingmergetree/) diff --git a/docs/en/operations/table_engines/replication.md b/docs/en/operations/table_engines/replication.md index 705dabe8e1b..9574cbaac82 100644 --- a/docs/en/operations/table_engines/replication.md +++ b/docs/en/operations/table_engines/replication.md @@ -78,7 +78,7 @@ Two parameters are also added in the beginning of the parameters list – the pa Example: -```text +``` ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/hits', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192) ``` @@ -180,3 +180,5 @@ After this, you can launch the server, create a `MergeTree` table, move the data ## Recovery When Metadata in The ZooKeeper Cluster is Lost or Damaged If the data in ZooKeeper was lost or damaged, you can save data by moving it to an unreplicated table as described above. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/replication/) diff --git a/docs/en/operations/table_engines/set.md b/docs/en/operations/table_engines/set.md index d207fe0d79e..75c1f3072d4 100644 --- a/docs/en/operations/table_engines/set.md +++ b/docs/en/operations/table_engines/set.md @@ -9,3 +9,5 @@ Data is always located in RAM. For INSERT, the blocks of inserted data are also For a rough server restart, the block of data on the disk might be lost or damaged. In the latter case, you may need to manually delete the file with damaged data. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/set/) diff --git a/docs/en/operations/table_engines/summingmergetree.md b/docs/en/operations/table_engines/summingmergetree.md index 73382e72075..d72922c1086 100644 --- a/docs/en/operations/table_engines/summingmergetree.md +++ b/docs/en/operations/table_engines/summingmergetree.md @@ -119,7 +119,7 @@ then this nested table is interpreted as a mapping of `key => (values...)`, and Examples: -```text +``` [(1, 100)] + [(2, 150)] -> [(1, 100), (2, 150)] [(1, 100)] + [(1, 150)] -> [(1, 250)] [(1, 100)] + [(1, 150), (2, 150)] -> [(1, 250), (2, 150)] @@ -129,3 +129,5 @@ Examples: When requesting data, use the [sumMap(key, value)](../../query_language/agg_functions/reference.md#agg_function-summary) function for aggregation of `Map`. For nested data structure, you do not need to specify its columns in the tuple of columns for summation. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/summingmergetree/) diff --git a/docs/en/operations/table_engines/tinylog.md b/docs/en/operations/table_engines/tinylog.md index 577596e2d96..6ec1cb8173a 100644 --- a/docs/en/operations/table_engines/tinylog.md +++ b/docs/en/operations/table_engines/tinylog.md @@ -17,3 +17,5 @@ The situation when you have a large number of small tables guarantees poor produ In Yandex.Metrica, TinyLog tables are used for intermediary data that is processed in small batches. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/tinylog/) diff --git a/docs/en/operations/table_engines/url.md b/docs/en/operations/table_engines/url.md index 9bd79ac1000..54d742e409a 100644 --- a/docs/en/operations/table_engines/url.md +++ b/docs/en/operations/table_engines/url.md @@ -23,7 +23,7 @@ respectively. For processing `POST` requests, the remote server must support **1.** Create a `url_engine_table` table on the server : -```sql +``` sql CREATE TABLE url_engine_table (word String, value UInt64) ENGINE=URL('http://127.0.0.1:12345/', CSV) ``` @@ -53,11 +53,11 @@ python3 server.py **3.** Request data: -```sql +``` sql SELECT * FROM url_engine_table ``` -```text +``` ┌─word──┬─value─┐ │ Hello │ 1 │ │ World │ 2 │ @@ -71,3 +71,5 @@ SELECT * FROM url_engine_table - `ALTER` and `SELECT...SAMPLE` operations. - Indexes. - Replication. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/url/) diff --git a/docs/en/operations/table_engines/view.md b/docs/en/operations/table_engines/view.md index 91f0ce9fdb3..c74eab262e4 100644 --- a/docs/en/operations/table_engines/view.md +++ b/docs/en/operations/table_engines/view.md @@ -2,3 +2,5 @@ Used for implementing views (for more information, see the `CREATE VIEW query`). It does not store data, but only stores the specified `SELECT` query. When reading from a table, it runs this query (and deletes all unnecessary columns from the query). + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/view/) diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index fe2e9670130..a34f79f869f 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -178,7 +178,7 @@ dynamicConfigFile=/etc/zookeeper-{{ cluster['name'] }}/conf/zoo.cfg.dynamic Java version: -```text +``` Java(TM) SE Runtime Environment (build 1.8.0_25-b17) Java HotSpot(TM) 64-Bit Server VM (build 25.25-b02, mixed mode) ``` @@ -226,7 +226,7 @@ JAVA_OPTS="-Xms{{ cluster.get('xms','128M') }} \ Salt init: -```text +``` description "zookeeper-{{ cluster['name'] }} centralized coordination service" start on runlevel [2345] @@ -255,3 +255,5 @@ script end script ``` + +[Original article](https://clickhouse.yandex/docs/en/operations/tips/) diff --git a/docs/en/operations/utils/clickhouse-copier.md b/docs/en/operations/utils/clickhouse-copier.md index 3109044724f..361834a681d 100644 --- a/docs/en/operations/utils/clickhouse-copier.md +++ b/docs/en/operations/utils/clickhouse-copier.md @@ -159,3 +159,5 @@ Parameters: `clickhouse-copier` tracks the changes in `/task/path/description` and applies them on the fly. For instance, if you change the value of `max_workers`, the number of processes running tasks will also change. + +[Original article](https://clickhouse.yandex/docs/en/operations/utils/clickhouse-copier/) diff --git a/docs/en/operations/utils/clickhouse-local.md b/docs/en/operations/utils/clickhouse-local.md index bfa612569f3..4b20473cb42 100644 --- a/docs/en/operations/utils/clickhouse-local.md +++ b/docs/en/operations/utils/clickhouse-local.md @@ -71,3 +71,5 @@ Read 186 rows, 4.15 KiB in 0.035 sec., 5302 rows/sec., 118.34 KiB/sec. ├──────────┼──────────┤ ... ``` + +[Original article](https://clickhouse.yandex/docs/en/operations/utils/clickhouse-local/) diff --git a/docs/en/operations/utils/index.md b/docs/en/operations/utils/index.md index 24faf79faa6..6406b486cc8 100644 --- a/docs/en/operations/utils/index.md +++ b/docs/en/operations/utils/index.md @@ -3,3 +3,5 @@ * [clickhouse-local](clickhouse-local.md#utils-clickhouse-local) — Allows running SQL queries on data without stopping the ClickHouse server, similar to how `awk` does this. * [clickhouse-copier](clickhouse-copier.md#utils-clickhouse-copier) — Copies (and reshards) data from one cluster to another cluster. + +[Original article](https://clickhouse.yandex/docs/en/operations/utils/) diff --git a/docs/en/query_language/agg_functions/combinators.md b/docs/en/query_language/agg_functions/combinators.md index e6cf9b70212..b63beee4a27 100644 --- a/docs/en/query_language/agg_functions/combinators.md +++ b/docs/en/query_language/agg_functions/combinators.md @@ -39,3 +39,6 @@ Merges the intermediate aggregation states in the same way as the -Merge combina ## -ForEach Converts an aggregate function for tables into an aggregate function for arrays that aggregates the corresponding array items and returns an array of results. For example, `sumForEach` for the arrays `[1, 2]`, `[3, 4, 5]`and`[6, 7]`returns the result `[10, 13, 5]` after adding together the corresponding array items. + + +[Original article](https://clickhouse.yandex/docs/en/query_language/agg_functions/combinators/) diff --git a/docs/en/query_language/agg_functions/index.md b/docs/en/query_language/agg_functions/index.md index 0f9cefc7b95..a3cb2132ed8 100644 --- a/docs/en/query_language/agg_functions/index.md +++ b/docs/en/query_language/agg_functions/index.md @@ -61,3 +61,5 @@ FROM t_null_big `groupArray` does not include `NULL` in the resulting array. + +[Original article](https://clickhouse.yandex/docs/en/query_language/agg_functions/) diff --git a/docs/en/query_language/agg_functions/parametric_functions.md b/docs/en/query_language/agg_functions/parametric_functions.md index 18f519035b0..134b4a4b985 100644 --- a/docs/en/query_language/agg_functions/parametric_functions.md +++ b/docs/en/query_language/agg_functions/parametric_functions.md @@ -23,7 +23,7 @@ Example: `sequenceMatch ('(?1).*(?2)')(EventTime, URL LIKE '%company%', URL LIKE This is a singular example. You could write it using other aggregate functions: -```text +``` minIf(EventTime, URL LIKE '%company%') < maxIf(EventTime, URL LIKE '%cart%'). ``` @@ -151,7 +151,9 @@ It works as fast as possible, except for cases when a large N value is used and Usage example: -```text +``` Problem: Generate a report that shows only keywords that produced at least 5 unique users. Solution: Write in the GROUP BY query SearchPhrase HAVING uniqUpTo(4)(UserID) >= 5 ``` + +[Original article](https://clickhouse.yandex/docs/en/query_language/agg_functions/parametric_functions/) diff --git a/docs/en/query_language/agg_functions/reference.md b/docs/en/query_language/agg_functions/reference.md index ce8e16e2581..c7268393408 100644 --- a/docs/en/query_language/agg_functions/reference.md +++ b/docs/en/query_language/agg_functions/reference.md @@ -37,7 +37,7 @@ anyHeavy(column) Take the [OnTime](../../getting_started/example_datasets/ontime.md#example_datasets-ontime) data set and select any frequently occurring value in the `AirlineID` column. -```sql +``` sql SELECT anyHeavy(AirlineID) AS res FROM ontime ``` @@ -107,7 +107,7 @@ Returns a tuple of two arrays: keys in sorted order, and values ​​summed for Example: -```sql +``` sql CREATE TABLE sum_map( date Date, timeslot DateTime, @@ -128,7 +128,7 @@ FROM sum_map GROUP BY timeslot ``` -```text +``` ┌────────────timeslot─┬─sumMap(statusMap.status, statusMap.requests)─┐ │ 2000-01-01 00:00:00 │ ([1,2,3,4,5],[10,10,20,10,10]) │ │ 2000-01-01 00:01:00 │ ([4,5,6,7,8],[10,10,20,10,10]) │ @@ -333,7 +333,7 @@ We recommend using the `N < 10 ` value; performance is reduced with large `N` va Take the [OnTime](../../getting_started/example_datasets/ontime.md#example_datasets-ontime) data set and select the three most frequently occurring values in the `AirlineID` column. -```sql +``` sql SELECT topK(3)(AirlineID) AS res FROM ontime ``` @@ -358,3 +358,5 @@ Calculates the value of `Σ((x - x̅)(y - y̅)) / n`. Calculates the Pearson correlation coefficient: `Σ((x - x̅)(y - y̅)) / sqrt(Σ((x - x̅)^2) * Σ((y - y̅)^2))`. + +[Original article](https://clickhouse.yandex/docs/en/query_language/agg_functions/reference/) diff --git a/docs/en/query_language/alter.md b/docs/en/query_language/alter.md index 8a6b0eb0874..cdc43271197 100644 --- a/docs/en/query_language/alter.md +++ b/docs/en/query_language/alter.md @@ -8,7 +8,7 @@ The `ALTER` query is only supported for `*MergeTree` tables, as well as `Merge`a Changing the table structure. -```sql +``` sql ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|MODIFY COLUMN ... ``` @@ -17,7 +17,7 @@ Each action is an operation on a column. The following actions are supported: -```sql +``` sql ADD COLUMN name [type] [default_expr] [AFTER name_after] ``` @@ -27,14 +27,14 @@ Adding a column just changes the table structure, without performing any actions This approach allows us to complete the ALTER query instantly, without increasing the volume of old data. -```sql +``` sql DROP COLUMN name ``` Deletes the column with the name 'name'. Deletes data from the file system. Since this deletes entire files, the query is completed almost instantly. -```sql +``` sql MODIFY COLUMN name [type] [default_expr] ``` @@ -86,7 +86,7 @@ A "part" in the table is part of the data from a single partition, sorted by the You can use the `system.parts` table to view the set of table parts and partitions: -```sql +``` sql SELECT * FROM system.parts WHERE active ``` @@ -123,7 +123,7 @@ For replicated tables, the set of parts can't be changed in any case. The `detached` directory contains parts that are not used by the server - detached from the table using the `ALTER ... DETACH` query. Parts that are damaged are also moved to this directory, instead of deleting them. You can add, delete, or modify the data in the 'detached' directory at any time – the server won't know about this until you make the `ALTER TABLE ... ATTACH` query. -```sql +``` sql ALTER TABLE [db.]table DETACH PARTITION 'name' ``` @@ -134,13 +134,13 @@ After the query is executed, you can do whatever you want with the data in the ' The query is replicated – data will be moved to the 'detached' directory and forgotten on all replicas. The query can only be sent to a leader replica. To find out if a replica is a leader, perform SELECT to the 'system.replicas' system table. Alternatively, it is easier to make a query on all replicas, and all except one will throw an exception. -```sql +``` sql ALTER TABLE [db.]table DROP PARTITION 'name' ``` The same as the `DETACH` operation. Deletes data from the table. Data parts will be tagged as inactive and will be completely deleted in approximately 10 minutes. The query is replicated – data will be deleted on all replicas. -```sql +``` sql ALTER TABLE [db.]table ATTACH PARTITION|PART 'name' ``` @@ -152,7 +152,7 @@ The query is replicated. Each replica checks whether there is data in the 'detac So you can put data in the 'detached' directory on one replica, and use the ALTER ... ATTACH query to add it to the table on all replicas. -```sql +``` sql ALTER TABLE [db.]table FREEZE PARTITION 'name' ``` @@ -196,7 +196,7 @@ For protection from device failures, you must use replication. For more informat Backups protect against human error (accidentally deleting data, deleting the wrong data or in the wrong cluster, or corrupting data). For high-volume databases, it can be difficult to copy backups to remote servers. In such cases, to protect from human error, you can keep a backup on the same server (it will reside in `/var/lib/clickhouse/shadow/`). -```sql +``` sql ALTER TABLE [db.]table FETCH PARTITION 'name' FROM 'path-in-zookeeper' ``` @@ -232,13 +232,13 @@ Existing tables are ready for mutations as-is (no conversion necessary), but aft Currently available commands: -```sql +``` sql ALTER TABLE [db.]table DELETE WHERE filter_expr ``` The `filter_expr` must be of type UInt8. The query deletes rows in the table for which this expression takes a non-zero value. -```sql +``` sql ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr ``` @@ -272,3 +272,5 @@ The table contains information about mutations of MergeTree tables and their pro **is_done** - Is the mutation done? Note that even if `parts_to_do = 0` it is possible that a mutation of a replicated table is not done yet because of a long-running INSERT that will create a new data part that will need to be mutated. + +[Original article](https://clickhouse.yandex/docs/en/query_language/alter/) diff --git a/docs/en/query_language/create.md b/docs/en/query_language/create.md index 0cd60df7e5d..8031d2fe580 100644 --- a/docs/en/query_language/create.md +++ b/docs/en/query_language/create.md @@ -2,7 +2,7 @@ Creating db_name databases -```sql +``` sql CREATE DATABASE [IF NOT EXISTS] db_name ``` @@ -15,7 +15,7 @@ If `IF NOT EXISTS` is included, the query won't return an error if the database The `CREATE TABLE` query can have several forms. -```sql +``` sql CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] ( name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], @@ -30,13 +30,13 @@ The structure of the table is a list of column descriptions. If indexes are supp A column description is `name type` in the simplest case. Example: `RegionID UInt32`. Expressions can also be defined for default values (see below). -```sql +``` sql CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name AS [db2.]name2 [ENGINE = engine] ``` Creates a table with the same structure as another table. You can specify a different engine for the table. If the engine is not specified, the same engine will be used as for the `db2.name2` table. -```sql +``` sql CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name ENGINE = engine AS SELECT ... ``` @@ -99,7 +99,7 @@ Distributed DDL queries (ON CLUSTER clause) The `CREATE`, `DROP`, `ALTER`, and `RENAME` queries support distributed execution on a cluster. For example, the following query creates the `all_hits` `Distributed` table on each host in `cluster`: -```sql +``` sql CREATE TABLE IF NOT EXISTS all_hits ON CLUSTER cluster (p Date, i Int32) ENGINE = Distributed(cluster, default, hits) ``` @@ -109,7 +109,7 @@ The local version of the query will eventually be implemented on each host in th ## CREATE VIEW -```sql +``` sql CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... ``` @@ -123,19 +123,19 @@ Normal views don't store any data, but just perform a read from another table. I As an example, assume you've created a view: -```sql +``` sql CREATE VIEW view AS SELECT ... ``` and written a query: -```sql +``` sql SELECT a, b, c FROM view ``` This query is fully equivalent to using the subquery: -```sql +``` sql SELECT a, b, c FROM (SELECT ...) ``` @@ -154,3 +154,5 @@ The execution of `ALTER` queries on materialized views has not been fully develo Views look the same as normal tables. For example, they are listed in the result of the `SHOW TABLES` query. There isn't a separate query for deleting views. To delete a view, use `DROP TABLE`. + +[Original article](https://clickhouse.yandex/docs/en/query_language/create/) diff --git a/docs/en/query_language/dicts/external_dicts.md b/docs/en/query_language/dicts/external_dicts.md index 7f1063a04e6..8840218e49f 100644 --- a/docs/en/query_language/dicts/external_dicts.md +++ b/docs/en/query_language/dicts/external_dicts.md @@ -41,3 +41,5 @@ See also "[Functions for working with external dictionaries](../functions/ext_di !!! attention You can convert values for a small dictionary by describing it in a `SELECT` query (see the [transform](../functions/other_functions.md#other_functions-transform) function). This functionality is not related to external dictionaries. + +[Original article](https://clickhouse.yandex/docs/en/query_language/dicts/external_dicts/) diff --git a/docs/en/query_language/dicts/external_dicts_dict.md b/docs/en/query_language/dicts/external_dicts_dict.md index daf00ce0250..ca070c8084b 100644 --- a/docs/en/query_language/dicts/external_dicts_dict.md +++ b/docs/en/query_language/dicts/external_dicts_dict.md @@ -31,3 +31,5 @@ The dictionary configuration has the following structure: - [layout](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout) — Dictionary layout in memory. - [structure](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure) — Structure of the dictionary . A key and attributes that can be retrieved by this key. - [lifetime](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime) — Frequency of dictionary updates. + +[Original article](https://clickhouse.yandex/docs/en/query_language/dicts/external_dicts_dict/) diff --git a/docs/en/query_language/dicts/external_dicts_dict_layout.md b/docs/en/query_language/dicts/external_dicts_dict_layout.md index d6802c4ccbf..932d90db605 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_layout.md +++ b/docs/en/query_language/dicts/external_dicts_dict_layout.md @@ -292,3 +292,5 @@ dictGetString('prefix', 'asn', tuple(IPv6StringToNum('2001:db8::1'))) Other types are not supported yet. The function returns the attribute for the prefix that corresponds to this IP address. If there are overlapping prefixes, the most specific one is returned. Data is stored in a `trie`. It must completely fit into RAM. + +[Original article](https://clickhouse.yandex/docs/en/query_language/dicts/external_dicts_dict_layout/) diff --git a/docs/en/query_language/dicts/external_dicts_dict_lifetime.md b/docs/en/query_language/dicts/external_dicts_dict_lifetime.md index 2fadfa51e0e..c04829fce10 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_lifetime.md +++ b/docs/en/query_language/dicts/external_dicts_dict_lifetime.md @@ -57,3 +57,5 @@ Example of settings: ``` + +[Original article](https://clickhouse.yandex/docs/en/query_language/dicts/external_dicts_dict_lifetime/) diff --git a/docs/en/query_language/dicts/external_dicts_dict_sources.md b/docs/en/query_language/dicts/external_dicts_dict_sources.md index 4c30852c770..977566afcc7 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_sources.md +++ b/docs/en/query_language/dicts/external_dicts_dict_sources.md @@ -111,7 +111,7 @@ Example of settings: ```xml DatabaseName - TableName
+ ShemaName.TableName
DSN=some_parameters SQL_QUERY
@@ -120,10 +120,11 @@ Example of settings: Setting fields: - `db` – Name of the database. Omit it if the database name is set in the `` parameters. -- `table` – Name of the table. +- `table` – Name of the table and schema if exists. - `connection_string` – Connection string. - `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime). +ClickHouse receives quoting symbols from ODBC-driver and quote all settings in queries to driver, so it's necessary to set table name accordingly to table name case in database. ### Known vulnerability of the ODBC dictionary functionality @@ -427,3 +428,5 @@ Setting fields: - `password` – Password of the MongoDB user. - `db` – Name of the database. - `collection` – Name of the collection. + +[Original article](https://clickhouse.yandex/docs/en/query_language/dicts/external_dicts_dict_sources/) diff --git a/docs/en/query_language/dicts/external_dicts_dict_structure.md b/docs/en/query_language/dicts/external_dicts_dict_structure.md index 06ab8e812ce..b499a474a4d 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_structure.md +++ b/docs/en/query_language/dicts/external_dicts_dict_structure.md @@ -115,3 +115,5 @@ Configuration fields: - `hierarchical` – Hierarchical support. Mirrored to the parent identifier. By default, ` false`. - `injective` – Whether the `id -> attribute` image is injective. If ` true`, then you can optimize the ` GROUP BY` clause. By default, `false`. - `is_object_id` – Whether the query is executed for a MongoDB document by `ObjectID`. + +[Original article](https://clickhouse.yandex/docs/en/query_language/dicts/external_dicts_dict_structure/) diff --git a/docs/en/query_language/dicts/index.md b/docs/en/query_language/dicts/index.md index 862dd686f2c..1c61a6534f6 100644 --- a/docs/en/query_language/dicts/index.md +++ b/docs/en/query_language/dicts/index.md @@ -11,3 +11,5 @@ ClickHouse supports: - [Built-in dictionaries](internal_dicts.md#internal_dicts) with a specific [set of functions](../functions/ym_dict_functions.md#ym_dict_functions). - [Plug-in (external) dictionaries](external_dicts.md#dicts-external_dicts) with a [set of functions](../functions/ext_dict_functions.md#ext_dict_functions). + +[Original article](https://clickhouse.yandex/docs/en/query_language/dicts/) diff --git a/docs/en/query_language/dicts/internal_dicts.md b/docs/en/query_language/dicts/internal_dicts.md index 9849bd21fb2..2b38ffe61f1 100644 --- a/docs/en/query_language/dicts/internal_dicts.md +++ b/docs/en/query_language/dicts/internal_dicts.md @@ -46,3 +46,5 @@ Dictionary updates (other than loading at first use) do not block queries. Durin We recommend periodically updating the dictionaries with the geobase. During an update, generate new files and write them to a separate location. When everything is ready, rename them to the files used by the server. There are also functions for working with OS identifiers and Yandex.Metrica search engines, but they shouldn't be used. + +[Original article](https://clickhouse.yandex/docs/en/query_language/dicts/internal_dicts/) diff --git a/docs/en/query_language/functions/arithmetic_functions.md b/docs/en/query_language/functions/arithmetic_functions.md index e457fe6684e..7420fbe532f 100644 --- a/docs/en/query_language/functions/arithmetic_functions.md +++ b/docs/en/query_language/functions/arithmetic_functions.md @@ -4,11 +4,11 @@ For all arithmetic functions, the result type is calculated as the smallest numb Example: -```sql +``` sql SELECT toTypeName(0), toTypeName(0 + 0), toTypeName(0 + 0 + 0), toTypeName(0 + 0 + 0 + 0) ``` -```text +``` ┌─toTypeName(0)─┬─toTypeName(plus(0, 0))─┬─toTypeName(plus(plus(0, 0), 0))─┬─toTypeName(plus(plus(plus(0, 0), 0), 0))─┐ │ UInt8 │ UInt16 │ UInt32 │ UInt64 │ └───────────────┴────────────────────────┴─────────────────────────────────┴──────────────────────────────────────────┘ @@ -73,3 +73,5 @@ An exception is thrown when dividing by zero or when dividing a minimal negative Returns the least common multiple of the numbers. An exception is thrown when dividing by zero or when dividing a minimal negative number by minus one. + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/arithmetic_functions/) diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index e250479e5b8..afb4f5f1e6e 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -59,7 +59,7 @@ arrayConcat(arrays) **Example** -```sql +``` sql SELECT arrayConcat([1, 2], [3, 4], [5, 6]) AS res ``` @@ -132,7 +132,7 @@ Returns the array \[1, 2, 3, ..., length (arr) \] This function is normally used with ARRAY JOIN. It allows counting something just once for each array after applying ARRAY JOIN. Example: -```sql +``` sql SELECT count() AS Reaches, countIf(num = 1) AS Hits @@ -144,7 +144,7 @@ WHERE CounterID = 160656 LIMIT 10 ``` -```text +``` ┌─Reaches─┬──Hits─┐ │ 95606 │ 31406 │ └─────────┴───────┘ @@ -152,7 +152,7 @@ LIMIT 10 In this example, Reaches is the number of conversions (the strings received after applying ARRAY JOIN), and Hits is the number of pageviews (strings before ARRAY JOIN). In this particular case, you can get the same result in an easier way: -```sql +``` sql SELECT sum(length(GoalsReached)) AS Reaches, count() AS Hits @@ -160,7 +160,7 @@ FROM test.hits WHERE (CounterID = 160656) AND notEmpty(GoalsReached) ``` -```text +``` ┌─Reaches─┬──Hits─┐ │ 95606 │ 31406 │ └─────────┴───────┘ @@ -176,7 +176,7 @@ For example: arrayEnumerateUniq(\[10, 20, 10, 30\]) = \[1, 1, 2, 1\]. This function is useful when using ARRAY JOIN and aggregation of array elements. Example: -```sql +``` sql SELECT Goals.ID AS GoalID, sum(Sign) AS Reaches, @@ -191,7 +191,7 @@ ORDER BY Reaches DESC LIMIT 10 ``` -```text +``` ┌──GoalID─┬─Reaches─┬─Visits─┐ │ 53225 │ 3214 │ 1097 │ │ 2825062 │ 3188 │ 1097 │ @@ -210,11 +210,11 @@ In this example, each goal ID has a calculation of the number of conversions (ea The arrayEnumerateUniq function can take multiple arrays of the same size as arguments. In this case, uniqueness is considered for tuples of elements in the same positions in all the arrays. -```sql +``` sql SELECT arrayEnumerateUniq([1, 1, 1, 2, 2, 2], [1, 1, 2, 1, 1, 2]) AS res ``` -```text +``` ┌─res───────────┐ │ [1,2,1,1,2,1] │ └───────────────┘ @@ -236,7 +236,7 @@ arrayPopBack(array) **Example** -```sql +``` sql SELECT arrayPopBack([1, 2, 3]) AS res ``` @@ -260,7 +260,7 @@ arrayPopFront(array) **Example** -```sql +``` sql SELECT arrayPopFront([1, 2, 3]) AS res ``` @@ -285,7 +285,7 @@ arrayPushBack(array, single_value) **Example** -```sql +``` sql SELECT arrayPushBack(['a'], 'b') AS res ``` @@ -310,7 +310,7 @@ arrayPushFront(array, single_value) **Example** -```sql +``` sql SELECT arrayPushBack(['b'], 'a') AS res ``` @@ -374,7 +374,7 @@ arraySlice(array, offset[, length]) **Example** -```sql +``` sql SELECT arraySlice([1, 2, NULL, 4, 5], 2, 3) AS res ``` @@ -396,3 +396,5 @@ If you want to get a list of unique items in an array, you can use arrayReduce(' ## arrayJoin(arr) A special function. See the section ["ArrayJoin function"](array_join.md#functions_arrayjoin). + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/array_functions/) diff --git a/docs/en/query_language/functions/array_join.md b/docs/en/query_language/functions/array_join.md index 6e18f8203c0..01b1d383e3f 100644 --- a/docs/en/query_language/functions/array_join.md +++ b/docs/en/query_language/functions/array_join.md @@ -17,11 +17,11 @@ Note the ARRAY JOIN syntax in the SELECT query, which provides broader possibili Example: -```sql +``` sql SELECT arrayJoin([1, 2, 3] AS src) AS dst, 'Hello', src ``` -```text +``` ┌─dst─┬─\'Hello\'─┬─src─────┐ │ 1 │ Hello │ [1,2,3] │ │ 2 │ Hello │ [1,2,3] │ @@ -29,3 +29,5 @@ SELECT arrayJoin([1, 2, 3] AS src) AS dst, 'Hello', src └─────┴───────────┴─────────┘ ``` + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/array_join/) diff --git a/docs/en/query_language/functions/bit_functions.md b/docs/en/query_language/functions/bit_functions.md index 523413f200a..1664664a6cf 100644 --- a/docs/en/query_language/functions/bit_functions.md +++ b/docs/en/query_language/functions/bit_functions.md @@ -16,3 +16,5 @@ The result type is an integer with bits equal to the maximum bits of its argumen ## bitShiftRight(a, b) + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/bit_functions/) diff --git a/docs/en/query_language/functions/comparison_functions.md b/docs/en/query_language/functions/comparison_functions.md index 9b95966ba84..39987ef2893 100644 --- a/docs/en/query_language/functions/comparison_functions.md +++ b/docs/en/query_language/functions/comparison_functions.md @@ -29,3 +29,5 @@ Note. Up until version 1.1.54134, signed and unsigned numbers were compared the ## greaterOrEquals, `>= operator` + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/comparison_functions/) diff --git a/docs/en/query_language/functions/conditional_functions.md b/docs/en/query_language/functions/conditional_functions.md index abd8c96e498..b5c7ce583cc 100644 --- a/docs/en/query_language/functions/conditional_functions.md +++ b/docs/en/query_language/functions/conditional_functions.md @@ -46,3 +46,5 @@ Run the query `SELECT multiIf(isNull(y) x, y < 3, y, NULL) FROM t_null`. Result: │ ᴺᵁᴸᴸ │ └────────────────────────────────────────────┘ ``` + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/conditional_functions/) diff --git a/docs/en/query_language/functions/date_time_functions.md b/docs/en/query_language/functions/date_time_functions.md index 8be3c14ffba..82821474431 100644 --- a/docs/en/query_language/functions/date_time_functions.md +++ b/docs/en/query_language/functions/date_time_functions.md @@ -4,7 +4,7 @@ Support for time zones All functions for working with the date and time that have a logical use for the time zone can accept a second optional time zone argument. Example: Asia/Yekaterinburg. In this case, they use the specified time zone instead of the local (default) one. -```sql +``` sql SELECT toDateTime('2016-06-15 23:00:00') AS time, toDate(time) AS date_local, @@ -12,7 +12,7 @@ SELECT toString(time, 'US/Samoa') AS time_samoa ``` -```text +``` ┌────────────────time─┬─date_local─┬─date_yekat─┬─time_samoa──────────┐ │ 2016-06-15 23:00:00 │ 2016-06-15 │ 2016-06-16 │ 2016-06-15 09:00:00 │ └─────────────────────┴────────────┴────────────┴─────────────────────┘ @@ -182,3 +182,5 @@ Supported modifiers for Format: |%y|Year, last two digits (00-99)|18| |%Y|Year|2018| |%%|a % sign|%| + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/date_time_functions/) diff --git a/docs/en/query_language/functions/encoding_functions.md b/docs/en/query_language/functions/encoding_functions.md index 27ba1554a15..74ef53f82f7 100644 --- a/docs/en/query_language/functions/encoding_functions.md +++ b/docs/en/query_language/functions/encoding_functions.md @@ -25,3 +25,5 @@ Accepts an integer. Returns a string containing the list of powers of two that t Accepts an integer. Returns an array of UInt64 numbers containing the list of powers of two that total the source number when summed. Numbers in the array are in ascending order. + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/encoding_functions/) diff --git a/docs/en/query_language/functions/ext_dict_functions.md b/docs/en/query_language/functions/ext_dict_functions.md index 68f5eaab7aa..ea692769404 100644 --- a/docs/en/query_language/functions/ext_dict_functions.md +++ b/docs/en/query_language/functions/ext_dict_functions.md @@ -45,3 +45,5 @@ The same as the `dictGetT` functions, but the default value is taken from the fu - Check whether the dictionary has the key. Returns a UInt8 value equal to 0 if there is no key and 1 if there is a key. + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/ext_dict_functions/) diff --git a/docs/en/query_language/functions/functions_for_nulls.md b/docs/en/query_language/functions/functions_for_nulls.md index c2a1e25aee7..d25de3332e5 100644 --- a/docs/en/query_language/functions/functions_for_nulls.md +++ b/docs/en/query_language/functions/functions_for_nulls.md @@ -292,3 +292,6 @@ SELECT toTypeName(toNullable(10)) │ Nullable(UInt8) │ └────────────────────────────┘ ``` + + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/functions_for_nulls/) diff --git a/docs/en/query_language/functions/geo.md b/docs/en/query_language/functions/geo.md index 91d07c3644a..05725b95aed 100644 --- a/docs/en/query_language/functions/geo.md +++ b/docs/en/query_language/functions/geo.md @@ -25,11 +25,11 @@ Generates an exception when the input parameter values fall outside of the range **Example** -```sql +``` sql SELECT greatCircleDistance(55.755831, 37.617673, -55.755831, -37.617673) ``` -```text +``` ┌─greatCircleDistance(55.755831, 37.617673, -55.755831, -37.617673)─┐ │ 14132374.194975413 │ └───────────────────────────────────────────────────────────────────┘ @@ -58,11 +58,11 @@ The input parameters must be `2+4⋅n`, where `n` is the number of ellipses. **Example** -```sql +``` sql SELECT pointInEllipses(55.755831, 37.617673, 55.755831, 37.617673, 1.0, 2.0) ``` -```text +``` ┌─pointInEllipses(55.755831, 37.617673, 55.755831, 37.617673, 1., 2.)─┐ │ 1 │ └─────────────────────────────────────────────────────────────────────┘ @@ -89,7 +89,7 @@ If the point is on the polygon boundary, the function may return either 0 or 1. **Example** -```sql +``` sql SELECT pointInPolygon((3., 3.), [(6, 0), (8, 4), (5, 8), (0, 2)]) AS res ``` @@ -99,3 +99,5 @@ SELECT pointInPolygon((3., 3.), [(6, 0), (8, 4), (5, 8), (0, 2)]) AS res └─────┘ ``` + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/geo/) diff --git a/docs/en/query_language/functions/hash_functions.md b/docs/en/query_language/functions/hash_functions.md index 42107ce5999..ffffe5584fc 100644 --- a/docs/en/query_language/functions/hash_functions.md +++ b/docs/en/query_language/functions/hash_functions.md @@ -64,3 +64,5 @@ A fast, decent-quality non-cryptographic hash function for a string obtained fro `URLHash(s, N)` – Calculates a hash from a string up to the N level in the URL hierarchy, without one of the trailing symbols `/`,`?` or `#` at the end, if present. Levels are the same as in URLHierarchy. This function is specific to Yandex.Metrica. + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/hash_functions/) diff --git a/docs/en/query_language/functions/higher_order_functions.md b/docs/en/query_language/functions/higher_order_functions.md index a80d5708e11..b00896cb4ab 100644 --- a/docs/en/query_language/functions/higher_order_functions.md +++ b/docs/en/query_language/functions/higher_order_functions.md @@ -22,17 +22,17 @@ Returns an array containing only the elements in 'arr1' for which 'func' returns Examples: -```sql +``` sql SELECT arrayFilter(x -> x LIKE '%World%', ['Hello', 'abc World']) AS res ``` -```text +``` ┌─res───────────┐ │ ['abc World'] │ └───────────────┘ ``` -```sql +``` sql SELECT arrayFilter( (i, x) -> x LIKE '%World%', @@ -41,7 +41,7 @@ SELECT AS res ``` -```text +``` ┌─res─┐ │ [2] │ └─────┘ @@ -77,11 +77,11 @@ Returns an array of partial sums of elements in the source array (a running sum) Example: -```sql +``` sql SELECT arrayCumSum([1, 1, 1, 1]) AS res ``` -```text +``` ┌─res──────────┐ │ [1, 2, 3, 4] │ └──────────────┘ @@ -95,11 +95,11 @@ The [Schwartzian transform](https://en.wikipedia.org/wiki/Schwartzian_transform) Example: -```sql +``` sql SELECT arraySort((x, y) -> y, ['hello', 'world'], [2, 1]); ``` -```text +``` ┌─res────────────────┐ │ ['world', 'hello'] │ └────────────────────┘ @@ -113,3 +113,5 @@ Returns an array as result of sorting the elements of `arr1` in descending order + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/higher_order_functions/) diff --git a/docs/en/query_language/functions/in_functions.md b/docs/en/query_language/functions/in_functions.md index 27ac483cdb5..b9295cac7d1 100644 --- a/docs/en/query_language/functions/in_functions.md +++ b/docs/en/query_language/functions/in_functions.md @@ -16,3 +16,5 @@ A function that allows getting a column from a tuple. 'N' is the column index, starting from 1. N must be a constant. 'N' must be a constant. 'N' must be a strict postive integer no greater than the size of the tuple. There is no cost to execute the function. + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/in_functions/) diff --git a/docs/en/query_language/functions/index.md b/docs/en/query_language/functions/index.md index 3700a277975..2e3c46779db 100644 --- a/docs/en/query_language/functions/index.md +++ b/docs/en/query_language/functions/index.md @@ -63,3 +63,5 @@ Another example is the `hostName` function, which returns the name of the server If a function in a query is performed on the requestor server, but you need to perform it on remote servers, you can wrap it in an 'any' aggregate function or add it to a key in `GROUP BY`. + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/) diff --git a/docs/en/query_language/functions/ip_address_functions.md b/docs/en/query_language/functions/ip_address_functions.md index f83f75319c7..27e1290c63c 100644 --- a/docs/en/query_language/functions/ip_address_functions.md +++ b/docs/en/query_language/functions/ip_address_functions.md @@ -14,7 +14,7 @@ Similar to IPv4NumToString, but using xxx instead of the last octet. Example: -```sql +``` sql SELECT IPv4NumToStringClassC(ClientIP) AS k, count() AS c @@ -24,7 +24,7 @@ ORDER BY c DESC LIMIT 10 ``` -```text +``` ┌─k──────────────┬─────c─┐ │ 83.149.9.xxx │ 26238 │ │ 217.118.81.xxx │ 26074 │ @@ -46,17 +46,17 @@ Since using 'xxx' is highly unusual, this may be changed in the future. We recom Accepts a FixedString(16) value containing the IPv6 address in binary format. Returns a string containing this address in text format. IPv6-mapped IPv4 addresses are output in the format ::ffff:111.222.33.44. Examples: -```sql +``` sql SELECT IPv6NumToString(toFixedString(unhex('2A0206B8000000000000000000000011'), 16)) AS addr ``` -```text +``` ┌─addr─────────┐ │ 2a02:6b8::11 │ └──────────────┘ ``` -```sql +``` sql SELECT IPv6NumToString(ClientIP6 AS k), count() AS c @@ -67,7 +67,7 @@ ORDER BY c DESC LIMIT 10 ``` -```text +``` ┌─IPv6NumToString(ClientIP6)──────────────┬─────c─┐ │ 2a02:2168:aaa:bbbb::2 │ 24695 │ │ 2a02:2698:abcd:abcd:abcd:abcd:8888:5555 │ 22408 │ @@ -82,7 +82,7 @@ LIMIT 10 └─────────────────────────────────────────┴───────┘ ``` -```sql +``` sql SELECT IPv6NumToString(ClientIP6 AS k), count() AS c @@ -93,7 +93,7 @@ ORDER BY c DESC LIMIT 10 ``` -```text +``` ┌─IPv6NumToString(ClientIP6)─┬──────c─┐ │ ::ffff:94.26.111.111 │ 747440 │ │ ::ffff:37.143.222.4 │ 529483 │ @@ -113,3 +113,5 @@ LIMIT 10 The reverse function of IPv6NumToString. If the IPv6 address has an invalid format, it returns a string of null bytes. HEX can be uppercase or lowercase. + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/ip_address_functions/) diff --git a/docs/en/query_language/functions/json_functions.md b/docs/en/query_language/functions/json_functions.md index 70f66d86b61..f28b329690d 100644 --- a/docs/en/query_language/functions/json_functions.md +++ b/docs/en/query_language/functions/json_functions.md @@ -35,7 +35,7 @@ Returns the value of a field, including separators. Examples: -```text +``` visitParamExtractRaw('{"abc":"\\n\\u0000"}', 'abc') = '"\\n\\u0000"' visitParamExtractRaw('{"abc":{"def":[1,2,3]}}', 'abc') = '{"def":[1,2,3]}' ``` @@ -46,7 +46,7 @@ Parses the string in double quotes. The value is unescaped. If unescaping failed Examples: -```text +``` visitParamExtractString('{"abc":"\\n\\u0000"}', 'abc') = '\n\0' visitParamExtractString('{"abc":"\\u263a"}', 'abc') = '☺' visitParamExtractString('{"abc":"\\u263"}', 'abc') = '' @@ -55,3 +55,5 @@ visitParamExtractString('{"abc":"hello}', 'abc') = '' There is currently no support for code points in the format `\uXXXX\uYYYY` that are not from the basic multilingual plane (they are converted to CESU-8 instead of UTF-8). + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/json_functions/) diff --git a/docs/en/query_language/functions/logical_functions.md b/docs/en/query_language/functions/logical_functions.md index 4ef0fe5fd32..45c722f52d7 100644 --- a/docs/en/query_language/functions/logical_functions.md +++ b/docs/en/query_language/functions/logical_functions.md @@ -12,3 +12,5 @@ Zero as an argument is considered "false," while any non-zero value is considere ## xor + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/logical_functions/) diff --git a/docs/en/query_language/functions/math_functions.md b/docs/en/query_language/functions/math_functions.md index 0ae7eb8427c..af4c9a30129 100644 --- a/docs/en/query_language/functions/math_functions.md +++ b/docs/en/query_language/functions/math_functions.md @@ -48,11 +48,11 @@ If 'x' is non-negative, then erf(x / σ√2) is the probability that a random Example (three sigma rule): -```sql +``` sql SELECT erf(3 / sqrt(2)) ``` -```text +``` ┌─erf(divide(3, sqrt(2)))─┐ │ 0.9973002039367398 │ └─────────────────────────┘ @@ -97,3 +97,5 @@ The arc tangent. ## pow(x, y) Takes two numeric arguments x and y. Returns a Float64 number close to x to the power of y. + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/math_functions/) diff --git a/docs/en/query_language/functions/other_functions.md b/docs/en/query_language/functions/other_functions.md index f3299bd5796..a8090fc30d4 100644 --- a/docs/en/query_language/functions/other_functions.md +++ b/docs/en/query_language/functions/other_functions.md @@ -83,7 +83,7 @@ The band is drawn with accuracy to one eighth of a symbol. Example: -```sql +``` sql SELECT toHour(EventTime) AS h, count() AS c, @@ -93,7 +93,7 @@ GROUP BY h ORDER BY h ASC ``` -```text +``` ┌──h─┬──────c─┬─bar────────────────┐ │ 0 │ 292907 │ █████████▋ │ │ 1 │ 180563 │ ██████ │ @@ -153,7 +153,7 @@ If the 'x' value is equal to one of the elements in the 'array_from' array, it r Example: -```sql +``` sql SELECT transform(SearchEngineID, [2, 3], ['Yandex', 'Google'], 'Other') AS title, count() AS c @@ -163,7 +163,7 @@ GROUP BY title ORDER BY c DESC ``` -```text +``` ┌─title─────┬──────c─┐ │ Yandex │ 498635 │ │ Google │ 229872 │ @@ -182,7 +182,7 @@ Types: Example: -```sql +``` sql SELECT transform(domain(Referer), ['yandex.ru', 'google.ru', 'vk.com'], ['www.yandex', 'example.com']) AS s, count() AS c @@ -192,7 +192,7 @@ ORDER BY count() DESC LIMIT 10 ``` -```text +``` ┌─s──────────────┬───────c─┐ │ │ 2906259 │ │ www.yandex │ 867767 │ @@ -212,13 +212,13 @@ Accepts the size (number of bytes). Returns a rounded size with a suffix (KiB, M Example: -```sql +``` sql SELECT arrayJoin([1, 1024, 1024*1024, 192851925]) AS filesize_bytes, formatReadableSize(filesize_bytes) AS filesize ``` -```text +``` ┌─filesize_bytes─┬─filesize───┐ │ 1 │ 1.00 B │ │ 1024 │ 1.00 KiB │ @@ -257,7 +257,7 @@ If you make a subquery with ORDER BY and call the function from outside the subq Example: -```sql +``` sql SELECT EventID, EventTime, @@ -274,7 +274,7 @@ FROM ) ``` -```text +``` ┌─EventID─┬───────────EventTime─┬─delta─┐ │ 1106 │ 2016-11-24 00:00:04 │ 0 │ │ 1107 │ 2016-11-24 00:00:05 │ 1 │ @@ -559,3 +559,5 @@ SELECT replicate(1, ['a', 'b', 'c']) └───────────────────────────────┘ ``` + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/other_functions/) diff --git a/docs/en/query_language/functions/random_functions.md b/docs/en/query_language/functions/random_functions.md index 915155f4af2..eca7e3279aa 100644 --- a/docs/en/query_language/functions/random_functions.md +++ b/docs/en/query_language/functions/random_functions.md @@ -16,3 +16,5 @@ Uses a linear congruential generator. Returns a pseudo-random UInt64 number, evenly distributed among all UInt64-type numbers. Uses a linear congruential generator. + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/random_functions/) diff --git a/docs/en/query_language/functions/rounding_functions.md b/docs/en/query_language/functions/rounding_functions.md index 916dd0a5082..17407aee852 100644 --- a/docs/en/query_language/functions/rounding_functions.md +++ b/docs/en/query_language/functions/rounding_functions.md @@ -31,7 +31,7 @@ The rounded number of the same type as the input number `x` **Example:** -```sql +``` sql SELECT number / 2 AS x, round(x) @@ -66,3 +66,5 @@ Accepts a number. If the number is less than one, it returns 0. Otherwise, it ro Accepts a number. If the number is less than 18, it returns 0. Otherwise, it rounds the number down to a number from the set: 18, 25, 35, 45, 55. This function is specific to Yandex.Metrica and used for implementing the report on user age. + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/rounding_functions/) diff --git a/docs/en/query_language/functions/splitting_merging_functions.md b/docs/en/query_language/functions/splitting_merging_functions.md index e2fb62ac479..0e1cf98ee20 100644 --- a/docs/en/query_language/functions/splitting_merging_functions.md +++ b/docs/en/query_language/functions/splitting_merging_functions.md @@ -26,4 +26,5 @@ SELECT alphaTokens('abca1abc') ┌─alphaTokens('abca1abc')─┐ │ ['abca','abc'] │ └─────────────────────────┘ -``` \ No newline at end of file +``` +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/splitting_merging_functions/) diff --git a/docs/en/query_language/functions/string_functions.md b/docs/en/query_language/functions/string_functions.md index 61e33040aa5..251fbd53a9b 100644 --- a/docs/en/query_language/functions/string_functions.md +++ b/docs/en/query_language/functions/string_functions.md @@ -74,3 +74,5 @@ If the 's' string is non-empty and does not contain the 'c' character at the end Returns the string 's' that was converted from the encoding in 'from' to the encoding in 'to'. + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/string_functions/) diff --git a/docs/en/query_language/functions/string_replace_functions.md b/docs/en/query_language/functions/string_replace_functions.md index d3773504278..400e4a7eff6 100644 --- a/docs/en/query_language/functions/string_replace_functions.md +++ b/docs/en/query_language/functions/string_replace_functions.md @@ -19,7 +19,7 @@ Also keep in mind that a string literal requires an extra escape. Example 1. Converting the date to American format: -```sql +``` sql SELECT DISTINCT EventDate, replaceRegexpOne(toString(EventDate), '(\\d{4})-(\\d{2})-(\\d{2})', '\\2/\\3/\\1') AS res @@ -28,7 +28,7 @@ LIMIT 7 FORMAT TabSeparated ``` -```text +``` 2014-03-17 03/17/2014 2014-03-18 03/18/2014 2014-03-19 03/19/2014 @@ -40,11 +40,11 @@ FORMAT TabSeparated Example 2. Copying a string ten times: -```sql +``` sql SELECT replaceRegexpOne('Hello, World!', '.*', '\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0') AS res ``` -```text +``` ┌─res────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ │ Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World! │ └────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ @@ -54,11 +54,11 @@ SELECT replaceRegexpOne('Hello, World!', '.*', '\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0') This does the same thing, but replaces all the occurrences. Example: -```sql +``` sql SELECT replaceRegexpAll('Hello, World!', '.', '\\0\\0') AS res ``` -```text +``` ┌─res────────────────────────┐ │ HHeelllloo,, WWoorrlldd!! │ └────────────────────────────┘ @@ -67,13 +67,15 @@ SELECT replaceRegexpAll('Hello, World!', '.', '\\0\\0') AS res As an exception, if a regular expression worked on an empty substring, the replacement is not made more than once. Example: -```sql +``` sql SELECT replaceRegexpAll('Hello, World!', '^', 'here: ') AS res ``` -```text +``` ┌─res─────────────────┐ │ here: Hello, World! │ └─────────────────────┘ ``` + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/string_replace_functions/) diff --git a/docs/en/query_language/functions/string_search_functions.md b/docs/en/query_language/functions/string_search_functions.md index a038162c023..182152e36ed 100644 --- a/docs/en/query_language/functions/string_search_functions.md +++ b/docs/en/query_language/functions/string_search_functions.md @@ -52,3 +52,5 @@ For other regular expressions, the code is the same as for the 'match' function. The same thing as 'like', but negative. + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/string_search_functions/) diff --git a/docs/en/query_language/functions/type_conversion_functions.md b/docs/en/query_language/functions/type_conversion_functions.md index 02014987e80..c4b4099d7e6 100644 --- a/docs/en/query_language/functions/type_conversion_functions.md +++ b/docs/en/query_language/functions/type_conversion_functions.md @@ -24,7 +24,7 @@ When converting dates with times to numbers or vice versa, the date with time co The date and date-with-time formats for the toDate/toDateTime functions are defined as follows: -```text +``` YYYY-MM-DD YYYY-MM-DD hh:mm:ss ``` @@ -37,13 +37,13 @@ Conversion between numeric types uses the same rules as assignments between diff Additionally, the toString function of the DateTime argument can take a second String argument containing the name of the time zone. Example: `Asia/Yekaterinburg` In this case, the time is formatted according to the specified time zone. -```sql +``` sql SELECT now() AS now_local, toString(now(), 'Asia/Yekaterinburg') AS now_yekat ``` -```text +``` ┌───────────now_local─┬─now_yekat───────────┐ │ 2016-06-15 00:11:21 │ 2016-06-15 02:11:21 │ └─────────────────────┴─────────────────────┘ @@ -62,21 +62,21 @@ Accepts a String or FixedString argument. Returns the String with the content tr Example: -```sql +``` sql SELECT toFixedString('foo', 8) AS s, toStringCutToZero(s) AS s_cut ``` -```text +``` ┌─s─────────────┬─s_cut─┐ │ foo\0\0\0\0\0 │ foo │ └───────────────┴───────┘ ``` -```sql +``` sql SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut ``` -```text +``` ┌─s──────────┬─s_cut─┐ │ foo\0bar\0 │ foo │ └────────────┴───────┘ @@ -102,7 +102,7 @@ Converts 'x' to the 't' data type. The syntax CAST(x AS t) is also supported. Example: -```sql +``` sql SELECT '2016-06-15 23:00:00' AS timestamp, CAST(timestamp AS DateTime) AS datetime, @@ -111,7 +111,7 @@ SELECT CAST(timestamp, 'FixedString(22)') AS fixed_string ``` -```text +``` ┌─timestamp───────────┬────────────datetime─┬───────date─┬─string──────────────┬─fixed_string──────────────┐ │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00 │ 2016-06-15 │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00\0\0\0 │ └─────────────────────┴─────────────────────┴────────────┴─────────────────────┴───────────────────────────┘ @@ -137,3 +137,5 @@ SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null └─────────────────────────────────────────┘ ``` + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/type_conversion_functions/) diff --git a/docs/en/query_language/functions/url_functions.md b/docs/en/query_language/functions/url_functions.md index ef0c7e08158..efe48970338 100644 --- a/docs/en/query_language/functions/url_functions.md +++ b/docs/en/query_language/functions/url_functions.md @@ -72,7 +72,7 @@ Returns an array containing the URL, truncated at the end by the symbols /,? in The same as above, but without the protocol and host in the result. The / element (root) is not included. Example: the function is used to implement tree reports the URL in Yandex. Metric. -```text +``` URLPathHierarchy('https://example.com/browse/CONV-6788') = [ '/browse/', @@ -85,11 +85,11 @@ URLPathHierarchy('https://example.com/browse/CONV-6788') = Returns the decoded URL. Example: -```sql +``` sql SELECT decodeURLComponent('http://127.0.0.1:8123/?query=SELECT%201%3B') AS DecodedURL; ``` -```text +``` ┌─DecodedURL─────────────────────────────┐ │ http://127.0.0.1:8123/?query=SELECT 1; │ └────────────────────────────────────────┘ @@ -119,3 +119,5 @@ Removes the query string and fragment identifier. The question mark and number s Removes the 'name' URL parameter, if present. This function works under the assumption that the parameter name is encoded in the URL exactly the same way as in the passed argument. + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/url_functions/) diff --git a/docs/en/query_language/functions/ym_dict_functions.md b/docs/en/query_language/functions/ym_dict_functions.md index 7ba7e7012cf..8039c77edea 100644 --- a/docs/en/query_language/functions/ym_dict_functions.md +++ b/docs/en/query_language/functions/ym_dict_functions.md @@ -20,7 +20,7 @@ All the dictionaries are re-loaded in runtime (once every certain number of seco All functions for working with regions have an optional argument at the end – the dictionary key. It is referred to as the geobase. Example: -```text +``` regionToCountry(RegionID) – Uses the default dictionary: /opt/geo/regions_hierarchy.txt regionToCountry(RegionID, '') – Uses the default dictionary: /opt/geo/regions_hierarchy.txt regionToCountry(RegionID, 'ua') – Uses the dictionary for the 'ua' key: /opt/geo/regions_hierarchy_ua.txt @@ -34,13 +34,13 @@ Accepts a UInt32 number – the region ID from the Yandex geobase. If this regio Converts a region to an area (type 5 in the geobase). In every other way, this function is the same as 'regionToCity'. -```sql +``` sql SELECT DISTINCT regionToName(regionToArea(toUInt32(number), 'ua')) FROM system.numbers LIMIT 15 ``` -```text +``` ┌─regionToName(regionToArea(toUInt32(number), \'ua\'))─┐ │ │ │ Moscow and Moscow region │ @@ -64,13 +64,13 @@ LIMIT 15 Converts a region to a federal district (type 4 in the geobase). In every other way, this function is the same as 'regionToCity'. -```sql +``` sql SELECT DISTINCT regionToName(regionToDistrict(toUInt32(number), 'ua')) FROM system.numbers LIMIT 15 ``` -```text +``` ┌─regionToName(regionToDistrict(toUInt32(number), \'ua\'))─┐ │ │ │ Central federal district │ @@ -123,3 +123,5 @@ Accepts a UInt32 number – the region ID from the Yandex geobase. A string with `ua` and `uk` both mean Ukrainian. + +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/ym_dict_functions/) diff --git a/docs/en/query_language/index.md b/docs/en/query_language/index.md index dcb59cbe6ad..bc8b8cfd876 100644 --- a/docs/en/query_language/index.md +++ b/docs/en/query_language/index.md @@ -6,3 +6,5 @@ * [ALTER](alter.md#query_language_queries_alter) * [Other types of queries](misc.md#miscellanous-queries) + +[Original article](https://clickhouse.yandex/docs/en/query_language/) diff --git a/docs/en/query_language/insert_into.md b/docs/en/query_language/insert_into.md index 99e11123507..d34e306777b 100644 --- a/docs/en/query_language/insert_into.md +++ b/docs/en/query_language/insert_into.md @@ -6,7 +6,7 @@ Adding data. Basic query format: -```sql +``` sql INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... ``` @@ -19,13 +19,13 @@ If [strict_insert_defaults=1](../operations/settings/settings.md#settings-strict Data can be passed to the INSERT in any [format](../interfaces/formats.md#formats) supported by ClickHouse. The format must be specified explicitly in the query: -```sql +``` sql INSERT INTO [db.]table [(c1, c2, c3)] FORMAT format_name data_set ``` For example, the following query format is identical to the basic version of INSERT ... VALUES: -```sql +``` sql INSERT INTO [db.]table [(c1, c2, c3)] FORMAT Values (v11, v12, v13), (v21, v22, v23), ... ``` @@ -33,7 +33,7 @@ ClickHouse removes all spaces and one line feed (if there is one) before the dat Example: -```sql +``` sql INSERT INTO t FORMAT TabSeparated 11 Hello, world! 22 Qwerty @@ -43,7 +43,7 @@ You can insert data separately from the query by using the command-line client o ### Inserting The Results of `SELECT` -```sql +``` sql INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... ``` @@ -66,3 +66,5 @@ Performance will not decrease if: - Data is added in real time. - You upload data that is usually sorted by time. + +[Original article](https://clickhouse.yandex/docs/en/query_language/insert_into/) diff --git a/docs/en/query_language/misc.md b/docs/en/query_language/misc.md index 253c1d59ef4..e81deca4b49 100644 --- a/docs/en/query_language/misc.md +++ b/docs/en/query_language/misc.md @@ -10,7 +10,7 @@ After executing an ATTACH query, the server will know about the existence of the If the table was previously detached (``DETACH``), meaning that its structure is known, you can use shorthand without defining the structure. -```sql +``` sql ATTACH TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] ``` @@ -20,14 +20,14 @@ This query is used when starting the server. The server stores table metadata as This query has two types: `DROP DATABASE` and `DROP TABLE`. -```sql +``` sql DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster] ``` Deletes all tables inside the 'db' database, then deletes the 'db' database itself. If `IF EXISTS` is specified, it doesn't return an error if the database doesn't exist. -```sql +``` sql DROP [TEMPORARY] TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] ``` @@ -38,7 +38,7 @@ If `IF EXISTS` is specified, it doesn't return an error if the table doesn't exi Deletes information about the 'name' table from the server. The server stops knowing about the table's existence. -```sql +``` sql DETACH TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] ``` @@ -51,7 +51,7 @@ There is no `DETACH DATABASE` query. Renames one or more tables. -```sql +``` sql RENAME TABLE [db11.]name11 TO [db12.]name12, [db21.]name21 TO [db22.]name22, ... [ON CLUSTER cluster] ``` @@ -59,7 +59,7 @@ All tables are renamed under global locking. Renaming tables is a light operatio ## SHOW DATABASES -```sql +``` sql SHOW DATABASES [INTO OUTFILE filename] [FORMAT format] ``` @@ -70,7 +70,7 @@ See also the section "Formats". ## SHOW TABLES -```sql +``` sql SHOW [TEMPORARY] TABLES [FROM db] [LIKE 'pattern'] [INTO OUTFILE filename] [FORMAT format] ``` @@ -85,7 +85,7 @@ See also the section "LIKE operator". ## SHOW PROCESSLIST -```sql +``` sql SHOW PROCESSLIST [INTO OUTFILE filename] [FORMAT format] ``` @@ -117,7 +117,7 @@ watch -n1 "clickhouse-client --query='SHOW PROCESSLIST'" ## SHOW CREATE TABLE -```sql +``` sql SHOW CREATE [TEMPORARY] TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] ``` @@ -125,7 +125,7 @@ Returns a single `String`-type 'statement' column, which contains a single value ## DESCRIBE TABLE -```sql +``` sql DESC|DESCRIBE TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] ``` @@ -135,7 +135,7 @@ Nested data structures are output in "expanded" format. Each column is shown sep ## EXISTS -```sql +``` sql EXISTS [TEMPORARY] TABLE [db.]name [INTO OUTFILE filename] [FORMAT format] ``` @@ -143,7 +143,7 @@ Returns a single `UInt8`-type column, which contains the single value `0` if the ## USE -```sql +``` sql USE db ``` @@ -153,7 +153,7 @@ This query can't be made when using the HTTP protocol, since there is no concept ## SET -```sql +``` sql SET param = value ``` @@ -166,7 +166,7 @@ To make settings that persist after a server restart, you can only use the serve ## OPTIMIZE -```sql +``` sql OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition] [FINAL] ``` @@ -180,7 +180,7 @@ If you specify `FINAL`, optimization will be performed even when all the data is ## KILL QUERY -```sql +``` sql KILL QUERY [ON CLUSTER cluster] WHERE [SYNC|ASYNC|TEST] @@ -192,7 +192,7 @@ The queries to terminate are selected from the system.processes table using the Examples: -```sql +``` sql -- Forcibly terminates all queries with the specified query_id: KILL QUERY WHERE query_id='2-857d-4a57-9ee0-327da5d60a90' @@ -212,3 +212,5 @@ The response contains the `kill_status` column, which can take the following val 3. The other values ​​explain why the query can't be stopped. A test query (`TEST`) only checks the user's rights and displays a list of queries to stop. + +[Original article](https://clickhouse.yandex/docs/en/query_language/misc/) diff --git a/docs/en/query_language/operators.md b/docs/en/query_language/operators.md index 5d3b6df4ca6..613dc50a58b 100644 --- a/docs/en/query_language/operators.md +++ b/docs/en/query_language/operators.md @@ -87,7 +87,7 @@ The conditional operator calculates the values of b and c, then checks whether c ## Conditional Expression -```sql +``` sql CASE [x] WHEN a THEN b [WHEN ... THEN ...] @@ -171,3 +171,5 @@ WHERE isNotNull(y) 1 rows in set. Elapsed: 0.002 sec. ``` + +[Original article](https://clickhouse.yandex/docs/en/query_language/operators/) diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index 39f261ec2ab..53164e67d80 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -2,7 +2,7 @@ `SELECT` performs data retrieval. -```sql +``` sql SELECT [DISTINCT] expr_list [FROM [db.]table | (subquery) | table_function] [FINAL] [SAMPLE sample_coeff] @@ -55,7 +55,7 @@ In the second case, the query will be executed on a sample of no more than 'n' r Example: -```sql +``` sql SELECT Title, count() * 10 AS PageViews @@ -86,7 +86,7 @@ Allows executing JOIN with an array or nested data structure. The intent is simi `ARRAY JOIN` is essentially `INNER JOIN` with an array. Example: -```text +``` :) CREATE TABLE arrays_test (s String, arr Array(UInt8)) ENGINE = Memory CREATE TABLE arrays_test @@ -139,7 +139,7 @@ ARRAY JOIN arr An alias can be specified for an array in the ARRAY JOIN clause. In this case, an array item can be accessed by this alias, but the array itself by the original name. Example: -```text +``` :) SELECT s, arr, a FROM arrays_test ARRAY JOIN arr AS a SELECT s, arr, a @@ -159,7 +159,7 @@ ARRAY JOIN arr AS a Multiple arrays of the same size can be comma-separated in the ARRAY JOIN clause. In this case, JOIN is performed with them simultaneously (the direct sum, not the direct product). Example: -```text +``` :) SELECT s, arr, a, num, mapped FROM arrays_test ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num, arrayMap(x -> x + 1, arr) AS mapped SELECT s, arr, a, num, mapped @@ -195,7 +195,7 @@ ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num ARRAY JOIN also works with nested data structures. Example: -```text +``` :) CREATE TABLE nested_test (s String, nest Nested(x UInt8, y UInt32)) ENGINE = Memory CREATE TABLE nested_test @@ -250,7 +250,7 @@ ARRAY JOIN nest When specifying names of nested data structures in ARRAY JOIN, the meaning is the same as ARRAY JOIN with all the array elements that it consists of. Example: -```text +``` :) SELECT s, nest.x, nest.y FROM nested_test ARRAY JOIN nest.x, nest.y SELECT s, `nest.x`, `nest.y` @@ -270,7 +270,7 @@ ARRAY JOIN `nest.x`, `nest.y` This variation also makes sense: -```text +``` :) SELECT s, nest.x, nest.y FROM nested_test ARRAY JOIN nest.x SELECT s, `nest.x`, `nest.y` @@ -290,7 +290,7 @@ ARRAY JOIN `nest.x` An alias may be used for a nested data structure, in order to select either the JOIN result or the source array. Example: -```text +``` :) SELECT s, n.x, n.y, nest.x, nest.y FROM nested_test ARRAY JOIN nest AS n SELECT s, `n.x`, `n.y`, `nest.x`, `nest.y` @@ -310,7 +310,7 @@ ARRAY JOIN nest AS n Example of using the arrayEnumerate function: -```text +``` :) SELECT s, n.x, n.y, nest.x, nest.y, num FROM nested_test ARRAY JOIN nest AS n, arrayEnumerate(nest.x) AS num SELECT s, `n.x`, `n.y`, `nest.x`, `nest.y`, num @@ -336,7 +336,7 @@ The corresponding conversion can be performed before the WHERE/PREWHERE clause ( The normal JOIN, which is not related to ARRAY JOIN described above. -```sql +``` sql [GLOBAL] ANY|ALL INNER|LEFT [OUTER] JOIN (subquery)|table USING columns_list ``` @@ -371,7 +371,7 @@ When running a JOIN, there is no optimization of the order of execution in relat Example: -```sql +``` sql SELECT CounterID, hits, @@ -395,7 +395,7 @@ ORDER BY hits DESC LIMIT 10 ``` -```text +``` ┌─CounterID─┬───hits─┬─visits─┐ │ 1143050 │ 523264 │ 13665 │ │ 731962 │ 475698 │ 102716 │ @@ -469,7 +469,7 @@ If a query contains only table columns inside aggregate functions, the GROUP BY Example: -```sql +``` sql SELECT count(), median(FetchTiming > 60 ? 60 : FetchTiming), @@ -483,7 +483,7 @@ As opposed to MySQL (and conforming to standard SQL), you can't get some value o Example: -```sql +``` sql SELECT domainWithoutWWW(URL) AS domain, count(), @@ -579,7 +579,7 @@ LIMIT N BY COLUMNS selects the top N rows for each group of COLUMNS. LIMIT N BY Example: -```sql +``` sql SELECT domainWithoutWWW(URL) AS domain, domainWithoutWWW(REFERRER_URL) AS referrer, @@ -698,7 +698,7 @@ If there isn't an ORDER BY clause that explicitly sorts results, the result may You can use UNION ALL to combine any number of queries. Example: -```sql +``` sql SELECT CounterID, 1 AS table, toInt64(count()) AS c FROM test.hits GROUP BY CounterID @@ -746,7 +746,7 @@ The left side of the operator is either a single column or a tuple. Examples: -```sql +``` sql SELECT UserID IN (123, 456) FROM ... SELECT (CounterID, UserID) IN ((34, 123), (101500, 456)) FROM ... ``` @@ -764,7 +764,7 @@ If the right side of the operator is a table name that has the Set engine (a pre The subquery may specify more than one column for filtering tuples. Example: -```sql +``` sql SELECT (CounterID, UserID) IN (SELECT CounterID, UserID FROM ...) FROM ... ``` @@ -773,7 +773,7 @@ The columns to the left and right of the IN operator should have the same type. The IN operator and subquery may occur in any part of the query, including in aggregate functions and lambda functions. Example: -```sql +``` sql SELECT EventDate, avg(UserID IN @@ -787,7 +787,7 @@ GROUP BY EventDate ORDER BY EventDate ASC ``` -```text +``` ┌──EventDate─┬────ratio─┐ │ 2014-03-17 │ 1 │ │ 2014-03-18 │ 0.807696 │ @@ -858,13 +858,13 @@ For a query to the **distributed_table**, the query will be sent to all the remo For example, the query -```sql +``` sql SELECT uniq(UserID) FROM distributed_table ``` will be sent to all remote servers as -```sql +``` sql SELECT uniq(UserID) FROM local_table ``` @@ -872,7 +872,7 @@ and run on each of them in parallel, until it reaches the stage where intermedia Now let's examine a query with IN: -```sql +``` sql SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM local_table WHERE CounterID = 34) ``` @@ -880,7 +880,7 @@ SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID I This query will be sent to all remote servers as -```sql +``` sql SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM local_table WHERE CounterID = 34) ``` @@ -890,19 +890,19 @@ This will work correctly and optimally if you are prepared for this case and hav To correct how the query works when data is spread randomly across the cluster servers, you could specify **distributed_table** inside a subquery. The query would look like this: -```sql +``` sql SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) ``` This query will be sent to all remote servers as -```sql +``` sql SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) ``` The subquery will begin running on each remote server. Since the subquery uses a distributed table, the subquery that is on each remote server will be resent to every remote server as -```sql +``` sql SELECT UserID FROM local_table WHERE CounterID = 34 ``` @@ -910,19 +910,19 @@ For example, if you have a cluster of 100 servers, executing the entire query wi In such cases, you should always use GLOBAL IN instead of IN. Let's look at how it works for the query -```sql +``` sql SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID GLOBAL IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) ``` The requestor server will run the subquery -```sql +``` sql SELECT UserID FROM distributed_table WHERE CounterID = 34 ``` and the result will be put in a temporary table in RAM. Then the request will be sent to each remote server as -```sql +``` sql SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID GLOBAL IN _data1 ``` @@ -964,3 +964,5 @@ You can put an asterisk in any part of a query instead of an expression. When th - In subqueries (since columns that aren't needed for the external query are excluded from subqueries). In all other cases, we don't recommend using the asterisk, since it only gives you the drawbacks of a columnar DBMS instead of the advantages. In other words using the asterisk is not recommended. + +[Original article](https://clickhouse.yandex/docs/en/query_language/select/) diff --git a/docs/en/query_language/syntax.md b/docs/en/query_language/syntax.md index 11ffb2e6a91..548a4771a40 100644 --- a/docs/en/query_language/syntax.md +++ b/docs/en/query_language/syntax.md @@ -4,7 +4,7 @@ There are two types of parsers in the system: the full SQL parser (a recursive d In all cases except the INSERT query, only the full SQL parser is used. The INSERT query uses both parsers: -```sql +``` sql INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') ``` @@ -103,7 +103,7 @@ Data types and table engines in the `CREATE` query are written the same way as i In the SELECT query, expressions can specify synonyms using the AS keyword. Any expression is placed to the left of AS. The identifier name for the synonym is placed to the right of AS. As opposed to standard SQL, synonyms are not only declared on the top level of expressions: -```sql +``` sql SELECT (1 AS n) + 2, n ``` @@ -119,3 +119,5 @@ An expression is a function, identifier, literal, application of an operator, ex A list of expressions is one or more expressions separated by commas. Functions and operators, in turn, can have expressions as arguments. + +[Original article](https://clickhouse.yandex/docs/en/query_language/syntax/) diff --git a/docs/en/query_language/table_functions/file.md b/docs/en/query_language/table_functions/file.md index a6b9006d69f..e7c7e5b8e89 100644 --- a/docs/en/query_language/table_functions/file.md +++ b/docs/en/query_language/table_functions/file.md @@ -47,3 +47,9 @@ LIMIT 2 └─────────┴─────────┴─────────┘ ``` +``` sql +-- getting the first 10 lines of a table that contains 3 columns of UInt32 type from a CSV file +SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') LIMIT 10 +``` + +[Original article](https://clickhouse.yandex/docs/en/query_language/table_functions/file/) diff --git a/docs/en/query_language/table_functions/index.md b/docs/en/query_language/table_functions/index.md index fd479dd3faf..f694c5e3518 100644 --- a/docs/en/query_language/table_functions/index.md +++ b/docs/en/query_language/table_functions/index.md @@ -4,3 +4,5 @@ Table functions can be specified in the FROM clause instead of the database and Table functions can only be used if 'readonly' is not set. Table functions aren't related to other functions. + +[Original article](https://clickhouse.yandex/docs/en/query_language/table_functions/) diff --git a/docs/en/query_language/table_functions/jdbc.md b/docs/en/query_language/table_functions/jdbc.md new file mode 100644 index 00000000000..240e12551d8 --- /dev/null +++ b/docs/en/query_language/table_functions/jdbc.md @@ -0,0 +1,25 @@ + + +# jdbc + +`jdbc(jdbc_connection_uri, schema, table)` - returns table that is connected via JDBC driver. + +This table function requires separate `clickhouse-jdbc-bridge` program to be running. +It supports Nullable types (based on DDL of remote table that is queried). + + +**Examples** + +``` sql +SELECT * FROM jdbc('jdbc:mysql://localhost:3306/?user=root&password=root', 'schema', 'table') +``` + +``` sql +SELECT * FROM jdbc('mysql://localhost:3306/?user=root&password=root', 'schema', 'table') +``` + +``` sql +SELECT * FROM jdbc('datasource://mysql-local', 'schema', 'table') +``` + +[Original article](https://clickhouse.yandex/docs/en/query_language/table_functions/jdbc/) diff --git a/docs/en/query_language/table_functions/merge.md b/docs/en/query_language/table_functions/merge.md index d1ddc12dc97..37a206d36a5 100644 --- a/docs/en/query_language/table_functions/merge.md +++ b/docs/en/query_language/table_functions/merge.md @@ -4,3 +4,5 @@ The table structure is taken from the first table encountered that matches the regular expression. + +[Original article](https://clickhouse.yandex/docs/en/query_language/table_functions/merge/) diff --git a/docs/en/query_language/table_functions/numbers.md b/docs/en/query_language/table_functions/numbers.md index 4486fece3d1..e8b025d922e 100644 --- a/docs/en/query_language/table_functions/numbers.md +++ b/docs/en/query_language/table_functions/numbers.md @@ -7,7 +7,7 @@ Similar to the `system.numbers` table, it can be used for testing and generating The following queries are equivalent: -```sql +``` sql SELECT * FROM numbers(10); SELECT * FROM numbers(0, 10); SELECT * FROM system.numbers LIMIT 10; @@ -15,8 +15,10 @@ SELECT * FROM system.numbers LIMIT 10; Examples: -```sql +``` sql -- Generate a sequence of dates from 2010-01-01 to 2010-12-31 select toDate('2010-01-01') + number as d FROM numbers(365); ``` + +[Original article](https://clickhouse.yandex/docs/en/query_language/table_functions/numbers/) diff --git a/docs/en/query_language/table_functions/remote.md b/docs/en/query_language/table_functions/remote.md index 425c6f81a7d..834dbf30be7 100644 --- a/docs/en/query_language/table_functions/remote.md +++ b/docs/en/query_language/table_functions/remote.md @@ -6,7 +6,7 @@ Allows you to access remote servers without creating a `Distributed` table. Signatures: -```sql +``` sql remote('addresses_expr', db, table[, 'user'[, 'password']]) remote('addresses_expr', db.table[, 'user'[, 'password']]) ``` @@ -18,7 +18,7 @@ remote('addresses_expr', db.table[, 'user'[, 'password']]) Examples: -```text +``` example01-01-1 example01-01-1:9000 localhost @@ -31,19 +31,19 @@ Multiple addresses can be comma-separated. In this case, ClickHouse will use dis Example: -```text +``` example01-01-1,example01-02-1 ``` Part of the expression can be specified in curly brackets. The previous example can be written as follows: -```text +``` example01-0{1,2}-1 ``` Curly brackets can contain a range of numbers separated by two dots (non-negative integers). In this case, the range is expanded to a set of values that generate shard addresses. If the first number starts with zero, the values are formed with the same zero alignment. The previous example can be written as follows: -```text +``` example01-{01..02}-1 ``` @@ -53,7 +53,7 @@ Addresses and parts of addresses in curly brackets can be separated by the pipe Example: -```text +``` example01-{01..02}-{1|2} ``` @@ -73,3 +73,5 @@ The `remote` table function can be useful in the following cases: If the user is not specified, `default` is used. If the password is not specified, an empty password is used. + +[Original article](https://clickhouse.yandex/docs/en/query_language/table_functions/remote/) diff --git a/docs/en/query_language/table_functions/url.md b/docs/en/query_language/table_functions/url.md index 7e30936bd45..4e012cff01f 100644 --- a/docs/en/query_language/table_functions/url.md +++ b/docs/en/query_language/table_functions/url.md @@ -13,7 +13,9 @@ structure - table structure in `'UserID UInt64, Name String'` format. Determines **Example** -```sql +``` sql -- getting the first 3 lines of a table that contains columns of String and UInt32 type from HTTP-server which answers in CSV format. SELECT * FROM url('http://127.0.0.1:12345/', CSV, 'column1 String, column2 UInt32') LIMIT 3 ``` + +[Original article](https://clickhouse.yandex/docs/en/query_language/table_functions/url/) diff --git a/docs/en/roadmap.md b/docs/en/roadmap.md index 51b6df94107..490e570b659 100644 --- a/docs/en/roadmap.md +++ b/docs/en/roadmap.md @@ -1,20 +1,14 @@ # Roadmap -## Q3 2018 - -- `ALTER UPDATE` for batch changing the data with approach similar to `ALTER DELETE` -- Protobuf and Parquet input and output formats -- Improved compatibility with Tableau and other BI tools - ## Q4 2018 - JOIN syntax compatible with SQL standard: - Mutliple `JOIN`s in single `SELECT` - - Connecting tables with `ON` - - Support table reference instead of subquery - JOIN execution improvements: - Distributed join not limited by memory - - Predicate pushdown through join +- Protobuf and Parquet input and output formats - Resource pools for more precise distribution of cluster capacity between users + +[Original article](https://clickhouse.yandex/docs/en/roadmap/) diff --git a/docs/en/security_changelog.md b/docs/en/security_changelog.md index 11e0557a8b8..62e98614ef2 100644 --- a/docs/en/security_changelog.md +++ b/docs/en/security_changelog.md @@ -19,3 +19,5 @@ Credits: Andrey Krasichkov and Evgeny Sidorov of Yandex Information Security Tea Incorrect configuration in deb package could lead to unauthorized use of the database. Credits: the UK's National Cyber Security Centre (NCSC) + +[Original article](https://clickhouse.yandex/docs/en/security_changelog/) diff --git a/docs/fa/data_types/array.md b/docs/fa/data_types/array.md index 9b7c70618a3..e14f698f6ed 100644 --- a/docs/fa/data_types/array.md +++ b/docs/fa/data_types/array.md @@ -4,4 +4,5 @@ آرایه ای از عناصر با تایپ T. تایپ T می تواند هر Type باشد، از جمله یک آرایه. ما توصیه به استفاده از آرایه های multidimensional نمی کنیم، چون آنها به خوبی پشتیبانی نمی شوند (برای مثال، شما نمی تونید در جداولی که موتور آنها MergeTree است، آرایه های multidimensional ذخیره سازی کنید). - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/array/) diff --git a/docs/fa/data_types/boolean.md b/docs/fa/data_types/boolean.md index 7fe6763ecc7..90f540468c7 100644 --- a/docs/fa/data_types/boolean.md +++ b/docs/fa/data_types/boolean.md @@ -5,3 +5,5 @@ type مخصوص مقادیر boolean وجود ندارد. از Uint8 و محدود شده به 0 و 1 می توان استفاده کرد. + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/boolean/) diff --git a/docs/fa/data_types/date.md b/docs/fa/data_types/date.md index 3b77e5ce0c5..2a71cbd24bd 100644 --- a/docs/fa/data_types/date.md +++ b/docs/fa/data_types/date.md @@ -6,4 +6,5 @@ Date، دو بایت به ازای هر تاریخ که به صورت عددی Date بدون time zone ذخیره می شود. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/date/) diff --git a/docs/fa/data_types/datetime.md b/docs/fa/data_types/datetime.md index 6731b1bea6c..d36243f7a4f 100644 --- a/docs/fa/data_types/datetime.md +++ b/docs/fa/data_types/datetime.md @@ -13,3 +13,5 @@ پس در هنگام کار با تاریخ متنی (برای مثال زمانی که دامپ text می گیرید)، به خاطر داشته باشید که ممکن است به دلیل تغییرات DST، نتایج مبهمی در خروجی ببینید، و ممکن است در صورت تغییر time zone مشکلی با مطابقت خروجی با داده ها وجود داشته باشد. + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/datetime/) diff --git a/docs/fa/data_types/enum.md b/docs/fa/data_types/enum.md index bf678647e8e..d84f2a1aa68 100644 --- a/docs/fa/data_types/enum.md +++ b/docs/fa/data_types/enum.md @@ -8,7 +8,7 @@ Enum8 یا Enum16، به شما اجازه ی ذخیره سازی مجموعه -```text +``` Enum8('hello' = 1, 'world' = 2) ``` @@ -30,4 +30,5 @@ Enum8('hello' = 1, 'world' = 2) استفاده از ALTER برای تبدیل Enum8 به Enum16 یا برعکس، ممکن است، دقیقا شبیه به Int8 به Int16. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/enum/) diff --git a/docs/fa/data_types/fixedstring.md b/docs/fa/data_types/fixedstring.md index f337fe0ef46..96234069307 100644 --- a/docs/fa/data_types/fixedstring.md +++ b/docs/fa/data_types/fixedstring.md @@ -6,4 +6,5 @@ توابع کمتری نسبت به String برای FixedString(N) وجود دارد، و برای استفاده کمتر مناسب است. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/fixedstring/) diff --git a/docs/fa/data_types/float.md b/docs/fa/data_types/float.md index 60fb9fb380a..77a17d6eca7 100644 --- a/docs/fa/data_types/float.md +++ b/docs/fa/data_types/float.md @@ -17,7 +17,7 @@ Type های float در ClickHouse مشابه C می باشد: -```sql +``` sql SELECT 1 - 0.9 ``` ``` @@ -40,7 +40,7 @@ SELECT 1 - 0.9 -```sql +``` sql SELECT 0.5 / 0 ``` @@ -56,7 +56,7 @@ SELECT 0.5 / 0 -```sql +``` sql SELECT -0.5 / 0 ``` @@ -86,4 +86,5 @@ SELECT 0 / 0 قوانین مربوط به مرتب سازی ` Nan ` را در بخش [ORDER BY clause](../query_language/select.md#query_language-queries-order_by) ببینید. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/float/) diff --git a/docs/fa/data_types/index.md b/docs/fa/data_types/index.md index acd91711ba6..5ffabb4344c 100644 --- a/docs/fa/data_types/index.md +++ b/docs/fa/data_types/index.md @@ -9,3 +9,5 @@ ClickHouse قابلیت ذخیره سازی انواع type های مختلف ب این بخش انواع data type های قابل پشتیبانی در ClickHouse را شرح می دهد، همچنین ملاحطات آنها در هنگام استفاده آنها را شرح می دهد. + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/) diff --git a/docs/fa/data_types/int_uint.md b/docs/fa/data_types/int_uint.md index fee85d72f33..0a1bbfe4a82 100644 --- a/docs/fa/data_types/int_uint.md +++ b/docs/fa/data_types/int_uint.md @@ -19,4 +19,5 @@ - UInt32 - [0 : 4294967295] - UInt64 - [0 : 18446744073709551615] - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/int_uint/) diff --git a/docs/fa/data_types/nested_data_structures/aggregatefunction.md b/docs/fa/data_types/nested_data_structures/aggregatefunction.md index 8d56156390e..d57a3eddba0 100644 --- a/docs/fa/data_types/nested_data_structures/aggregatefunction.md +++ b/docs/fa/data_types/nested_data_structures/aggregatefunction.md @@ -4,4 +4,5 @@ حالت متوسط از توابع aggregate. برای دریافت آن، از توابع aggregate به همراه پسوند '-State' استفاده کنید. برای اطلاعات بیشتر قسمت "AggregatingMergeTree" را ببینید. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/nested_data_structures/aggregatefunction/) diff --git a/docs/fa/data_types/nested_data_structures/index.md b/docs/fa/data_types/nested_data_structures/index.md index def4991e5bc..b5a3818646a 100644 --- a/docs/fa/data_types/nested_data_structures/index.md +++ b/docs/fa/data_types/nested_data_structures/index.md @@ -3,3 +3,5 @@ # Nested data structures + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/nested_data_structures/) diff --git a/docs/fa/data_types/nested_data_structures/nested.md b/docs/fa/data_types/nested_data_structures/nested.md index a02aec74872..1f3b675eef1 100644 --- a/docs/fa/data_types/nested_data_structures/nested.md +++ b/docs/fa/data_types/nested_data_structures/nested.md @@ -8,7 +8,7 @@ -```sql +``` sql CREATE TABLE test.visits ( CounterID UInt32, @@ -43,7 +43,7 @@ CREATE TABLE test.visits -```sql +``` sql SELECT Goals.ID, Goals.EventTime @@ -52,7 +52,7 @@ WHERE CounterID = 101500 AND length(Goals.ID) < 5 LIMIT 10 ``` -```text +``` ┌─Goals.ID───────────────────────┬─Goals.EventTime───────────────────────────────────────────────────────────────────────────┐ │ [1073752,591325,591325] │ ['2014-03-17 16:38:10','2014-03-17 16:38:48','2014-03-17 16:42:27'] │ │ [1073752] │ ['2014-03-17 00:28:25'] │ @@ -75,7 +75,7 @@ LIMIT 10 -```sql +``` sql SELECT Goal.ID, Goal.EventTime @@ -85,7 +85,7 @@ WHERE CounterID = 101500 AND length(Goals.ID) < 5 LIMIT 10 ``` -```text +``` ┌─Goal.ID─┬──────Goal.EventTime─┐ │ 1073752 │ 2014-03-17 16:38:10 │ │ 591325 │ 2014-03-17 16:38:48 │ @@ -110,4 +110,5 @@ LIMIT 10 دستور ALTER برای عناصر داخل nested بسیار محدود است. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/nested_data_structures/nested/) diff --git a/docs/fa/data_types/special_data_types/expression.md b/docs/fa/data_types/special_data_types/expression.md index 69ac90a3976..ada0eb0856e 100644 --- a/docs/fa/data_types/special_data_types/expression.md +++ b/docs/fa/data_types/special_data_types/expression.md @@ -4,4 +4,5 @@ برای نشان دادن توابع لامبدا در توابع high-order استفاده می شود. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/special_data_types/expression/) diff --git a/docs/fa/data_types/special_data_types/index.md b/docs/fa/data_types/special_data_types/index.md index 88f32012867..257b7508753 100644 --- a/docs/fa/data_types/special_data_types/index.md +++ b/docs/fa/data_types/special_data_types/index.md @@ -4,4 +4,5 @@ مقادیر نوع داده special، نمیتوانند در در جدول ذخیره و یا در نتایج خروجی قرار بگیرند، اما در نتایج متوسط (intermediate) یک query در حال اجرا استفاده می شوند. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/special_data_types/) diff --git a/docs/fa/data_types/special_data_types/set.md b/docs/fa/data_types/special_data_types/set.md index e518997271b..f2332e9eb86 100644 --- a/docs/fa/data_types/special_data_types/set.md +++ b/docs/fa/data_types/special_data_types/set.md @@ -4,4 +4,5 @@ برای نصف سمت راست IN استفاده می شود. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/special_data_types/set/) diff --git a/docs/fa/data_types/string.md b/docs/fa/data_types/string.md index 51734007388..71685574cc3 100644 --- a/docs/fa/data_types/string.md +++ b/docs/fa/data_types/string.md @@ -8,4 +8,5 @@ String یک type برای قرار دادن رشته با طول دلخواه م ClickHouse مفهومی به نام encoding ندارد. String ها می توانند شامل مجموعه ای بایت ها باشند که با همان شکل که نوشته می شوند به همان شکل هم در خروجی دیده شوند. اگر شما نیاز به ذخیره سازی متن دارید، توصیه می کنیم از UTF-8 استفاده کنید. حداقل اگر ترمینال شما از UTF-8 (پیشنهاد شده)، استفاده می کند، شما می توانید به راحتی مقادیر خود را نوشته و بخوانید.به طور مشابه توابع خاصی برای کار با رشته های متنوع وجود دارند که تخل این فرضیه عمل می کنند که رشته شامل مجوعه ای از بایت ها می باشند که نماینده ی متن های UTF-8 هستند. برای مثال تابع 'length' برای محاسبه طول رشته براساس بایت است، در حالی که تابع 'lengthUTF8' برای محاسبه طول رشته بر اساس UNICODE می باشد. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/string/) diff --git a/docs/fa/data_types/tuple.md b/docs/fa/data_types/tuple.md index 1acb21fb2fd..3d0a22cb16c 100644 --- a/docs/fa/data_types/tuple.md +++ b/docs/fa/data_types/tuple.md @@ -6,4 +6,5 @@ Tuple ها نمیتوانند در جدول نوشته شوند (به غیر ج Tuple می توانند در خروجی نتیجه query در حال اجرا باشند. در این مورد، برای فرمت های text به غیر از JSON\*، مقادیر به صورت comma-separate داخل براکت قرار میگیرند. در فرمت های JSON\* مقادیر tuple به صورت آرایه در خروجی می آیند. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/tuple/) diff --git a/docs/fa/getting_started/example_datasets/amplab_benchmark.md b/docs/fa/getting_started/example_datasets/amplab_benchmark.md index acdf6dc8704..6ed6f97a344 100644 --- a/docs/fa/getting_started/example_datasets/amplab_benchmark.md +++ b/docs/fa/getting_started/example_datasets/amplab_benchmark.md @@ -29,7 +29,7 @@ cd .. -```sql +``` sql CREATE TABLE rankings_tiny ( pageURL String, @@ -112,7 +112,7 @@ query های گرفتن data sample -```sql +``` sql SELECT pageURL, pageRank FROM rankings_1node WHERE pageRank > 1000 SELECT substring(sourceIP, 1, 8), sum(adRevenue) FROM uservisits_1node GROUP BY substring(sourceIP, 1, 8) @@ -134,3 +134,5 @@ GROUP BY sourceIP ORDER BY totalRevenue DESC LIMIT 1 ``` + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/getting_started/example_datasets/amplab_benchmark/) diff --git a/docs/fa/getting_started/example_datasets/criteo.md b/docs/fa/getting_started/example_datasets/criteo.md index 57f4007efa9..47013f7aaab 100644 --- a/docs/fa/getting_started/example_datasets/criteo.md +++ b/docs/fa/getting_started/example_datasets/criteo.md @@ -8,7 +8,7 @@ -```sql +``` sql CREATE TABLE criteo_log (date Date, clicked UInt8, int1 Int32, int2 Int32, int3 Int32, int4 Int32, int5 Int32, int6 Int32, int7 Int32, int8 Int32, int9 Int32, int10 Int32, int11 Int32, int12 Int32, int13 Int32, cat1 String, cat2 String, cat3 String, cat4 String, cat5 String, cat6 String, cat7 String, cat8 String, cat9 String, cat10 String, cat11 String, cat12 String, cat13 String, cat14 String, cat15 String, cat16 String, cat17 String, cat18 String, cat19 String, cat20 String, cat21 String, cat22 String, cat23 String, cat24 String, cat25 String, cat26 String) ENGINE = Log ``` @@ -28,7 +28,7 @@ for i in {00..23}; do echo $i; zcat datasets/criteo/day_${i#0}.gz | sed -r 's/^/ -```sql +``` sql CREATE TABLE criteo ( date Date, @@ -81,9 +81,11 @@ CREATE TABLE criteo -```sql +``` sql INSERT INTO criteo SELECT date, clicked, int1, int2, int3, int4, int5, int6, int7, int8, int9, int10, int11, int12, int13, reinterpretAsUInt32(unhex(cat1)) AS icat1, reinterpretAsUInt32(unhex(cat2)) AS icat2, reinterpretAsUInt32(unhex(cat3)) AS icat3, reinterpretAsUInt32(unhex(cat4)) AS icat4, reinterpretAsUInt32(unhex(cat5)) AS icat5, reinterpretAsUInt32(unhex(cat6)) AS icat6, reinterpretAsUInt32(unhex(cat7)) AS icat7, reinterpretAsUInt32(unhex(cat8)) AS icat8, reinterpretAsUInt32(unhex(cat9)) AS icat9, reinterpretAsUInt32(unhex(cat10)) AS icat10, reinterpretAsUInt32(unhex(cat11)) AS icat11, reinterpretAsUInt32(unhex(cat12)) AS icat12, reinterpretAsUInt32(unhex(cat13)) AS icat13, reinterpretAsUInt32(unhex(cat14)) AS icat14, reinterpretAsUInt32(unhex(cat15)) AS icat15, reinterpretAsUInt32(unhex(cat16)) AS icat16, reinterpretAsUInt32(unhex(cat17)) AS icat17, reinterpretAsUInt32(unhex(cat18)) AS icat18, reinterpretAsUInt32(unhex(cat19)) AS icat19, reinterpretAsUInt32(unhex(cat20)) AS icat20, reinterpretAsUInt32(unhex(cat21)) AS icat21, reinterpretAsUInt32(unhex(cat22)) AS icat22, reinterpretAsUInt32(unhex(cat23)) AS icat23, reinterpretAsUInt32(unhex(cat24)) AS icat24, reinterpretAsUInt32(unhex(cat25)) AS icat25, reinterpretAsUInt32(unhex(cat26)) AS icat26 FROM criteo_log; DROP TABLE criteo_log; ``` + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/getting_started/example_datasets/criteo/) diff --git a/docs/fa/getting_started/example_datasets/nyc_taxi.md b/docs/fa/getting_started/example_datasets/nyc_taxi.md index bf478583304..892b706e616 100644 --- a/docs/fa/getting_started/example_datasets/nyc_taxi.md +++ b/docs/fa/getting_started/example_datasets/nyc_taxi.md @@ -33,7 +33,7 @@ PostgreSQL تقریبا 20 تا 30 دقیقه برای پردازش هر ماه -```text +``` time psql nyc-taxi-data -c "SELECT count(*) FROM trips;" ## count 1298979494 @@ -52,7 +52,7 @@ Export گیری داده ها از PostgreSQL: -```sql +``` sql COPY ( SELECT trips.id, @@ -130,7 +130,7 @@ snapshot از داده ها با سرعت 50 مگابایت در ثانیه ان -```sql +``` sql CREATE TABLE trips ( trip_id UInt32, @@ -193,7 +193,7 @@ dropoff_puma Nullable(String) -```text +``` time clickhouse-client --query="INSERT INTO trips FORMAT TabSeparated" < trips.tsv real 75m56.214s @@ -213,7 +213,7 @@ real 75m56.214s -```text +``` CREATE TABLE trips_mergetree ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) AS SELECT @@ -283,7 +283,7 @@ FROM trips -```text +``` :) SELECT formatReadableSize(sum(bytes)) FROM system.parts WHERE table = 'trips_mergetree' AND active SELECT formatReadableSize(sum(bytes)) @@ -305,7 +305,7 @@ WHERE (table = 'trips_mergetree') AND active Q1: -```sql +``` sql SELECT cab_type, count(*) FROM trips_mergetree GROUP BY cab_type ``` @@ -313,7 +313,7 @@ SELECT cab_type, count(*) FROM trips_mergetree GROUP BY cab_type Q2: -```sql +``` sql SELECT passenger_count, avg(total_amount) FROM trips_mergetree GROUP BY passenger_count ``` @@ -321,7 +321,7 @@ SELECT passenger_count, avg(total_amount) FROM trips_mergetree GROUP BY passenge Q3: -```sql +``` sql SELECT passenger_count, toYear(pickup_date) AS year, count(*) FROM trips_mergetree GROUP BY passenger_count, year ``` @@ -329,7 +329,7 @@ SELECT passenger_count, toYear(pickup_date) AS year, count(*) FROM trips_mergetr Q4: -```sql +``` sql SELECT passenger_count, toYear(pickup_date) AS year, round(trip_distance) AS distance, count(*) FROM trips_mergetree GROUP BY passenger_count, year, distance @@ -354,7 +354,7 @@ Two Intel(R) Xeon(R) CPU E5-2650 v2 @ 2.60GHz, 16 physical kernels total, -```text +``` CREATE TABLE default.trips_mergetree_third ( trip_id UInt32, vendor_id Enum8('1' = 1, '2' = 2, 'CMT' = 3, 'VTS' = 4, 'DDS' = 5, 'B02512' = 10, 'B02598' = 11, 'B02617' = 12, 'B02682' = 13, 'B02764' = 14), pickup_date Date, pickup_datetime DateTime, dropoff_date Date, dropoff_datetime DateTime, store_and_fwd_flag UInt8, rate_code_id UInt8, pickup_longitude Float64, pickup_latitude Float64, dropoff_longitude Float64, dropoff_latitude Float64, passenger_count UInt8, trip_distance Float64, fare_amount Float32, extra Float32, mta_tax Float32, tip_amount Float32, tolls_amount Float32, ehail_fee Float32, improvement_surcharge Float32, total_amount Float32, payment_type_ Enum8('UNK' = 0, 'CSH' = 1, 'CRE' = 2, 'NOC' = 3, 'DIS' = 4), trip_type UInt8, pickup FixedString(25), dropoff FixedString(25), cab_type Enum8('yellow' = 1, 'green' = 2, 'uber' = 3), pickup_nyct2010_gid UInt8, pickup_ctlabel Float32, pickup_borocode UInt8, pickup_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), pickup_ct2010 FixedString(6), pickup_boroct2010 FixedString(7), pickup_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), pickup_ntacode FixedString(4), pickup_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), pickup_puma UInt16, dropoff_nyct2010_gid UInt8, dropoff_ctlabel Float32, dropoff_borocode UInt8, dropoff_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), dropoff_ct2010 FixedString(6), dropoff_boroct2010 FixedString(7), dropoff_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), dropoff_ntacode FixedString(4), dropoff_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), dropoff_puma UInt16) ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) ``` @@ -364,7 +364,7 @@ CREATE TABLE default.trips_mergetree_third ( trip_id UInt32, vendor_id Enum8('1 -```sql +``` sql CREATE TABLE trips_mergetree_x3 AS trips_mergetree_third ENGINE = Distributed(perftest, default, trips_mergetree_third, rand()) ``` @@ -374,7 +374,7 @@ query زیر دادها را توزیع مجدد می کند: -```sql +``` sql INSERT INTO trips_mergetree_x3 SELECT * FROM trips_mergetree ``` @@ -408,4 +408,5 @@ Q4: 0.072 ثانیه. | 3 | 0.212 | 0.438 | 0.733 | 1.241 | | 140 | 0.028 | 0.043 | 0.051 | 0.072 | - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/getting_started/example_datasets/nyc_taxi/) diff --git a/docs/fa/getting_started/example_datasets/ontime.md b/docs/fa/getting_started/example_datasets/ontime.md index f5bf5b865af..443e37ed9fe 100644 --- a/docs/fa/getting_started/example_datasets/ontime.md +++ b/docs/fa/getting_started/example_datasets/ontime.md @@ -26,7 +26,7 @@ done -```sql +``` sql CREATE TABLE `ontime` ( `Year` UInt16, `Quarter` UInt8, @@ -157,7 +157,7 @@ Q0. -```sql +``` sql select avg(c1) from (select Year, Month, count(*) as c1 from ontime group by Year, Month); ``` @@ -167,7 +167,7 @@ Q1. تعداد پروازهای به تفکیک روز از تاریخ 2000 تا -```sql +``` sql SELECT DayOfWeek, count(*) AS c FROM ontime WHERE Year >= 2000 AND Year <= 2008 GROUP BY DayOfWeek ORDER BY c DESC; ``` @@ -177,7 +177,7 @@ Q2. تعداد پروازهای بیش از 10 دقیقه تاخیر خورده -```sql +``` sql SELECT DayOfWeek, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Year <= 2008 GROUP BY DayOfWeek ORDER BY c DESC ``` @@ -187,7 +187,7 @@ Q3. تعداد تاخیرها براساس airport از سال 2000 تا 2008 -```sql +``` sql SELECT Origin, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Year <= 2008 GROUP BY Origin ORDER BY c DESC LIMIT 10 ``` @@ -197,7 +197,7 @@ Q4. تعداد تاخیرها براساس carrier در سال 78 -```sql +``` sql SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC ``` @@ -207,7 +207,7 @@ Q5. درصد تاخیر ها براساس carrier در سال 2007 -```sql +``` sql SELECT Carrier, c, c2, c*1000/c2 as c3 FROM ( @@ -237,7 +237,7 @@ ORDER BY c3 DESC; -```sql +``` sql SELECT Carrier, avg(DepDelay > 10) * 1000 AS c3 FROM ontime WHERE Year = 2007 GROUP BY Carrier ORDER BY Carrier ``` @@ -247,7 +247,7 @@ Q6. مانند query قبلی اما برای طیف وسیعی از سال ها -```sql +``` sql SELECT Carrier, c, c2, c*1000/c2 as c3 FROM ( @@ -277,7 +277,7 @@ ORDER BY c3 DESC; -```sql +``` sql SELECT Carrier, avg(DepDelay > 10) * 1000 AS c3 FROM ontime WHERE Year >= 2000 AND Year <= 2008 GROUP BY Carrier ORDER BY Carrier ``` @@ -287,7 +287,7 @@ Q7. درصد تاخیر بیش از 10 دقیقه پروازها به تفکیک -```sql +``` sql SELECT Year, c1/c2 FROM ( @@ -315,7 +315,7 @@ ORDER BY Year -```sql +``` sql SELECT Year, avg(DepDelay > 10) FROM ontime GROUP BY Year ORDER BY Year ``` @@ -325,7 +325,7 @@ Q8. مقصدهای پرطرفدار براساس تعداد اتصال های م -```sql +``` sql SELECT DestCityName, uniqExact(OriginCityName) AS u FROM ontime WHERE Year >= 2000 and Year <= 2010 GROUP BY DestCityName ORDER BY u DESC LIMIT 10; ``` @@ -335,7 +335,7 @@ Q9. -```sql +``` sql select Year, count(*) as c1 from ontime group by Year; ``` @@ -345,7 +345,7 @@ Q10. -```sql +``` sql select min(Year), max(Year), Carrier, count(*) as cnt, sum(ArrDelayMinutes>30) as flights_delayed, @@ -367,7 +367,7 @@ query های بیشتر: -```sql +``` sql SELECT avg(cnt) FROM (SELECT Year,Month,count(*) AS cnt FROM ontime WHERE DepDel15=1 GROUP BY Year,Month) select avg(c1) from (select Year,Month,count(*) as c1 from ontime group by Year,Month) @@ -391,3 +391,5 @@ SELECT OriginCityName, count() AS c FROM ontime GROUP BY OriginCityName ORDER BY - + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/getting_started/example_datasets/ontime/) diff --git a/docs/fa/getting_started/example_datasets/star_schema.md b/docs/fa/getting_started/example_datasets/star_schema.md index 8d09090f35e..90bf5246e5c 100644 --- a/docs/fa/getting_started/example_datasets/star_schema.md +++ b/docs/fa/getting_started/example_datasets/star_schema.md @@ -33,7 +33,7 @@ make -```sql +``` sql CREATE TABLE lineorder ( LO_ORDERKEY UInt32, LO_LINENUMBER UInt8, @@ -96,3 +96,5 @@ CREATE TABLE partd AS part ENGINE = Distributed(perftest_3shards_1replicas, defa cat customer.tbl | sed 's/$/2000-01-01/' | clickhouse-client --query "INSERT INTO customer FORMAT CSV" cat lineorder.tbl | clickhouse-client --query "INSERT INTO lineorder FORMAT CSV" ``` + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/getting_started/example_datasets/star_schema/) diff --git a/docs/fa/getting_started/example_datasets/wikistat.md b/docs/fa/getting_started/example_datasets/wikistat.md index 543a41901ab..c49fd4ae271 100644 --- a/docs/fa/getting_started/example_datasets/wikistat.md +++ b/docs/fa/getting_started/example_datasets/wikistat.md @@ -8,7 +8,7 @@ -```sql +``` sql CREATE TABLE wikistat ( date Date, @@ -33,3 +33,5 @@ for i in {2007..2016}; do for j in {01..12}; do echo $i-$j >&2; curl -sSL "http: cat links.txt | while read link; do wget http://dumps.wikimedia.org/other/pagecounts-raw/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1/')/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1-\2/')/$link; done ls -1 /opt/wikistat/ | grep gz | while read i; do echo $i; gzip -cd /opt/wikistat/$i | ./wikistat-loader --time="$(echo -n $i | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})([0-9]{2})-([0-9]{2})([0-9]{2})([0-9]{2})\.gz/\1-\2-\3 \4-00-00/')" | clickhouse-client --query="INSERT INTO wikistat FORMAT TabSeparated"; done ``` + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/getting_started/example_datasets/wikistat/) diff --git a/docs/fa/getting_started/index.md b/docs/fa/getting_started/index.md index 8d5a0bcfa11..dba3d4d080f 100644 --- a/docs/fa/getting_started/index.md +++ b/docs/fa/getting_started/index.md @@ -26,7 +26,7 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su -```text +``` deb http://repo.yandex.ru/clickhouse/deb/stable/ main/ ``` @@ -58,7 +58,7 @@ ClickHouse دارای تنظیمات محدودیت دسترسی می باشد. -```text +``` Client: dbms/programs/clickhouse-client Server: dbms/programs/clickhouse-server ``` @@ -69,7 +69,7 @@ Server: dbms/programs/clickhouse-server -```text +``` /opt/clickhouse/data/default/ /opt/clickhouse/metadata/default/ ``` @@ -167,4 +167,5 @@ SELECT 1 برای ادامه آزمایشات، شما میتوانید دیتاست های تستی را دریافت و امتحان کنید. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/getting_started/) diff --git a/docs/fa/index.md b/docs/fa/index.md index aa93081b948..d5f6c20d2a4 100644 --- a/docs/fa/index.md +++ b/docs/fa/index.md @@ -84,9 +84,8 @@ ClickHouse یک مدیریت دیتابیس (DBMS) ستون گرا برای پر برای مثال، query "تعداد رکوردها به ازای هر بستر نیازمندی" نیازمند خواندن ستون "آیدی بستر آگهی"، که 1 بایت بدون فشرده طول می کشد، خواهد بود. اگر بیشتر ترافیک مربوط به بستر های نیازمندی نبود، شما می توانید انتظار حداقل 10 برابر فشرده سازی این ستون را داشته باشید. زمانی که از الگوریتم فشرده سازی quick استفاده می کنید، عملیات decompression داده ها با سرعت حداقل چندین گیگابایت در ثانیه انجام می شود. به عبارت دیگر، این query توانایی پردازش تقریبا چندین میلیارد رکورد در ثانیه به ازای یک سرور را دارد. این سرعت در عمل واقعی و دست یافتنی است. -
مثال -

-

+
مثال +``` $ clickhouse-client ClickHouse client version 0.0.52053. Connecting to localhost:9000. @@ -128,8 +127,7 @@ LIMIT 20 20 rows in set. Elapsed: 0.153 sec. Processed 1.00 billion rows, 4.00 GB (6.53 billion rows/s., 26.10 GB/s.) :) -
-

+```
### CPU @@ -146,4 +144,6 @@ LIMIT 20 توجه کنید که برای کارایی CPU، query language باید SQL یا MDX باشد، یا حداقل یک بردارد (J, K) باشد. query برای بهینه سازی باید فقط دارای حلقه های implicit باشد. - \ No newline at end of file + + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/) diff --git a/docs/fa/interfaces/cli.md b/docs/fa/interfaces/cli.md index 6c7372c8363..b4c0bc7e0de 100644 --- a/docs/fa/interfaces/cli.md +++ b/docs/fa/interfaces/cli.md @@ -120,3 +120,5 @@ command line برا پایه 'readline' (و 'history' یا 'libedit'، یه بد ``` + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/interfaces/cli/) diff --git a/docs/fa/interfaces/formats.md b/docs/fa/interfaces/formats.md index c2aaf851434..197946df71d 100644 --- a/docs/fa/interfaces/formats.md +++ b/docs/fa/interfaces/formats.md @@ -34,39 +34,117 @@ Format | INSERT | SELECT [XML](formats.md#xml) | ✗ | ✔ | [CapnProto](formats.md#capnproto) | ✔ | ✔ | - -## CapnProto +## TabSeparated -Cap'n Proto یک فرمت پیام باینری شبیه به Protocol Buffer و Thrift می باشد، اما شبیه به JSON یا MessagePack نیست. +در فرمت TabSeparated، داده ها به صورت سطر نوشته می شوند. هر سطر شامل مقادیر جدا شده با tab می باشد. هر مقدار با یک tab دنبال می شود، به جز آخرین مقدار یک سطر، که با line feed دنبال می شود. line feed unix در همه جا مورد تسافده قرار می گیرد. آخرین سطر از خروجی هم باید شامل line feed در انتها باشد. مقادیر در فرمت متنی بدون enclose با کوتیشون، و یا escape با کاراکترهای ویژه، نوشته می شوند. -پیغام های Cap'n Proto به صورت self-describing نیستند، به این معنی که آنها نیاز دارند که به صورت external، schema آنها شرح داده شود. schema به صورت on the fly اضافه می شود و برای هر query، cache می شود. +اعداد Integer با فرم decimal نوشته می شوند. اعداد می توانند شامل کاراکتر اضافه "+" در ابتدای خود باشند. (در هنگام پارس کردن نادیده گرفته می شوند، و در هنگام فرمت کردن، ثبت نمی شوند). اعداد غیر منفی نمیتوانند شامل علامت منفی باشند. در هنگام خواندن، اجازه داده می شود که رشته خالی را به عنوان صفر، پارس کرد، یا (برای تایپ های sign) یک رشته که شامل فقط یک علامت منفی است به عنوان صفر پارس کرد. اعدادی که در data type مربوطه فیت نشوند ممکن است به عددی متفاوت تبدیل شوند و پیغام خطایی هم نمایش ندهند. - +اعداد Floating-point به فرم decimal نوشته می شوند. از دات به عنوان جدا کننده decimal استفاده می شود. نوشته های نمایشی مثل 'inf'، '+inf'، '-inf' و 'nan' پشتیبانی می شوند. ورودی اعداد floating-point می تواند با یه نقطه اعشار شروع یا پایان یابد. در هنگام فرمت، دقت اعداد floating-point ممکن است گم شوند. در هنگام پارس کردن، دقیقا نیازی به خواندن نزدیکترین عدد machine-representable نیست. -```sql -SELECT SearchPhrase, count() AS c FROM test.hits - GROUP BY SearchPhrase FORMAT CapnProto SETTINGS schema = 'schema:Message' -``` +Dates با فرمت YYY-MM-DD نوشته می شوند و به همین حالت پارس می شوند، اما با هر کاراکتری به عنوان جدا کننده. Dates به همراه زمان با فرمت YYYY-MM-DD hh:mm:ss نوشته می شوند و با همین فرمت پارس می شوند، اما با هر کاراکتری به عنوان جداکننده. این در منطقه زمان سیستم در زمانی که کلاینت یا سرور شروع می شود (بسته به اینکه کدام یک از داده ها را تشکیل می دهد) رخ می دهد. برای تاریخ همراه با زمان DST مشخص نمی شود. پس اگر یک دامپ دارای زمان DST باشد، دامپ، داده ها را به طور غیرمستقیم مطابقت نمی دهد و پارسینگ، یکی از دو ساعت را انتخاب خواهد کرد. در طول عملیات خواندن، تاریخ ها و تاریخ و ساعت های نادرست می توانند به صورت null و یا natural overflow پارس شوند، بدون اینکه پیغام خطایی نمایش دهند. -
+به عنوان یک استثنا، پارس کردن تاریخ به همراه ساعت، اگر مقدار دقیقا شامل 10 عدد decimal باشد، به عنوان فرمت unix timestamp پشتیبانی خواهد کرد. خروجی وابسته به time-zone نمی باشد. فرمت های YYYY-MM-DD hh: mm: ss و NNNNNNNNNN به صورت خودکار تمایز می یابند. -جایی که `schema.capnp` شبیه این است: +رشته های دارای کاراکتر های ویژه backslash-escaped چاپ می شوند. escape های در ادامه برای خروجی استفاده می شوند: `\b`، `\f`، `\r`، `\n`، `\t`، `\0`, `\'`، `\\`. پارسر همچنین از `\a`، `\v`، و `\xHH` (hex escape) و هر `\c` پشتیبانی می کند. بدین ترتیب خواندن داده ها از فرمت line feed که می تواند به صورت `\n` یا `\` نوشته شود پشتیبانی می کند. برای مثال، رشته ی `Hello world` به همراه line feed بین کلمات به جای space می تواند به هر یک از حالات زیر پارس شود::
``` -struct Message { - SearchPhrase @0 :Text; - c @1 :Uint64; -} +Hello\nworld + +Hello\ +world ```
-فایل های Schema در فایلی قرار دارند که این فایل در دایرکتوری مشخص شده کانفیگ [ format_schema_path](../operations/server_settings/settings.md#server_settings-format_schema_path) قرار گرفته اند. +نوع دوم به دلیل پشتیبانی MySQL در هنگام نوشتن دامپ به صورت tab-separate، پشتیبانی می شود. -عملیات Deserialization موثر است و معمولا لود سیستم را افزایش نمی دهد. +حداقل مجموعه از کاراکترهایی که در هنگام پاس دادن داده در فرمت TabSeperate نیاز به escape آن دارید: tab، line feed (LF) بک اسلش. + +فقط مجموعه ی کمی از نماد ها escape می شوند. شما به راحتی می توانید بر روی مقدار رشته که در ترمینال شما در خروجی نمایش داده می شود حرکت کنید. + +آرایه ها به صورت لیستی از مقادیر که به comma از هم جدا شده اند و در داخل براکت قرار گرفته اند نوشته می شوند. آیتم های عددی در آرای به صورت نرمال فرمت می شوند، اما تاریخ و تاریخ با ساعت و رشته ها در داخل تک کوتیشن به همراه قوانین escape که بالا اشاره شد، نوشته می شوند. + +فرمت TabSeparate برای پردازش داده ها با استفاده از برنامه های شخصی سازی شده و اسکریپت ها مناسب است. TabSeparate به صورت پیش فرض در HTTP interface و در حالت batch کلاینت command-line مورد استفاده قرار می گیرد. همچنین این فرمت اجازه ی انتقال داده ها بین DBMS های مختلف را می دهد. برای مثال، شما می توانید از MySQL با این روش دامپ بگیرید و آن را در ClickHouse یا vice versa آپلود کنید. + +فرمت TabSeparated از خروحی total values (هنگام استفاده از WITH TOTALS) و extreme values (در هنگامی که 'extreme' برابر با 1 است) پشتیبانی می کند. در این موارد، total value و extreme بعد از داده های اصلی در خروجی می آیند. نتایج اصلی، total values و extreme همگی با یک empty line از هم جدا می شوند. مثال: + +
+ +``` sql +SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT TabSeparated`` +``` + +``` +2014-03-17 1406958 +2014-03-18 1383658 +2014-03-19 1405797 +2014-03-20 1353623 +2014-03-21 1245779 +2014-03-22 1031592 +2014-03-23 1046491 + +0000-00-00 8873898 + +2014-03-17 1031592 +2014-03-23 1406958 +``` + +
+ +این فرمت نیز تحت نام `TSV` موجود است. + + + +## TabSeparatedRaw + +تفاوت آن با `TabSeperated` در این است که در این فرمت سطرها بدون escape نوشته می شوند. این فرمت فقط مناسب خروجی نتایج query ها می باشد، نه برای پارس کردن (دریافت داده ها و درج آن در جدول). + +همچنین این فرمت تحت عنوان ` TSVRaw`وجود دارد. + +## TabSeparatedWithNames + +تفاوت آن با فرمت `TabSeparated` در این است که، در این فرمت نام ستون ها در سطر اول قرار می گیرد. در طول پارس کردن، سطر اول به طور کامل نادیده گرفته می شود. شما نمی توانید نام ستون ها را برای تعیین موقعیت آنها یا بررسی صحت آنها استفاده کنید. (پشتیبانی از پارس کردن سطر header ممکن است در آینده اضافه شود.) + +همچنین این فرمت تحت عنوان ` TSVWithNames`وجود دارد. + +## TabSeparatedWithNamesAndTypes + +تفاوت آن با `TabSeparated` در این است که در این فرمت نام ستون ها در سطر اول نوشته می شود، و type ستون ها در سطر دوم نوشته می شود. در طی پارسینگ، سطر اول و دوم به طور کامل نادیده گرفته می شوند. + +همچنین این فرمت تحت عنوان ` TSVWithNamesAndTypes`وجود دارد. + +## TSKV + +مشابه فرمت TabSeparated، اما خروجی به صورت name=value می باشد. نام ها مشابه روش TabSeparated، escape می شوند، و همچنین = symbol هم escape می شود. + +
+ +``` +SearchPhrase= count()=8267016 +SearchPhrase=bathroom interior design count()=2166 +SearchPhrase=yandex count()=1655 +SearchPhrase=spring 2014 fashion count()=1549 +SearchPhrase=freeform photos count()=1480 +SearchPhrase=angelina jolia count()=1245 +SearchPhrase=omsk count()=1112 +SearchPhrase=photos of dog breeds count()=1091 +SearchPhrase=curtain design count()=1064 +SearchPhrase=baku count()=1000 +``` + +
+ +وقتی تعداد زیادی از ستون ها وجود دارد، این فرمت بی فایده است، و در حالت کلی دلیلی بر استفاده از این فرمت در این مواقع وجود ندارد. این فرمت در بعضی از دپارتمان های Yandex استفاده می شد. + +خروجی داده ها و پارس کردن هر دو در این فرمت پشتیبانی می شوند. برای پارس کردن، هر ترتیبی برای مقادیر ستون های مختلف پشتیبانی می شود. حذف بعضی از مقادیر قابل قبول است. این مقادیر با مقادیر پیش فرض خود برابر هستند. در این مورد، صفر و سطر خالی، توسط مقادیر پیش فرض پر می شوند. مقادیر پیچیده ای که می تواند در جدول مشخص شود به عنوان پیش فرض در این فرمت پشتیبانی نمیشوند. + +پارس کردن، اجازه می دهد که فیلد اضافه ی `tskv` بدون علامت و مقدار وجود داشته باشد. این فیلد نادیده گرفته می شود. + + ## CSV @@ -98,7 +176,7 @@ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMA
-```sql +``` sql SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTALS ORDER BY c DESC LIMIT 5 FORMAT JSON ``` @@ -272,11 +350,11 @@ JSON با جاوااسکریپت سازگار است. برای اطمینان ا -```sql +``` sql SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT PrettyCompact ``` -```text +``` ┌──EventDate─┬───────c─┐ │ 2014-03-17 │ 1406958 │ │ 2014-03-18 │ 1383658 │ @@ -347,115 +425,6 @@ watch -n1 "clickhouse-client --query='SELECT event, value FROM system.events FOR آرایه به عنوان variant length نشان داده می شود (unsigned [LEB128](https://en.wikipedia.org/wiki/LEB128))، دنباله ای از عانصر پیوسته آرایه -## TabSeparated - -در فرمت TabSeparated، داده ها به صورت سطر نوشته می شوند. هر سطر شامل مقادیر جدا شده با tab می باشد. هر مقدار با یک tab دنبال می شود، به جز آخرین مقدار یک سطر، که با line feed دنبال می شود. line feed unix در همه جا مورد تسافده قرار می گیرد. آخرین سطر از خروجی هم باید شامل line feed در انتها باشد. مقادیر در فرمت متنی بدون enclose با کوتیشون، و یا escape با کاراکترهای ویژه، نوشته می شوند. - -اعداد Integer با فرم decimal نوشته می شوند. اعداد می توانند شامل کاراکتر اضافه "+" در ابتدای خود باشند. (در هنگام پارس کردن نادیده گرفته می شوند، و در هنگام فرمت کردن، ثبت نمی شوند). اعداد غیر منفی نمیتوانند شامل علامت منفی باشند. در هنگام خواندن، اجازه داده می شود که رشته خالی را به عنوان صفر، پارس کرد، یا (برای تایپ های sign) یک رشته که شامل فقط یک علامت منفی است به عنوان صفر پارس کرد. اعدادی که در data type مربوطه فیت نشوند ممکن است به عددی متفاوت تبدیل شوند و پیغام خطایی هم نمایش ندهند. - -اعداد Floating-point به فرم decimal نوشته می شوند. از دات به عنوان جدا کننده decimal استفاده می شود. نوشته های نمایشی مثل 'inf'، '+inf'، '-inf' و 'nan' پشتیبانی می شوند. ورودی اعداد floating-point می تواند با یه نقطه اعشار شروع یا پایان یابد. در هنگام فرمت، دقت اعداد floating-point ممکن است گم شوند. در هنگام پارس کردن، دقیقا نیازی به خواندن نزدیکترین عدد machine-representable نیست. - -Dates با فرمت YYY-MM-DD نوشته می شوند و به همین حالت پارس می شوند، اما با هر کاراکتری به عنوان جدا کننده. Dates به همراه زمان با فرمت YYYY-MM-DD hh:mm:ss نوشته می شوند و با همین فرمت پارس می شوند، اما با هر کاراکتری به عنوان جداکننده. این در منطقه زمان سیستم در زمانی که کلاینت یا سرور شروع می شود (بسته به اینکه کدام یک از داده ها را تشکیل می دهد) رخ می دهد. برای تاریخ همراه با زمان DST مشخص نمی شود. پس اگر یک دامپ دارای زمان DST باشد، دامپ، داده ها را به طور غیرمستقیم مطابقت نمی دهد و پارسینگ، یکی از دو ساعت را انتخاب خواهد کرد. در طول عملیات خواندن، تاریخ ها و تاریخ و ساعت های نادرست می توانند به صورت null و یا natural overflow پارس شوند، بدون اینکه پیغام خطایی نمایش دهند. - -به عنوان یک استثنا، پارس کردن تاریخ به همراه ساعت، اگر مقدار دقیقا شامل 10 عدد decimal باشد، به عنوان فرمت unix timestamp پشتیبانی خواهد کرد. خروجی وابسته به time-zone نمی باشد. فرمت های YYYY-MM-DD hh: mm: ss و NNNNNNNNNN به صورت خودکار تمایز می یابند. - -رشته های دارای کاراکتر های ویژه backslash-escaped چاپ می شوند. escape های در ادامه برای خروجی استفاده می شوند: `\b`، `\f`، `\r`، `\n`، `\t`، `\0`, `\'`، `\\`. پارسر همچنین از `\a`، `\v`، و `\xHH` (hex escape) و هر `\c` پشتیبانی می کند. بدین ترتیب خواندن داده ها از فرمت line feed که می تواند به صورت `\n` یا `\` نوشته شود پشتیبانی می کند. برای مثال، رشته ی `Hello world` به همراه line feed بین کلمات به جای space می تواند به هر یک از حالات زیر پارس شود:: - - - -```text -Hello\nworld - -Hello\ -world -``` - -
- -نوع دوم به دلیل پشتیبانی MySQL در هنگام نوشتن دامپ به صورت tab-separate، پشتیبانی می شود. - -حداقل مجموعه از کاراکترهایی که در هنگام پاس دادن داده در فرمت TabSeperate نیاز به escape آن دارید: tab، line feed (LF) بک اسلش. - -فقط مجموعه ی کمی از نماد ها escape می شوند. شما به راحتی می توانید بر روی مقدار رشته که در ترمینال شما در خروجی نمایش داده می شود حرکت کنید. - -آرایه ها به صورت لیستی از مقادیر که به comma از هم جدا شده اند و در داخل براکت قرار گرفته اند نوشته می شوند. آیتم های عددی در آرای به صورت نرمال فرمت می شوند، اما تاریخ و تاریخ با ساعت و رشته ها در داخل تک کوتیشن به همراه قوانین escape که بالا اشاره شد، نوشته می شوند. - -فرمت TabSeparate برای پردازش داده ها با استفاده از برنامه های شخصی سازی شده و اسکریپت ها مناسب است. TabSeparate به صورت پیش فرض در HTTP interface و در حالت batch کلاینت command-line مورد استفاده قرار می گیرد. همچنین این فرمت اجازه ی انتقال داده ها بین DBMS های مختلف را می دهد. برای مثال، شما می توانید از MySQL با این روش دامپ بگیرید و آن را در ClickHouse یا vice versa آپلود کنید. - -فرمت TabSeparated از خروحی total values (هنگام استفاده از WITH TOTALS) و extreme values (در هنگامی که 'extreme' برابر با 1 است) پشتیبانی می کند. در این موارد، total value و extreme بعد از داده های اصلی در خروجی می آیند. نتایج اصلی، total values و extreme همگی با یک empty line از هم جدا می شوند. مثال: - -
- -```sql -SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT TabSeparated`` -``` - -```text -2014-03-17 1406958 -2014-03-18 1383658 -2014-03-19 1405797 -2014-03-20 1353623 -2014-03-21 1245779 -2014-03-22 1031592 -2014-03-23 1046491 - -0000-00-00 8873898 - -2014-03-17 1031592 -2014-03-23 1406958 -``` - -
- -این فرمت نیز تحت نام `TSV` موجود است. - - - -## TabSeparatedRaw - -تفاوت آن با `TabSeperated` در این است که در این فرمت سطرها بدون escape نوشته می شوند. این فرمت فقط مناسب خروجی نتایج query ها می باشد، نه برای پارس کردن (دریافت داده ها و درج آن در جدول). - -همچنین این فرمت تحت عنوان ` TSVRaw`وجود دارد. - -## TabSeparatedWithNames - -تفاوت آن با فرمت `TabSeparated` در این است که، در این فرمت نام ستون ها در سطر اول قرار می گیرد. در طول پارس کردن، سطر اول به طور کامل نادیده گرفته می شود. شما نمی توانید نام ستون ها را برای تعیین موقعیت آنها یا بررسی صحت آنها استفاده کنید. (پشتیبانی از پارس کردن سطر header ممکن است در آینده اضافه شود.) - -همچنین این فرمت تحت عنوان ` TSVWithNames`وجود دارد. - -## TabSeparatedWithNamesAndTypes - -تفاوت آن با `TabSeparated` در این است که در این فرمت نام ستون ها در سطر اول نوشته می شود، و type ستون ها در سطر دوم نوشته می شود. در طی پارسینگ، سطر اول و دوم به طور کامل نادیده گرفته می شوند. - -همچنین این فرمت تحت عنوان ` TSVWithNamesAndTypes`وجود دارد. - -## TSKV - -مشابه فرمت TabSeparated، اما خروجی به صورت name=value می باشد. نام ها مشابه روش TabSeparated، escape می شوند، و همچنین = symbol هم escape می شود. - -
- -```text -SearchPhrase= count()=8267016 -SearchPhrase=bathroom interior design count()=2166 -SearchPhrase=yandex count()=1655 -SearchPhrase=spring 2014 fashion count()=1549 -SearchPhrase=freeform photos count()=1480 -SearchPhrase=angelina jolia count()=1245 -SearchPhrase=omsk count()=1112 -SearchPhrase=photos of dog breeds count()=1091 -SearchPhrase=curtain design count()=1064 -SearchPhrase=baku count()=1000 -``` - -
- -وقتی تعداد زیادی از ستون ها وجود دارد، این فرمت بی فایده است، و در حالت کلی دلیلی بر استفاده از این فرمت در این مواقع وجود ندارد. این فرمت در بعضی از دپارتمان های Yandex استفاده می شد. - -خروجی داده ها و پارس کردن هر دو در این فرمت پشتیبانی می شوند. برای پارس کردن، هر ترتیبی برای مقادیر ستون های مختلف پشتیبانی می شود. حذف بعضی از مقادیر قابل قبول است. این مقادیر با مقادیر پیش فرض خود برابر هستند. در این مورد، صفر و سطر خالی، توسط مقادیر پیش فرض پر می شوند. مقادیر پیچیده ای که می تواند در جدول مشخص شود به عنوان پیش فرض در این فرمت پشتیبانی نمیشوند. - -پارس کردن، اجازه می دهد که فیلد اضافه ی `tskv` بدون علامت و مقدار وجود داشته باشد. این فیلد نادیده گرفته می شود. - ## Values هر سطر داخل براکت چاپ می شود. سطر ها توسط comma جدا می شوند. برای آخرین سطر comma وجود ندارد. مقادیر داخل براکت همچنین توسط comma جدا می شوند. اعداد با فرمت decimal و بدون کوتیشن چاپ می شوند. آرایه ها در براکت ها چاپ می شوند. رشته ها، تاریخ و تاریخ با ساعت داخل کوتیشن قرار می گیرند. قوانین escape و پارس کردن شبیه به فرمت TabSeparated انجام می شود. در طول فرمت، extra spaces درج نمی شوند، اما در هنگام پارس کردن، آنها مجاز و skip می شوند. (به جز space های داخل مقادیر آرایه، که مجاز نیستند). @@ -579,4 +548,40 @@ test: string with \'quotes\' and \t with some special \n characters آرایه ها به شکل `HelloWorld...` و tuple ها به صورت `HelloWorld...` در خروجی می آیند. + + +## CapnProto + +Cap'n Proto یک فرمت پیام باینری شبیه به Protocol Buffer و Thrift می باشد، اما شبیه به JSON یا MessagePack نیست. + +پیغام های Cap'n Proto به صورت self-describing نیستند، به این معنی که آنها نیاز دارند که به صورت external، schema آنها شرح داده شود. schema به صورت on the fly اضافه می شود و برای هر query، cache می شود. +
+ +``` sql +SELECT SearchPhrase, count() AS c FROM test.hits + GROUP BY SearchPhrase FORMAT CapnProto SETTINGS schema = 'schema:Message' +``` + +
+ +جایی که `schema.capnp` شبیه این است: + +
+ +``` +struct Message { + SearchPhrase @0 :Text; + c @1 :Uint64; +} +``` + +
+ +فایل های Schema در فایلی قرار دارند که این فایل در دایرکتوری مشخص شده کانفیگ [ format_schema_path](../operations/server_settings/settings.md#server_settings-format_schema_path) قرار گرفته اند. + +عملیات Deserialization موثر است و معمولا لود سیستم را افزایش نمی دهد. + +
+ +[مقاله اصلی](https://clickhouse.yandex/docs/fa/interfaces/formats/) diff --git a/docs/fa/interfaces/http_interface.md b/docs/fa/interfaces/http_interface.md index 3251b2d4ccd..418a92f4daf 100644 --- a/docs/fa/interfaces/http_interface.md +++ b/docs/fa/interfaces/http_interface.md @@ -280,3 +280,5 @@ curl -sS 'http://localhost:8123/?max_result_bytes=4000000&buffer_size=3000000&wa از بافرینگ به منظور اجتناب از شرایطی که یک خطای پردازش query رخ داده بعد از response کد و هدر های ارسال شده به کلاینت استفاده کنید. در این شرایط، پیغام خطا در انتهای بنده response نوشته می شود، و در سمت کلاینت، پیغام خطا فقط از طریق مرحله پارس کردن قابل شناسایی است. + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/interfaces/http_interface/) diff --git a/docs/fa/interfaces/index.md b/docs/fa/interfaces/index.md index 4d4600bd26f..d06c8ec8fd0 100644 --- a/docs/fa/interfaces/index.md +++ b/docs/fa/interfaces/index.md @@ -6,4 +6,5 @@ برای کشف قابلیت های سیستم، دانلو داده ها به جداول، یا ساخت query های دستی، از برنامه clikhouse-client استفاده کنید. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/interfaces/) diff --git a/docs/fa/interfaces/jdbc.md b/docs/fa/interfaces/jdbc.md index dc0acb15994..e72b5504c06 100644 --- a/docs/fa/interfaces/jdbc.md +++ b/docs/fa/interfaces/jdbc.md @@ -11,4 +11,5 @@ JDBC drivers implemented by other organizations: - [ClickHouse-Native-JDBC](https://github.com/housepower/ClickHouse-Native-JDBC) - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/interfaces/jdbc/) diff --git a/docs/fa/interfaces/tcp.md b/docs/fa/interfaces/tcp.md index 963555b5400..5e767b876f8 100644 --- a/docs/fa/interfaces/tcp.md +++ b/docs/fa/interfaces/tcp.md @@ -4,4 +4,5 @@ native interface در محیط ترمینال "clickhouse-client" برای تعامل بین سرور با پردازش query توزیع شده مورد استفاده قرار می گیرد. همچنین native interface در برنامه های C++ مورد استفاده قرار می گیرد. ما فقط کلاینت command-line را پوشش میدیم. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/interfaces/tcp/) diff --git a/docs/fa/interfaces/third-party_client_libraries.md b/docs/fa/interfaces/third-party_client_libraries.md index 2fb5de3cc9b..3f6536ccf22 100644 --- a/docs/fa/interfaces/third-party_client_libraries.md +++ b/docs/fa/interfaces/third-party_client_libraries.md @@ -47,4 +47,5 @@ ما این کتابخانه ها را تست نکردیم. آنها به صورت تصادفی انتخاب شده اند. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/interfaces/third-party_client_libraries/) diff --git a/docs/fa/interfaces/third-party_gui.md b/docs/fa/interfaces/third-party_gui.md index 8972edd880a..7f537b1c26e 100644 --- a/docs/fa/interfaces/third-party_gui.md +++ b/docs/fa/interfaces/third-party_gui.md @@ -35,4 +35,5 @@ interface تحت وب برای ClickHouse در پروژه [Tabix](https://github - مانیتورینگ کافکا و جداول replicate (بزودی); - و بسیاری از ویژگی های دیگر برای شما. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/interfaces/third-party_gui/) diff --git a/docs/fa/introduction/distinctive_features.md b/docs/fa/introduction/distinctive_features.md index 61b2115a3b5..94d4198e12e 100644 --- a/docs/fa/introduction/distinctive_features.md +++ b/docs/fa/introduction/distinctive_features.md @@ -63,4 +63,5 @@ ClickHouse از روش asynchronous multimaster replication استفاده می برای اطلاعات بیشتر، به بخش [replication داده ها](../operations/table_engines/replication.md#table_engines-replication) مراجعه کنید. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/introduction/distinctive_features/) diff --git a/docs/fa/introduction/features_considered_disadvantages.md b/docs/fa/introduction/features_considered_disadvantages.md index 0c9e1b0cf8a..3a963d05140 100644 --- a/docs/fa/introduction/features_considered_disadvantages.md +++ b/docs/fa/introduction/features_considered_disadvantages.md @@ -7,3 +7,5 @@ 3. Sparse index باعث می شود ClickHouse چندان مناسب اجرای پرسمان های point query برای دریافت یک ردیف از داده ها با استفاده از کلید آنها نباشد. + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/introduction/features_considered_disadvantages/) diff --git a/docs/fa/introduction/performance.md b/docs/fa/introduction/performance.md index 8d738cdbf9f..2fb2a9dc036 100644 --- a/docs/fa/introduction/performance.md +++ b/docs/fa/introduction/performance.md @@ -24,4 +24,5 @@ benchmark های زیادی وجود دارند که این نتایج را تا پیشنهاد می کنیم درج داده ها را به صورت دسته ای و حداقل 100 سطر در هر دسته انجام دهید و یا بیش از یک درخواست insert در ثانیه را نداشته باشید. در هنگام درج داده در جدول MergeTree از یک dump جدا شده با tab، سرعت درج داده از 50 تا 200 مگابایت در ثانیه می باشد. اگر سطر های درج شده حدود 1 کیلوبایت باشند، سرعت حدود 50 هزار تا 200 هزار سطر در ثانیه می باشد. اگر سطر ها کوچک باشند بازدهی بالایی در تعداد سطر در ثانیه خواهیم داشت. در Banner System Data -`>` 500 هزار سطر در ثانیه، در Graphite data -`>` 1 میلیون سطر در ثانیه). برای بهبود کارایی، شما می توانید چندین insert را به صورت موازی اجرا کنید، که در این حالت کارایی سیستم به صورت خطی افزایش می یابد. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/introduction/performance/) diff --git a/docs/fa/introduction/ya_metrika_task.md b/docs/fa/introduction/ya_metrika_task.md index d1f4aa1d1fd..1ea434f248c 100644 --- a/docs/fa/introduction/ya_metrika_task.md +++ b/docs/fa/introduction/ya_metrika_task.md @@ -45,4 +45,5 @@ Yandex.Metrica دارای یک سیستم تخصصی برای aggregate کردن برای حذف محدودیت های OLAPServer و حل مشکلات کار با داده های Non-Aggregate برای تمام گزارش ها، ما مدیریت دیتابیس ClicHouse را توسعه دادیم.. - \ No newline at end of file + +[مقاله اصلی](https://clickhouse.yandex/docs/fa/introduction/ya_metrika_task/) diff --git a/docs/fa/query_language/table_functions/jdbc.md b/docs/fa/query_language/table_functions/jdbc.md new file mode 120000 index 00000000000..73bec80ca58 --- /dev/null +++ b/docs/fa/query_language/table_functions/jdbc.md @@ -0,0 +1 @@ +../../../en/query_language/table_functions/jdbc.md \ No newline at end of file diff --git a/docs/ru/data_types/array.md b/docs/ru/data_types/array.md index 73b86d7c9d1..450c4cf5178 100644 --- a/docs/ru/data_types/array.md +++ b/docs/ru/data_types/array.md @@ -81,3 +81,5 @@ Code: 386. DB::Exception: Received from localhost:9000, 127.0.0.1. DB::Exception 0 rows in set. Elapsed: 0.246 sec. ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/array/) diff --git a/docs/ru/data_types/boolean.md b/docs/ru/data_types/boolean.md index 541f8ef0345..b85f1393a80 100644 --- a/docs/ru/data_types/boolean.md +++ b/docs/ru/data_types/boolean.md @@ -1,3 +1,5 @@ # Булевы значения Отдельного типа для булевых значений нет. Для них используется тип UInt8, в котором используются только значения 0 и 1. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/boolean/) diff --git a/docs/ru/data_types/date.md b/docs/ru/data_types/date.md index c458deaf627..887bb19e967 100644 --- a/docs/ru/data_types/date.md +++ b/docs/ru/data_types/date.md @@ -6,3 +6,5 @@ Минимальное значение выводится как 0000-00-00. Дата хранится без учёта часового пояса. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/date/) diff --git a/docs/ru/data_types/datetime.md b/docs/ru/data_types/datetime.md index dfc54e43add..d9f5457e3ac 100644 --- a/docs/ru/data_types/datetime.md +++ b/docs/ru/data_types/datetime.md @@ -14,3 +14,5 @@ Поддерживаются только часовые пояса, для которых для всего диапазона времён, с которым вы будете работать, не существовало моментов времени, в которые время отличалось от UTC на нецелое число часов (без учёта секунд координации). То есть, при работе с датой в виде текста (например, при сохранении текстовых дампов), следует иметь ввиду о проблемах с неоднозначностью во время перевода стрелок назад, и о проблемах с соответствием данных, при смене часового пояса. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/datetime/) diff --git a/docs/ru/data_types/decimal.md b/docs/ru/data_types/decimal.md index 9e554395a0f..cc0a3e4761b 100644 --- a/docs/ru/data_types/decimal.md +++ b/docs/ru/data_types/decimal.md @@ -93,3 +93,5 @@ SELECT toDecimal32(1, 8) < 100 ``` DB::Exception: Can't compare. ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/decimal/) diff --git a/docs/ru/data_types/enum.md b/docs/ru/data_types/enum.md index aaae8aad378..62250d8667b 100644 --- a/docs/ru/data_types/enum.md +++ b/docs/ru/data_types/enum.md @@ -110,3 +110,5 @@ INSERT INTO t_enum_null Values('hello'),('world'),(NULL) При ALTER, есть возможность бесплатно изменить тип Enum-а, если меняется только множество значений. При этом, можно добавлять новые значения; можно удалять старые значения (это безопасно только если они ни разу не использовались, так как это не проверяется). В качестве "защиты от дурака", нельзя менять числовые значения у имеющихся строк - в этом случае, кидается исключение. При ALTER, есть возможность поменять Enum8 на Enum16 и обратно - так же, как можно поменять Int8 на Int16. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/enum/) diff --git a/docs/ru/data_types/fixedstring.md b/docs/ru/data_types/fixedstring.md index 75fb9650b02..5c7fa19d506 100644 --- a/docs/ru/data_types/fixedstring.md +++ b/docs/ru/data_types/fixedstring.md @@ -7,3 +7,5 @@ Обратите внимание, как это поведение отличается от поведения MySQL для типа CHAR (строки дополняются пробелами, пробелы перед выводом вырезаются). С типом FixedString(N) умеет работать меньше функций, чем с типом String - то есть, он менее удобен в использовании. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/fixedstring/) diff --git a/docs/ru/data_types/float.md b/docs/ru/data_types/float.md index 228fd93193e..8f50419cf92 100644 --- a/docs/ru/data_types/float.md +++ b/docs/ru/data_types/float.md @@ -13,7 +13,7 @@ - При вычислениях с числами с плавающей запятой возможна ошибка округления. -```sql +``` sql SELECT 1 - 0.9 ``` ``` @@ -32,7 +32,7 @@ SELECT 1 - 0.9 - `Inf` - бесконечность. -```sql +``` sql SELECT 0.5 / 0 ``` @@ -44,7 +44,7 @@ SELECT 0.5 / 0 - `-Inf` - отрицательная бесконечность; -```sql +``` sql SELECT -0.5 / 0 ``` @@ -66,3 +66,5 @@ SELECT 0 / 0 ``` Смотрите правила сортировки `NaN` в разделе [Секция ORDER BY](../query_language/select.md#query_language-queries-order_by). + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/float/) diff --git a/docs/ru/data_types/index.md b/docs/ru/data_types/index.md index e41d457ee4c..a167ec3859f 100644 --- a/docs/ru/data_types/index.md +++ b/docs/ru/data_types/index.md @@ -4,4 +4,5 @@ ClickHouse может сохранять в ячейках таблиц данные различных типов. -Раздел содержит описания поддерживаемых типов данных и специфику их использования и/или реализации, если таковые имеются. \ No newline at end of file +Раздел содержит описания поддерживаемых типов данных и специфику их использования и/или реализации, если таковые имеются. +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/) diff --git a/docs/ru/data_types/int_uint.md b/docs/ru/data_types/int_uint.md index 49b7bbbbcf8..562da33e730 100644 --- a/docs/ru/data_types/int_uint.md +++ b/docs/ru/data_types/int_uint.md @@ -19,3 +19,5 @@ - UInt16 - [ 0 : 65535 ] - UInt32 - [ 0 : 4294967295 ] - UInt64 - [ 0 : 18446744073709551615 ] + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/int_uint/) diff --git a/docs/ru/data_types/nested_data_structures/aggregatefunction.md b/docs/ru/data_types/nested_data_structures/aggregatefunction.md index 26196695346..384a432b494 100644 --- a/docs/ru/data_types/nested_data_structures/aggregatefunction.md +++ b/docs/ru/data_types/nested_data_structures/aggregatefunction.md @@ -61,3 +61,5 @@ SELECT uniqMerge(state) FROM (SELECT uniqState(UserID) AS state FROM table GROUP ## Пример использования Смотрите в описании движка [AggregatingMergeTree](../../operations/table_engines/aggregatingmergetree.md#table_engine-aggregatingmergetree). + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/nested_data_structures/aggregatefunction/) diff --git a/docs/ru/data_types/nested_data_structures/index.md b/docs/ru/data_types/nested_data_structures/index.md index b7bc28de3da..3e5ea1bd69e 100644 --- a/docs/ru/data_types/nested_data_structures/index.md +++ b/docs/ru/data_types/nested_data_structures/index.md @@ -1 +1,2 @@ -# Вложенные структуры данных \ No newline at end of file +# Вложенные структуры данных +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/nested_data_structures/) diff --git a/docs/ru/data_types/nested_data_structures/nested.md b/docs/ru/data_types/nested_data_structures/nested.md index 6db4e9e83ab..06a62801e6e 100644 --- a/docs/ru/data_types/nested_data_structures/nested.md +++ b/docs/ru/data_types/nested_data_structures/nested.md @@ -4,7 +4,7 @@ Пример: -```sql +``` sql CREATE TABLE test.visits ( CounterID UInt32, @@ -35,7 +35,7 @@ CREATE TABLE test.visits Пример: -```sql +``` sql SELECT Goals.ID, Goals.EventTime @@ -44,7 +44,7 @@ WHERE CounterID = 101500 AND length(Goals.ID) < 5 LIMIT 10 ``` -```text +``` ┌─Goals.ID───────────────────────┬─Goals.EventTime───────────────────────────────────────────────────────────────────────────┐ │ [1073752,591325,591325] │ ['2014-03-17 16:38:10','2014-03-17 16:38:48','2014-03-17 16:42:27'] │ │ [1073752] │ ['2014-03-17 00:28:25'] │ @@ -63,7 +63,7 @@ LIMIT 10 Единственное место, где в запросе SELECT можно указать имя целой вложенной структуры данных, а не отдельных столбцов - секция ARRAY JOIN. Подробнее см. раздел "Секция ARRAY JOIN". Пример: -```sql +``` sql SELECT Goal.ID, Goal.EventTime @@ -73,7 +73,7 @@ WHERE CounterID = 101500 AND length(Goals.ID) < 5 LIMIT 10 ``` -```text +``` ┌─Goal.ID─┬──────Goal.EventTime─┐ │ 1073752 │ 2014-03-17 16:38:10 │ │ 591325 │ 2014-03-17 16:38:48 │ @@ -95,3 +95,5 @@ LIMIT 10 При запросе DESCRIBE, столбцы вложенной структуры данных перечисляются так же по отдельности. Работоспособность запроса ALTER для элементов вложенных структур данных, является сильно ограниченной. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/nested_data_structures/nested/) diff --git a/docs/ru/data_types/nullable.md b/docs/ru/data_types/nullable.md index f098cac2f76..e4b3bb830c7 100644 --- a/docs/ru/data_types/nullable.md +++ b/docs/ru/data_types/nullable.md @@ -61,3 +61,5 @@ FROM t_null 2 rows in set. Elapsed: 0.144 sec. ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/nullable/) diff --git a/docs/ru/data_types/special_data_types/expression.md b/docs/ru/data_types/special_data_types/expression.md index 8451a034743..438492c5399 100644 --- a/docs/ru/data_types/special_data_types/expression.md +++ b/docs/ru/data_types/special_data_types/expression.md @@ -1,3 +1,5 @@ # Expression Используется для представления лямбда-выражений в функциях высшего порядка. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/special_data_types/expression/) diff --git a/docs/ru/data_types/special_data_types/index.md b/docs/ru/data_types/special_data_types/index.md index 9adb03853ba..b9eda72c556 100644 --- a/docs/ru/data_types/special_data_types/index.md +++ b/docs/ru/data_types/special_data_types/index.md @@ -1,3 +1,5 @@ # Служебные типы данных Значения служебных типов данных не могут сохраняться в таблицу и выводиться в качестве результата, а возникают как промежуточный результат выполнения запроса. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/special_data_types/) diff --git a/docs/ru/data_types/special_data_types/nothing.md b/docs/ru/data_types/special_data_types/nothing.md index 6b83c354d5e..c452aa2f198 100644 --- a/docs/ru/data_types/special_data_types/nothing.md +++ b/docs/ru/data_types/special_data_types/nothing.md @@ -17,3 +17,5 @@ SELECT toTypeName([]) 1 rows in set. Elapsed: 0.062 sec. ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/special_data_types/nothing/) diff --git a/docs/ru/data_types/special_data_types/set.md b/docs/ru/data_types/special_data_types/set.md index 72a9a2647e5..fe31450ab29 100644 --- a/docs/ru/data_types/special_data_types/set.md +++ b/docs/ru/data_types/special_data_types/set.md @@ -1,3 +1,5 @@ # Set Используется для представления правой части выражения IN. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/special_data_types/set/) diff --git a/docs/ru/data_types/string.md b/docs/ru/data_types/string.md index f4a100470c7..18d18eecb0e 100644 --- a/docs/ru/data_types/string.md +++ b/docs/ru/data_types/string.md @@ -11,3 +11,5 @@ Если вам нужно хранить тексты, рекомендуется использовать кодировку UTF-8. По крайней мере, если у вас терминал работает в кодировке UTF-8 (это рекомендуется), вы сможете читать и писать свои значения без каких-либо преобразований. Также, некоторые функции по работе со строками, имеют отдельные варианты, которые работают при допущении, что строка содержит набор байт, представляющий текст в кодировке UTF-8. Например, функция length вычисляет длину строки в байтах, а функция lengthUTF8 - длину строки в кодовых точках Unicode, при допущении, что значение в кодировке UTF-8. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/string/) diff --git a/docs/ru/data_types/tuple.md b/docs/ru/data_types/tuple.md index 66aca46d939..cd954a2e477 100644 --- a/docs/ru/data_types/tuple.md +++ b/docs/ru/data_types/tuple.md @@ -51,3 +51,5 @@ SELECT 1 rows in set. Elapsed: 0.002 sec. ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/data_types/tuple/) diff --git a/docs/ru/development/style.md b/docs/ru/development/style.md index f5470b48fcd..4bcff77ca3a 100644 --- a/docs/ru/development/style.md +++ b/docs/ru/development/style.md @@ -836,3 +836,5 @@ function( const & RangesInDataParts ranges, size_t limit) ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/development/style/) diff --git a/docs/ru/faq/general.md b/docs/ru/faq/general.md index 15a19bbeb85..8cecad1aaf8 100644 --- a/docs/ru/faq/general.md +++ b/docs/ru/faq/general.md @@ -9,4 +9,5 @@ Распределённая сортировка не является оптимальным способом выполнения операции reduce, если результат выполнения операции и все промежуточные результаты, при их наличии, помещаются в оперативку на одном сервере, как обычно бывает в запросах, выполняющихся в режиме онлайн. В таком случае, оптимальным способом выполнения операции reduce является хэш-таблица. Частым способом оптимизации map-reduce задач является предагрегация (частичный reduce) с использованием хэш-таблицы в оперативной памяти. Эта оптимизация делается пользователем в ручном режиме. Распределённая сортировка является основной причиной тормозов при выполнении несложных map-reduce задач. -Большинство реализаций MapReduce позволяют выполнять произвольный код на кластере. Но для OLAP задач лучше подходит декларативный язык запросов, который позволяет быстро проводить исследования. Для примера, для Hadoop существует Hive и Pig. Также смотрите Cloudera Impala, Shark (устаревший) для Spark, а также Spark SQL, Presto, Apache Drill. Впрочем, производительность при выполнении таких задач является сильно неоптимальной по сравнению со специализированными системами, а сравнительно высокая latency не позволяет использовать эти системы в качестве бэкенда для веб-интерфейса. \ No newline at end of file +Большинство реализаций MapReduce позволяют выполнять произвольный код на кластере. Но для OLAP задач лучше подходит декларативный язык запросов, который позволяет быстро проводить исследования. Для примера, для Hadoop существует Hive и Pig. Также смотрите Cloudera Impala, Shark (устаревший) для Spark, а также Spark SQL, Presto, Apache Drill. Впрочем, производительность при выполнении таких задач является сильно неоптимальной по сравнению со специализированными системами, а сравнительно высокая latency не позволяет использовать эти системы в качестве бэкенда для веб-интерфейса. +[Оригинальная статья](https://clickhouse.yandex/docs/ru/faq/general/) diff --git a/docs/ru/getting_started/example_datasets/amplab_benchmark.md b/docs/ru/getting_started/example_datasets/amplab_benchmark.md index 602a12eaea1..87b8de2be43 100644 --- a/docs/ru/getting_started/example_datasets/amplab_benchmark.md +++ b/docs/ru/getting_started/example_datasets/amplab_benchmark.md @@ -22,7 +22,7 @@ cd .. Выполните следующие запросы к ClickHouse: -```sql +``` sql CREATE TABLE rankings_tiny ( pageURL String, @@ -97,7 +97,7 @@ for i in 5nodes/uservisits/*.deflate; do echo $i; zlib-flate -uncompress < $i | Запросы для получения выборок данных: -```sql +``` sql SELECT pageURL, pageRank FROM rankings_1node WHERE pageRank > 1000 SELECT substring(sourceIP, 1, 8), sum(adRevenue) FROM uservisits_1node GROUP BY substring(sourceIP, 1, 8) @@ -119,3 +119,5 @@ GROUP BY sourceIP ORDER BY totalRevenue DESC LIMIT 1 ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/getting_started/example_datasets/amplab_benchmark/) diff --git a/docs/ru/getting_started/example_datasets/criteo.md b/docs/ru/getting_started/example_datasets/criteo.md index 2d2f44242f8..32fc8e234dc 100644 --- a/docs/ru/getting_started/example_datasets/criteo.md +++ b/docs/ru/getting_started/example_datasets/criteo.md @@ -4,7 +4,7 @@ Создайте таблицу для импорта лога: -```sql +``` sql CREATE TABLE criteo_log (date Date, clicked UInt8, int1 Int32, int2 Int32, int3 Int32, int4 Int32, int5 Int32, int6 Int32, int7 Int32, int8 Int32, int9 Int32, int10 Int32, int11 Int32, int12 Int32, int13 Int32, cat1 String, cat2 String, cat3 String, cat4 String, cat5 String, cat6 String, cat7 String, cat8 String, cat9 String, cat10 String, cat11 String, cat12 String, cat13 String, cat14 String, cat15 String, cat16 String, cat17 String, cat18 String, cat19 String, cat20 String, cat21 String, cat22 String, cat23 String, cat24 String, cat25 String, cat26 String) ENGINE = Log ``` @@ -16,7 +16,7 @@ for i in {00..23}; do echo $i; zcat datasets/criteo/day_${i#0}.gz | sed -r 's/^/ Создайте таблицу для сконвертированных данных: -```sql +``` sql CREATE TABLE criteo ( date Date, @@ -65,8 +65,10 @@ CREATE TABLE criteo Преобразуем данные из сырого лога и положим во вторую таблицу: -```sql +``` sql INSERT INTO criteo SELECT date, clicked, int1, int2, int3, int4, int5, int6, int7, int8, int9, int10, int11, int12, int13, reinterpretAsUInt32(unhex(cat1)) AS icat1, reinterpretAsUInt32(unhex(cat2)) AS icat2, reinterpretAsUInt32(unhex(cat3)) AS icat3, reinterpretAsUInt32(unhex(cat4)) AS icat4, reinterpretAsUInt32(unhex(cat5)) AS icat5, reinterpretAsUInt32(unhex(cat6)) AS icat6, reinterpretAsUInt32(unhex(cat7)) AS icat7, reinterpretAsUInt32(unhex(cat8)) AS icat8, reinterpretAsUInt32(unhex(cat9)) AS icat9, reinterpretAsUInt32(unhex(cat10)) AS icat10, reinterpretAsUInt32(unhex(cat11)) AS icat11, reinterpretAsUInt32(unhex(cat12)) AS icat12, reinterpretAsUInt32(unhex(cat13)) AS icat13, reinterpretAsUInt32(unhex(cat14)) AS icat14, reinterpretAsUInt32(unhex(cat15)) AS icat15, reinterpretAsUInt32(unhex(cat16)) AS icat16, reinterpretAsUInt32(unhex(cat17)) AS icat17, reinterpretAsUInt32(unhex(cat18)) AS icat18, reinterpretAsUInt32(unhex(cat19)) AS icat19, reinterpretAsUInt32(unhex(cat20)) AS icat20, reinterpretAsUInt32(unhex(cat21)) AS icat21, reinterpretAsUInt32(unhex(cat22)) AS icat22, reinterpretAsUInt32(unhex(cat23)) AS icat23, reinterpretAsUInt32(unhex(cat24)) AS icat24, reinterpretAsUInt32(unhex(cat25)) AS icat25, reinterpretAsUInt32(unhex(cat26)) AS icat26 FROM criteo_log; DROP TABLE criteo_log; ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/getting_started/example_datasets/criteo/) diff --git a/docs/ru/getting_started/example_datasets/nyc_taxi.md b/docs/ru/getting_started/example_datasets/nyc_taxi.md index f66f37de89a..d0402b9e2f4 100644 --- a/docs/ru/getting_started/example_datasets/nyc_taxi.md +++ b/docs/ru/getting_started/example_datasets/nyc_taxi.md @@ -24,7 +24,7 @@ mv data/yellow_tripdata_2010-03.csv_ data/yellow_tripdata_2010-03.csv Проверить количество загруженных строк можно следующим образом: -```text +``` time psql nyc-taxi-data -c "SELECT count(*) FROM trips;" ## count 1298979494 @@ -39,7 +39,7 @@ real 7m9.164s Экспорт данных из PostgreSQL: -```sql +``` sql COPY ( SELECT trips.id, @@ -114,7 +114,7 @@ COPY Создание временной таблицы в ClickHouse: -```sql +``` sql CREATE TABLE trips ( trip_id UInt32, @@ -173,7 +173,7 @@ dropoff_puma Nullable(String) Она нужна для преобразование полей к более правильным типам данных и, если возможно, чтобы избавиться от NULL'ов. -```text +``` time clickhouse-client --query="INSERT INTO trips FORMAT TabSeparated" < trips.tsv real 75m56.214s @@ -191,7 +191,7 @@ real 75m56.214s Создадим и заполним итоговую таблицу: -```text +``` CREATE TABLE trips_mergetree ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) AS SELECT @@ -258,7 +258,7 @@ FROM trips Таблица заняла 126 Гб дискового пространства. -```text +``` :) SELECT formatReadableSize(sum(bytes)) FROM system.parts WHERE table = 'trips_mergetree' AND active SELECT formatReadableSize(sum(bytes)) @@ -276,7 +276,7 @@ WHERE (table = 'trips_mergetree') AND active Q1: -```sql +``` sql SELECT cab_type, count(*) FROM trips_mergetree GROUP BY cab_type ``` @@ -284,7 +284,7 @@ SELECT cab_type, count(*) FROM trips_mergetree GROUP BY cab_type Q2: -```sql +``` sql SELECT passenger_count, avg(total_amount) FROM trips_mergetree GROUP BY passenger_count ``` @@ -292,7 +292,7 @@ SELECT passenger_count, avg(total_amount) FROM trips_mergetree GROUP BY passenge Q3: -```sql +``` sql SELECT passenger_count, toYear(pickup_date) AS year, count(*) FROM trips_mergetree GROUP BY passenger_count, year ``` @@ -300,7 +300,7 @@ SELECT passenger_count, toYear(pickup_date) AS year, count(*) FROM trips_mergetr Q4: -```sql +``` sql SELECT passenger_count, toYear(pickup_date) AS year, round(trip_distance) AS distance, count(*) FROM trips_mergetree GROUP BY passenger_count, year, distance @@ -322,19 +322,19 @@ ORDER BY year, count(*) DESC На каждом сервере: -```text +``` CREATE TABLE default.trips_mergetree_third ( trip_id UInt32, vendor_id Enum8('1' = 1, '2' = 2, 'CMT' = 3, 'VTS' = 4, 'DDS' = 5, 'B02512' = 10, 'B02598' = 11, 'B02617' = 12, 'B02682' = 13, 'B02764' = 14), pickup_date Date, pickup_datetime DateTime, dropoff_date Date, dropoff_datetime DateTime, store_and_fwd_flag UInt8, rate_code_id UInt8, pickup_longitude Float64, pickup_latitude Float64, dropoff_longitude Float64, dropoff_latitude Float64, passenger_count UInt8, trip_distance Float64, fare_amount Float32, extra Float32, mta_tax Float32, tip_amount Float32, tolls_amount Float32, ehail_fee Float32, improvement_surcharge Float32, total_amount Float32, payment_type_ Enum8('UNK' = 0, 'CSH' = 1, 'CRE' = 2, 'NOC' = 3, 'DIS' = 4), trip_type UInt8, pickup FixedString(25), dropoff FixedString(25), cab_type Enum8('yellow' = 1, 'green' = 2, 'uber' = 3), pickup_nyct2010_gid UInt8, pickup_ctlabel Float32, pickup_borocode UInt8, pickup_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), pickup_ct2010 FixedString(6), pickup_boroct2010 FixedString(7), pickup_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), pickup_ntacode FixedString(4), pickup_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), pickup_puma UInt16, dropoff_nyct2010_gid UInt8, dropoff_ctlabel Float32, dropoff_borocode UInt8, dropoff_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), dropoff_ct2010 FixedString(6), dropoff_boroct2010 FixedString(7), dropoff_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), dropoff_ntacode FixedString(4), dropoff_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), dropoff_puma UInt16) ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) ``` На исходном сервере: -```sql +``` sql CREATE TABLE trips_mergetree_x3 AS trips_mergetree_third ENGINE = Distributed(perftest, default, trips_mergetree_third, rand()) ``` Следующим запрос перераспределит данные: -```sql +``` sql INSERT INTO trips_mergetree_x3 SELECT * FROM trips_mergetree ``` @@ -366,3 +366,5 @@ Q4: 0.072 sec. | 1 | 0.490 | 1.224 | 2.104 | 3.593 | | 3 | 0.212 | 0.438 | 0.733 | 1.241 | | 140 | 0.028 | 0.043 | 0.051 | 0.072 | + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/getting_started/example_datasets/nyc_taxi/) diff --git a/docs/ru/getting_started/example_datasets/ontime.md b/docs/ru/getting_started/example_datasets/ontime.md index afd4de06893..fa832c99847 100644 --- a/docs/ru/getting_started/example_datasets/ontime.md +++ b/docs/ru/getting_started/example_datasets/ontime.md @@ -18,7 +18,7 @@ done Создание таблицы: -```sql +``` sql CREATE TABLE `ontime` ( `Year` UInt16, `Quarter` UInt8, @@ -142,37 +142,37 @@ for i in *.zip; do echo $i; unzip -cq $i '*.csv' | sed 's/\.00//g' | clickhouse- Q0. -```sql +``` sql select avg(c1) from (select Year, Month, count(*) as c1 from ontime group by Year, Month); ``` Q1. Количество полетов в день с 2000 по 2008 года -```sql +``` sql SELECT DayOfWeek, count(*) AS c FROM ontime WHERE Year >= 2000 AND Year <= 2008 GROUP BY DayOfWeek ORDER BY c DESC; ``` Q2. Количество полетов, задержанных более чем на 10 минут, с группировкой по дням неделе, за 2000-2008 года -```sql +``` sql SELECT DayOfWeek, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Year <= 2008 GROUP BY DayOfWeek ORDER BY c DESC ``` Q3. Количество задержек по аэропортам за 2000-2008 -```sql +``` sql SELECT Origin, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Year <= 2008 GROUP BY Origin ORDER BY c DESC LIMIT 10 ``` Q4. Количество задержек по перевозчикам за 2007 год -```sql +``` sql SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC ``` Q5. Процент задержек по перевозчикам за 2007 год -```sql +``` sql SELECT Carrier, c, c2, c*1000/c2 as c3 FROM ( @@ -198,13 +198,13 @@ ORDER BY c3 DESC; Более оптимальная версия того же запроса: -```sql +``` sql SELECT Carrier, avg(DepDelay > 10) * 1000 AS c3 FROM ontime WHERE Year = 2007 GROUP BY Carrier ORDER BY Carrier ``` Q6. Предыдущий запрос за более широкий диапазон лет, 2000-2008 -```sql +``` sql SELECT Carrier, c, c2, c*1000/c2 as c3 FROM ( @@ -230,13 +230,13 @@ ORDER BY c3 DESC; Более оптимальная версия того же запроса: -```sql +``` sql SELECT Carrier, avg(DepDelay > 10) * 1000 AS c3 FROM ontime WHERE Year >= 2000 AND Year <= 2008 GROUP BY Carrier ORDER BY Carrier ``` Q7. Процент полетов, задержанных на более 10 минут, в разбивке по годам -```sql +``` sql SELECT Year, c1/c2 FROM ( @@ -260,25 +260,25 @@ ORDER BY Year Более оптимальная версия того же запроса: -```sql +``` sql SELECT Year, avg(DepDelay > 10) FROM ontime GROUP BY Year ORDER BY Year ``` Q8. Самые популярные направления по количеству напрямую соединенных городов для различных диапазонов лет -```sql +``` sql SELECT DestCityName, uniqExact(OriginCityName) AS u FROM ontime WHERE Year >= 2000 and Year <= 2010 GROUP BY DestCityName ORDER BY u DESC LIMIT 10; ``` Q9. -```sql +``` sql select Year, count(*) as c1 from ontime group by Year; ``` Q10. -```sql +``` sql select min(Year), max(Year), Carrier, count(*) as cnt, sum(ArrDelayMinutes>30) as flights_delayed, @@ -296,7 +296,7 @@ LIMIT 1000; Бонус: -```sql +``` sql SELECT avg(cnt) FROM (SELECT Year,Month,count(*) AS cnt FROM ontime WHERE DepDel15=1 GROUP BY Year,Month) select avg(c1) from (select Year,Month,count(*) as c1 from ontime group by Year,Month) @@ -316,3 +316,5 @@ SELECT OriginCityName, count() AS c FROM ontime GROUP BY OriginCityName ORDER BY - - - + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/getting_started/example_datasets/ontime/) diff --git a/docs/ru/getting_started/example_datasets/star_schema.md b/docs/ru/getting_started/example_datasets/star_schema.md index 279daed3fdd..ddff4c59294 100644 --- a/docs/ru/getting_started/example_datasets/star_schema.md +++ b/docs/ru/getting_started/example_datasets/star_schema.md @@ -21,7 +21,7 @@ make Создание таблиц в ClickHouse: -```sql +``` sql CREATE TABLE lineorder ( LO_ORDERKEY UInt32, LO_LINENUMBER UInt8, @@ -82,3 +82,5 @@ CREATE TABLE partd AS part ENGINE = Distributed(perftest_3shards_1replicas, defa cat customer.tbl | sed 's/$/2000-01-01/' | clickhouse-client --query "INSERT INTO customer FORMAT CSV" cat lineorder.tbl | clickhouse-client --query "INSERT INTO lineorder FORMAT CSV" ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/getting_started/example_datasets/star_schema/) diff --git a/docs/ru/getting_started/example_datasets/wikistat.md b/docs/ru/getting_started/example_datasets/wikistat.md index 5f07989262e..ed8037ffc8e 100644 --- a/docs/ru/getting_started/example_datasets/wikistat.md +++ b/docs/ru/getting_started/example_datasets/wikistat.md @@ -4,7 +4,7 @@ Создание таблицы: -```sql +``` sql CREATE TABLE wikistat ( date Date, @@ -24,3 +24,5 @@ for i in {2007..2016}; do for j in {01..12}; do echo $i-$j >&2; curl -sSL "http: cat links.txt | while read link; do wget http://dumps.wikimedia.org/other/pagecounts-raw/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1/')/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1-\2/')/$link; done ls -1 /opt/wikistat/ | grep gz | while read i; do echo $i; gzip -cd /opt/wikistat/$i | ./wikistat-loader --time="$(echo -n $i | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})([0-9]{2})-([0-9]{2})([0-9]{2})([0-9]{2})\.gz/\1-\2-\3 \4-00-00/')" | clickhouse-client --query="INSERT INTO wikistat FORMAT TabSeparated"; done ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/getting_started/example_datasets/wikistat/) diff --git a/docs/ru/getting_started/index.md b/docs/ru/getting_started/index.md index 18e006f01c3..037dfb7379b 100644 --- a/docs/ru/getting_started/index.md +++ b/docs/ru/getting_started/index.md @@ -24,7 +24,7 @@ ClickHouse также работает на FreeBSD и Mac OS X; может бы Пропишите в `/etc/apt/sources.list` (или в отдельный файл `/etc/apt/sources.list.d/clickhouse.list`) репозитории: -```text +``` deb http://repo.yandex.ru/clickhouse/deb/stable/ main/ ``` @@ -51,14 +51,14 @@ ClickHouse содержит настройки ограничения досту Вы можете собрать пакеты и установить их. Также вы можете использовать программы без установки пакетов. -```text +``` Client: dbms/programs/clickhouse-client Server: dbms/programs/clickhouse-server ``` Для сервера создаёте директории с данными, например: -```text +``` /opt/clickhouse/data/default/ /opt/clickhouse/metadata/default/ ``` @@ -136,3 +136,5 @@ SELECT 1 **Поздравляем, система работает!** Для дальнейших экспериментов можно попробовать загрузить из тестовых наборов данных. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/getting_started/) diff --git a/docs/ru/index.md b/docs/ru/index.md index f93ba69574f..a2f6945bd04 100644 --- a/docs/ru/index.md +++ b/docs/ru/index.md @@ -52,7 +52,7 @@ ClickHouse - столбцовая система управления базам - транзакции отсутствуют; - низкие требования к консистентности данных; - в запросе одна большая таблица, все таблицы кроме одной маленькие; -- результат выполнения запроса существенно меньше исходных данных - то есть, данные фильтруются или агрегируются; результат выполнения помещается в оперативку на одном сервере; +- результат выполнения запроса существенно меньше исходных данных - то есть, данные фильтруются или агрегируются; результат выполнения помещается в оперативку на одном сервере. Легко видеть, что OLAP сценарий работы существенно отличается от других распространённых сценариев работы (например, OLTP или Key-Value сценариев работы). Таким образом, не имеет никакого смысла пытаться использовать OLTP или Key-Value БД для обработки аналитических запросов, если вы хотите получить приличную производительность ("выше плинтуса"). Например, если вы попытаетесь использовать для аналитики MongoDB или Redis - вы получите анекдотически низкую производительность по сравнению с OLAP-СУБД. @@ -78,9 +78,8 @@ ClickHouse - столбцовая система управления базам Для примера, для запроса "посчитать количество записей для каждой рекламной системы", требуется прочитать один столбец "идентификатор рекламной системы", который занимает 1 байт в несжатом виде. Если большинство переходов было не с рекламных систем, то можно рассчитывать хотя бы на десятикратное сжатие этого столбца. При использовании быстрого алгоритма сжатия, возможно разжатие данных со скоростью более нескольких гигабайт несжатых данных в секунду. То есть, такой запрос может выполняться со скоростью около нескольких миллиардов строк в секунду на одном сервере. На практике, такая скорость действительно достигается. -
Пример -

-

+
Пример +``` $ clickhouse-client ClickHouse client version 0.0.52053. Connecting to localhost:9000. @@ -122,8 +121,7 @@ LIMIT 20 20 rows in set. Elapsed: 0.153 sec. Processed 1.00 billion rows, 4.00 GB (6.53 billion rows/s., 26.10 GB/s.) :) -
-

+```
### По вычислениям @@ -140,3 +138,5 @@ LIMIT 20 В "обычных" БД этого не делается, так как не имеет смысла при выполнении простых запросов. Хотя есть исключения. Например, в MemSQL кодогенерация используется для уменьшения latency при выполнении SQL запросов. (Для сравнения - в аналитических СУБД, требуется оптимизация throughput, а не latency). Стоит заметить, что для эффективности по CPU требуется, чтобы язык запросов был декларативным (SQL, MDX) или хотя бы векторным (J, K). То есть, чтобы запрос содержал циклы только в неявном виде, открывая возможности для оптимизации. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/) diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index 09f21a64f83..ec7bc7d2c1b 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -115,4 +115,5 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA username password -``` \ No newline at end of file +``` +[Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/cli/) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 6944713c995..8d31b6707d7 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -32,32 +32,130 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT [XML](#xml) | ✗ | ✔ | [CapnProto](#capnproto) | ✔ | ✔ | - + -## CapnProto +## TabSeparated -Cap'n Proto - формат бинарных сообщений, похож на Protocol Buffers и Thrift, но не похож на JSON или MessagePack. +В TabSeparated формате данные пишутся по строкам. Каждая строчка содержит значения, разделённые табами. После каждого значения идёт таб, кроме последнего значения в строке, после которого идёт перевод строки. Везде подразумеваются исключительно unix-переводы строк. Последняя строка также обязана содержать перевод строки на конце. Значения пишутся в текстовом виде, без обрамляющих кавычек, с экранированием служебных символов. -Сообщения Cap'n Proto строго типизированы и не самоописывающиеся, т.е. нуждаются во внешнем описании схемы. Схема применяется "на лету" и кешируется для каждого запроса. +Этот формат также доступен под именем `TSV`. -```sql -SELECT SearchPhrase, count() AS c FROM test.hits - GROUP BY SearchPhrase FORMAT CapnProto SETTINGS schema = 'schema:Message' +Формат `TabSeparated` удобен для обработки данных произвольными программами и скриптами. Он используется по умолчанию в HTTP-интерфейсе, а также в batch-режиме клиента командной строки. Также формат позволяет переносить данные между разными СУБД. Например, вы можете получить дамп из MySQL и загрузить его в ClickHouse, или наоборот. + +Формат `TabSeparated` поддерживает вывод тотальных значений (при использовании WITH TOTALS) и экстремальных значений (при настройке extremes выставленной в 1). В этих случаях, после основных данных выводятся тотальные значения, и экстремальные значения. Основной результат, тотальные значения и экстремальные значения, отделяются друг от друга пустой строкой. Пример: + +``` sql +SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT TabSeparated`` ``` -Где `schema.capnp` выглядит следующим образом: - ``` -struct Message { - SearchPhrase @0 :Text; - c @1 :Uint64; -} +2014-03-17 1406958 +2014-03-18 1383658 +2014-03-19 1405797 +2014-03-20 1353623 +2014-03-21 1245779 +2014-03-22 1031592 +2014-03-23 1046491 + +0000-00-00 8873898 + +2014-03-17 1031592 +2014-03-23 1406958 ``` +### Форматирование данных -Файлы со схемами находятся в файле, который находится в каталоге указанном в параметре [format_schema_path](../operations/server_settings/settings.md#server_settings-format_schema_path) конфигурации сервера. +Целые числа пишутся в десятичной форме. Числа могут содержать лишний символ "+" в начале (игнорируется при парсинге, а при форматировании не пишется). Неотрицательные числа не могут содержать знак отрицания. При чтении допустим парсинг пустой строки, как числа ноль, или (для знаковых типов) строки, состоящей из одного минуса, как числа ноль. Числа, не помещающиеся в соответствующий тип данных, могут парсится, как некоторое другое число, без сообщения об ошибке. + +Числа с плавающей запятой пишутся в десятичной форме. При этом, десятичный разделитель - точка. Поддерживается экспоненциальная запись, а также inf, +inf, -inf, nan. Запись числа с плавающей запятой может начинаться или заканчиваться на десятичную точку. +При форматировании возможна потеря точности чисел с плавающей запятой. +При парсинге, допустимо чтение не обязательно наиболее близкого к десятичной записи машинно-представимого числа. + +Даты выводятся в формате YYYY-MM-DD, парсятся в том же формате, но с любыми символами в качестве разделителей. +Даты-с-временем выводятся в формате YYYY-MM-DD hh:mm:ss, парсятся в том же формате, но с любыми символами в качестве разделителей. +Всё это происходит в системном часовом поясе на момент старта клиента (если клиент занимается форматированием данных) или сервера. Для дат-с-временем не указывается, действует ли daylight saving time. То есть, если в дампе есть времена во время перевода стрелок назад, то дамп не соответствует данным однозначно, и при парсинге будет выбрано какое-либо из двух времён. +При парсинге, некорректные даты и даты-с-временем могут парситься с естественным переполнением или как нулевые даты/даты-с-временем без сообщения об ошибке. + +В качестве исключения, поддерживается также парсинг даты-с-временем в формате unix timestamp, если он состоит ровно из 10 десятичных цифр. Результат не зависит от часового пояса. Различение форматов YYYY-MM-DD hh:mm:ss и NNNNNNNNNN делается автоматически. + +Строки выводятся с экранированием спец-символов с помощью обратного слеша. При выводе, используются следующие escape-последовательности: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\'`, `\\`. При парсинге, также поддерживаются последовательности `\a`, `\v`, а также `\xHH` (hex escape-последовательности) и любые последовательности вида `\c`, где `c` - любой символ - такие последовательности преобразуется в `c`. Таким образом, при чтении поддерживаются форматы, где перевод строки может быть записан как `\n` и как `\` и перевод строки. Например, строка `Hello world`, где между словами вместо пробела стоит перевод строки, может быть считана в любом из следующих вариантов: + +``` +Hello\nworld + +Hello\ +world +``` + +Второй вариант поддерживается, так как его использует MySQL при записи tab-separated дампа. + +Минимальный набор символов, которых вам необходимо экранировать при передаче в TabSeparated формате: таб, перевод строки (LF) и обратный слеш. + +Экранируется лишь небольшой набор символов. Вы можете легко наткнуться на строковое значение, которое испортит ваш терминал при выводе в него. + +Массивы форматируются в виде списка значений через запятую в квадратных скобках. Элементы массива - числа форматируются как обычно, а даты, даты-с-временем и строки - в одинарных кавычках с такими же правилами экранирования, как указано выше. + +[NULL](../query_language/syntax.md#null-literal) форматируется в виде `\N`. + + + +## TabSeparatedRaw + +Отличается от формата `TabSeparated` тем, что строки выводятся без экранирования. +Этот формат подходит только для вывода результата выполнения запроса, но не для парсинга (приёма данных для вставки в таблицу). + +Этот формат также доступен под именем `TSVRaw`. + + +## TabSeparatedWithNames + +Отличается от формата `TabSeparated` тем, что в первой строке пишутся имена столбцов. +При парсинге, первая строка полностью игнорируется: вы не можете использовать имена столбцов, чтобы указать их порядок расположения, или чтобы проверить их корректность. +(Поддержка обработки заголовка при парсинге может быть добавлена в будущем.) + +Этот формат также доступен под именем `TSVWithNames`. + + +## TabSeparatedWithNamesAndTypes + +Отличается от формата `TabSeparated` тем, что в первой строке пишутся имена столбцов, а во второй - типы столбцов. +При парсинге, первая и вторая строка полностью игнорируется. + +Этот формат также доступен под именем `TSVWithNamesAndTypes`. + + +## TSKV + +Похож на TabSeparated, но выводит значения в формате name=value. Имена экранируются так же, как строки в формате TabSeparated и, дополнительно, экранируется также символ =. + +``` +SearchPhrase= count()=8267016 +SearchPhrase=интерьер ванной комнаты count()=2166 +SearchPhrase=яндекс count()=1655 +SearchPhrase=весна 2014 мода count()=1549 +SearchPhrase=фриформ фото count()=1480 +SearchPhrase=анджелина джоли count()=1245 +SearchPhrase=омск count()=1112 +SearchPhrase=фото собак разных пород count()=1091 +SearchPhrase=дизайн штор count()=1064 +SearchPhrase=баку count()=1000 +``` + +[NULL](../query_language/syntax.md#null-literal) форматируется в виде `\N`. + +``` sql +SELECT * FROM t_null FORMAT TSKV +``` +``` +x=1 y=\N +``` + +При большом количестве маленьких столбцов, этот формат существенно неэффективен, и обычно нет причин его использовать. Он реализован, так как используется в некоторых отделах Яндекса. + +Поддерживается как вывод, так и парсинг данных в этом формате. При парсинге, поддерживается расположение значений разных столбцов в произвольном порядке. Допустимо отсутствие некоторых значений - тогда они воспринимаются как равные значениям по умолчанию. При этом, в качестве значений по умолчанию используются нули, пустые строки и не поддерживаются сложные значения по умолчанию, которые могут быть заданы в таблице. + +При парсинге, в качестве дополнительного поля, может присутствовать `tskv` без знака равенства и без значения. Это поле игнорируется. -Десериализация эффективна и обычно не повышает нагрузку на систему. ## CSV @@ -87,7 +185,7 @@ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMA Выводит данные в формате JSON. Кроме таблицы с данными, также выводятся имена и типы столбцов, и некоторая дополнительная информация - общее количество выведенных строк, а также количество строк, которое могло бы быть выведено, если бы не было LIMIT-а. Пример: -```sql +``` sql SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTALS ORDER BY c DESC LIMIT 5 FORMAT JSON ``` @@ -264,7 +362,7 @@ ClickHouse поддерживает [NULL](../query_language/syntax.md#null-lite [NULL](../query_language/syntax.md#null-literal) выводится как `ᴺᵁᴸᴸ`. -```sql +``` sql SELECT * FROM t_null ``` ``` @@ -278,11 +376,11 @@ SELECT * FROM t_null Формат Pretty поддерживает вывод тотальных значений (при использовании WITH TOTALS) и экстремальных значений (при настройке extremes выставленной в 1). В этих случаях, после основных данных выводятся тотальные значения, и экстремальные значения, в отдельных табличках. Пример (показан для формата PrettyCompact): -```sql +``` sql SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT PrettyCompact ``` -```text +``` ┌──EventDate─┬───────c─┐ │ 2014-03-17 │ 1406958 │ │ 2014-03-18 │ 1383658 │ @@ -359,133 +457,6 @@ Array представлены как длина в формате varint (unsig Для поддержки [NULL](../query_language/syntax.md#null-literal) перед каждым значением типа [Nullable](../data_types/nullable.md#data_type-nullable) в строке добавлен дополнительный байт, который содержит 1 или 0. Если 1, то значение — `NULL` и этот байт трактуется как отдельное значение. Если 0, то после байта идёт не `NULL`-значение. - - -## TabSeparated - -В TabSeparated формате данные пишутся по строкам. Каждая строчка содержит значения, разделённые табами. После каждого значения идёт таб, кроме последнего значения в строке, после которого идёт перевод строки. Везде подразумеваются исключительно unix-переводы строк. Последняя строка также обязана содержать перевод строки на конце. Значения пишутся в текстовом виде, без обрамляющих кавычек, с экранированием служебных символов. - -Этот формат также доступен под именем `TSV`. - -Формат `TabSeparated` удобен для обработки данных произвольными программами и скриптами. Он используется по умолчанию в HTTP-интерфейсе, а также в batch-режиме клиента командной строки. Также формат позволяет переносить данные между разными СУБД. Например, вы можете получить дамп из MySQL и загрузить его в ClickHouse, или наоборот. - -Формат `TabSeparated` поддерживает вывод тотальных значений (при использовании WITH TOTALS) и экстремальных значений (при настройке extremes выставленной в 1). В этих случаях, после основных данных выводятся тотальные значения, и экстремальные значения. Основной результат, тотальные значения и экстремальные значения, отделяются друг от друга пустой строкой. Пример: - -```sql -SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT TabSeparated`` -``` - -```text -2014-03-17 1406958 -2014-03-18 1383658 -2014-03-19 1405797 -2014-03-20 1353623 -2014-03-21 1245779 -2014-03-22 1031592 -2014-03-23 1046491 - -0000-00-00 8873898 - -2014-03-17 1031592 -2014-03-23 1406958 -``` - - - -## Форматирование данных - -Целые числа пишутся в десятичной форме. Числа могут содержать лишний символ "+" в начале (игнорируется при парсинге, а при форматировании не пишется). Неотрицательные числа не могут содержать знак отрицания. При чтении допустим парсинг пустой строки, как числа ноль, или (для знаковых типов) строки, состоящей из одного минуса, как числа ноль. Числа, не помещающиеся в соответствующий тип данных, могут парсится, как некоторое другое число, без сообщения об ошибке. - -Числа с плавающей запятой пишутся в десятичной форме. При этом, десятичный разделитель - точка. Поддерживается экспоненциальная запись, а также inf, +inf, -inf, nan. Запись числа с плавающей запятой может начинаться или заканчиваться на десятичную точку. -При форматировании возможна потеря точности чисел с плавающей запятой. -При парсинге, допустимо чтение не обязательно наиболее близкого к десятичной записи машинно-представимого числа. - -Даты выводятся в формате YYYY-MM-DD, парсятся в том же формате, но с любыми символами в качестве разделителей. -Даты-с-временем выводятся в формате YYYY-MM-DD hh:mm:ss, парсятся в том же формате, но с любыми символами в качестве разделителей. -Всё это происходит в системном часовом поясе на момент старта клиента (если клиент занимается форматированием данных) или сервера. Для дат-с-временем не указывается, действует ли daylight saving time. То есть, если в дампе есть времена во время перевода стрелок назад, то дамп не соответствует данным однозначно, и при парсинге будет выбрано какое-либо из двух времён. -При парсинге, некорректные даты и даты-с-временем могут парситься с естественным переполнением или как нулевые даты/даты-с-временем без сообщения об ошибке. - -В качестве исключения, поддерживается также парсинг даты-с-временем в формате unix timestamp, если он состоит ровно из 10 десятичных цифр. Результат не зависит от часового пояса. Различение форматов YYYY-MM-DD hh:mm:ss и NNNNNNNNNN делается автоматически. - -Строки выводятся с экранированием спец-символов с помощью обратного слеша. При выводе, используются следующие escape-последовательности: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\'`, `\\`. При парсинге, также поддерживаются последовательности `\a`, `\v`, а также `\xHH` (hex escape-последовательности) и любые последовательности вида `\c`, где `c` - любой символ - такие последовательности преобразуется в `c`. Таким образом, при чтении поддерживаются форматы, где перевод строки может быть записан как `\n` и как `\` и перевод строки. Например, строка `Hello world`, где между словами вместо пробела стоит перевод строки, может быть считана в любом из следующих вариантов: - -```text -Hello\nworld - -Hello\ -world -``` - -Второй вариант поддерживается, так как его использует MySQL при записи tab-separated дампа. - -Минимальный набор символов, которых вам необходимо экранировать при передаче в TabSeparated формате: таб, перевод строки (LF) и обратный слеш. - -Экранируется лишь небольшой набор символов. Вы можете легко наткнуться на строковое значение, которое испортит ваш терминал при выводе в него. - -Массивы форматируются в виде списка значений через запятую в квадратных скобках. Элементы массива - числа форматируются как обычно, а даты, даты-с-временем и строки - в одинарных кавычках с такими же правилами экранирования, как указано выше. - -[NULL](../query_language/syntax.md#null-literal) форматируется в виде `\N`. - - - -## TabSeparatedRaw - -Отличается от формата `TabSeparated` тем, что строки выводятся без экранирования. -Этот формат подходит только для вывода результата выполнения запроса, но не для парсинга (приёма данных для вставки в таблицу). - -Этот формат также доступен под именем `TSVRaw`. - - -## TabSeparatedWithNames - -Отличается от формата `TabSeparated` тем, что в первой строке пишутся имена столбцов. -При парсинге, первая строка полностью игнорируется: вы не можете использовать имена столбцов, чтобы указать их порядок расположения, или чтобы проверить их корректность. -(Поддержка обработки заголовка при парсинге может быть добавлена в будущем.) - -Этот формат также доступен под именем `TSVWithNames`. - - -## TabSeparatedWithNamesAndTypes - -Отличается от формата `TabSeparated` тем, что в первой строке пишутся имена столбцов, а во второй - типы столбцов. -При парсинге, первая и вторая строка полностью игнорируется. - -Этот формат также доступен под именем `TSVWithNamesAndTypes`. - - -## TSKV - -Похож на TabSeparated, но выводит значения в формате name=value. Имена экранируются так же, как строки в формате TabSeparated и, дополнительно, экранируется также символ =. - -```text -SearchPhrase= count()=8267016 -SearchPhrase=интерьер ванной комнаты count()=2166 -SearchPhrase=яндекс count()=1655 -SearchPhrase=весна 2014 мода count()=1549 -SearchPhrase=фриформ фото count()=1480 -SearchPhrase=анджелина джоли count()=1245 -SearchPhrase=омск count()=1112 -SearchPhrase=фото собак разных пород count()=1091 -SearchPhrase=дизайн штор count()=1064 -SearchPhrase=баку count()=1000 -``` - -[NULL](../query_language/syntax.md#null-literal) форматируется в виде `\N`. - -```sql -SELECT * FROM t_null FORMAT TSKV -``` -``` -x=1 y=\N -``` - -При большом количестве маленьких столбцов, этот формат существенно неэффективен, и обычно нет причин его использовать. Он реализован, так как используется в некоторых отделах Яндекса. - -Поддерживается как вывод, так и парсинг данных в этом формате. При парсинге, поддерживается расположение значений разных столбцов в произвольном порядке. Допустимо отсутствие некоторых значений - тогда они воспринимаются как равные значениям по умолчанию. При этом, в качестве значений по умолчанию используются нули, пустые строки и не поддерживаются сложные значения по умолчанию, которые могут быть заданы в таблице. - -При парсинге, в качестве дополнительного поля, может присутствовать `tskv` без знака равенства и без значения. Это поле игнорируется. - - ## Values Выводит каждую строку в скобках. Строки разделены запятыми. После последней строки запятой нет. Значения внутри скобок также разделены запятыми. Числа выводятся в десятичном виде без кавычек. Массивы выводятся в квадратных скобках. Строки, даты, даты-с-временем выводятся в кавычках. Правила экранирования и особенности парсинга аналогичны формату [TabSeparated](#tabseparated). При форматировании, лишние пробелы не ставятся, а при парсинге - допустимы и пропускаются (за исключением пробелов внутри значений типа массив, которые недопустимы). [NULL](../query_language/syntax.md#null-literal) представляется как `NULL`. @@ -504,7 +475,7 @@ x=1 y=\N Пример: -```sql +``` sql SELECT * FROM t_null FORMAT Vertical ``` ``` @@ -619,3 +590,32 @@ test: string with \'quotes\' and \t with some special \n characters Массивы выводятся как `HelloWorld...`, а кортежи как `HelloWorld...`. + + + +## CapnProto + +Cap'n Proto - формат бинарных сообщений, похож на Protocol Buffers и Thrift, но не похож на JSON или MessagePack. + +Сообщения Cap'n Proto строго типизированы и не самоописывающиеся, т.е. нуждаются во внешнем описании схемы. Схема применяется "на лету" и кешируется для каждого запроса. + +``` sql +SELECT SearchPhrase, count() AS c FROM test.hits + GROUP BY SearchPhrase FORMAT CapnProto SETTINGS schema = 'schema:Message' +``` + +Где `schema.capnp` выглядит следующим образом: + +``` +struct Message { + SearchPhrase @0 :Text; + c @1 :Uint64; +} +``` + + +Файлы со схемами находятся в файле, который находится в каталоге указанном в параметре [format_schema_path](../operations/server_settings/settings.md#server_settings-format_schema_path) конфигурации сервера. + +Десериализация эффективна и обычно не повышает нагрузку на систему. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/formats/) diff --git a/docs/ru/interfaces/http_interface.md b/docs/ru/interfaces/http_interface.md index 6aa2d786ae9..6c8c0de1c11 100644 --- a/docs/ru/interfaces/http_interface.md +++ b/docs/ru/interfaces/http_interface.md @@ -219,3 +219,5 @@ curl -sS 'http://localhost:8123/?max_result_bytes=4000000&buffer_size=3000000&wa ``` Буферизация позволяет избежать ситуации когда код ответа и HTTP-заголовки были отправлены клиенту, после чего возникла ошибка выполнения запроса. В такой ситуации сообщение об ошибке записывается в конце тела ответа, и на стороне клиента ошибка может быть обнаружена только на этапе парсинга. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/http_interface/) diff --git a/docs/ru/interfaces/index.md b/docs/ru/interfaces/index.md index 348cf975fce..4560cda1fb7 100644 --- a/docs/ru/interfaces/index.md +++ b/docs/ru/interfaces/index.md @@ -3,3 +3,5 @@ # Интерфейсы Для изучения возможностей системы, загрузки данных в таблицы, ручных запросов, используйте программу clickhouse-client. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/) diff --git a/docs/ru/interfaces/jdbc.md b/docs/ru/interfaces/jdbc.md index 90051ad464f..8c54419d34a 100644 --- a/docs/ru/interfaces/jdbc.md +++ b/docs/ru/interfaces/jdbc.md @@ -2,3 +2,5 @@ - [Официальный драйвер](https://github.com/yandex/clickhouse-jdbc). - Драйвер от сторонней огранизации [ClickHouse-Native-JDBC](https://github.com/housepower/ClickHouse-Native-JDBC). + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/jdbc/) diff --git a/docs/ru/interfaces/tcp.md b/docs/ru/interfaces/tcp.md index e73a90ddff3..98672b505e4 100644 --- a/docs/ru/interfaces/tcp.md +++ b/docs/ru/interfaces/tcp.md @@ -1,3 +1,5 @@ # Родной интерфейс (TCP) Родной интерфейс используется в клиенте командной строки clickhouse-client, при межсерверном взаимодействии для распределённой обработки запроса, а также в программах на C++. Будет рассмотрен только клиент командной строки. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/tcp/) diff --git a/docs/ru/interfaces/third-party_client_libraries.md b/docs/ru/interfaces/third-party_client_libraries.md index c12522a196e..6cebd98271d 100644 --- a/docs/ru/interfaces/third-party_client_libraries.md +++ b/docs/ru/interfaces/third-party_client_libraries.md @@ -7,6 +7,7 @@ - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) - [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver) - [clickhouse-client](https://github.com/yurial/clickhouse-client) + - [aiochclient](https://github.com/maximdanilchenko/aiochclient) - PHP - [phpClickHouse](https://github.com/smi2/phpClickHouse) - [clickhouse-php-client](https://github.com/8bitov/clickhouse-php-client) @@ -42,5 +43,9 @@ - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) - Java - [clickhouse-client-java](https://github.com/VirtusAI/clickhouse-client-java) +- Kotlin + - [AORM](https://github.com/TanVD/AORM) - Nim - [nim-clickhouse](https://github.com/leonardoce/nim-clickhouse) + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/third-party_client_libraries/) diff --git a/docs/ru/interfaces/third-party_gui.md b/docs/ru/interfaces/third-party_gui.md index f6097806618..33b3e59ab8d 100644 --- a/docs/ru/interfaces/third-party_gui.md +++ b/docs/ru/interfaces/third-party_gui.md @@ -6,10 +6,10 @@ Основные возможности: -- Работает с ClickHouse напрямую из браузера, без необходимости установки дополнительного ПО. -- Редактор запросов с подсветкой синтаксиса. -- Автодополнение команд. -- Инструменты графического анализа выполнения запросов. +- Работает с ClickHouse напрямую из браузера, без необходимости установки дополнительного ПО; +- Редактор запросов с подсветкой синтаксиса; +- Автодополнение команд; +- Инструменты графического анализа выполнения запросов; - Цветовые схемы на выбор. [Документация Tabix](https://tabix.io/doc/). @@ -20,28 +20,33 @@ Основные возможности: -- Построение запросов с подсветкой синтаксиса. Просмотр ответа в табличном или JSON представлении. -- Экспортирование результатов запроса в формате CSV или JSON. -- Список процессов с описанием. Режим записи. Возможность остановки (`KILL`) процесса. -- Граф базы данных. Показывает все таблицы и их столбцы с дополнительной информацией. -- Быстрый просмотр размера столбца. +- Построение запросов с подсветкой синтаксиса; +- Просмотр ответа в табличном или JSON представлении; +- Экспортирование результатов запроса в формате CSV или JSON; +- Список процессов с описанием; +- Режим записи; +- Возможность остановки (`KILL`) запроса; +- Граф базы данных. Показывает все таблицы и их столбцы с дополнительной информацией; +- Быстрый просмотр размера столбца; - Конфигурирование сервера. Планируется разработка следующих возможностей: -- Управление базами. -- Управление пользователями. -- Анализ данных в режиме реального времени. -- Мониторинг кластера. -- Управление кластером. +- Управление базами; +- Управление пользователями; +- Анализ данных в режиме реального времени; +- Мониторинг кластера; +- Управление кластером; - Мониторинг реплицированных и Kafka таблиц. ## DBeaver -[DBeaver](https://dbeaver.io/) - Универсальный клиент баз данных +[DBeaver](https://dbeaver.io/) - универсальный desktop клиент баз данных с поддержкой ClickHouse. Основные возможности: -- Построение запросов с подсветкой синтаксиса -- Просмотр таблиц -- Автодополнение команд +- Построение запросов с подсветкой синтаксиса; +- Просмотр таблиц; +- Автодополнение команд. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/third-party_gui/) diff --git a/docs/ru/introduction/distinctive_features.md b/docs/ru/introduction/distinctive_features.md index 7851e580985..db00c6af6c9 100644 --- a/docs/ru/introduction/distinctive_features.md +++ b/docs/ru/introduction/distinctive_features.md @@ -60,3 +60,5 @@ ClickHouse предоставляет различные способы разм Используется асинхронная multimaster репликация. После записи на любую доступную реплику, данные распространяются на все остальные реплики в фоне. Система поддерживает полную идентичность данных на разных репликах. Восстановление после большинства сбоев осуществляется автоматически, а в сложных случаях — полуавтоматически. При необходимости, можно [включить кворумную запись](../operations/settings/settings.md#setting-insert_quorum) данных. Подробнее смотрите раздел [Репликация данных](../operations/table_engines/replication.md#table_engines-replication). + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/introduction/distinctive_features/) diff --git a/docs/ru/introduction/features_considered_disadvantages.md b/docs/ru/introduction/features_considered_disadvantages.md index b7ac877cc32..9e04f747c10 100644 --- a/docs/ru/introduction/features_considered_disadvantages.md +++ b/docs/ru/introduction/features_considered_disadvantages.md @@ -4,3 +4,5 @@ 2. Возможность изменять или удалять ранее записанные данные с низкими задержками и высокой частотой запросов не предоставляется. Есть массовое удаление данных для очистки более не нужного или соответствия [GDPR](https://gdpr-info.eu). Массовое изменение данных находится в разработке (на момент июля 2018). 3. Разреженный индекс делает ClickHouse плохо пригодным для точечных чтений одиночных строк по своим ключам. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/introduction/features_considered_disadvantages/) diff --git a/docs/ru/introduction/performance.md b/docs/ru/introduction/performance.md index 95e1d1cd008..f9ac886971f 100644 --- a/docs/ru/introduction/performance.md +++ b/docs/ru/introduction/performance.md @@ -21,3 +21,5 @@ ## Производительность при вставке данных Данные рекомендуется вставлять пачками не менее 1000 строк или не более одного запроса в секунду. При вставке в таблицу типа MergeTree из tab-separated дампа, скорость вставки будет в районе 50-200 МБ/сек. Если вставляются строчки размером около 1 КБ, то скорость будет в районе 50 000 - 200 000 строчек в секунду. Если строчки маленькие - производительность в строчках в секунду будет выше (на данных БК - `>` 500 000 строк в секунду, на данных Graphite - `>` 1 000 000 строк в секунду). Для увеличения производительности, можно производить несколько запросов INSERT параллельно - при этом производительность растёт линейно. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/introduction/performance/) diff --git a/docs/ru/introduction/ya_metrika_task.md b/docs/ru/introduction/ya_metrika_task.md index b4030376ba0..c7e22346ae5 100644 --- a/docs/ru/introduction/ya_metrika_task.md +++ b/docs/ru/introduction/ya_metrika_task.md @@ -45,3 +45,5 @@ ClickHouse имеет более десятка инсталляций в дру OLAPServer хорошо подходил для неагрегированных данных, но содержал много ограничений, не позволяющих использовать его для всех отчётов так, как хочется: отсутствие поддержки типов данных (только числа), невозможность инкрементального обновления данных в реальном времени (только перезаписью данных за сутки). OLAPServer не является СУБД, а является специализированной БД. Чтобы снять ограничения OLAPServer-а и решить задачу работы с неагрегированными данными для всех отчётов, разработана СУБД ClickHouse. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/introduction/ya_metrika_task/) diff --git a/docs/ru/operations/access_rights.md b/docs/ru/operations/access_rights.md index 7f09a917c7a..93e496b3dc1 100644 --- a/docs/ru/operations/access_rights.md +++ b/docs/ru/operations/access_rights.md @@ -98,3 +98,5 @@ Пользователь может получить список всех БД и таблиц в них с помощью запросов `SHOW` или системных таблиц, даже если у него нет доступа к отдельным БД. Доступ к БД не связан с настройкой [readonly](settings/query_complexity.md#query_complexity_readonly). Невозможно дать полный доступ к одной БД и `readonly` к другой. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/access_rights/) diff --git a/docs/ru/operations/configuration_files.md b/docs/ru/operations/configuration_files.md index 97670cd9118..c99351f7308 100644 --- a/docs/ru/operations/configuration_files.md +++ b/docs/ru/operations/configuration_files.md @@ -40,3 +40,5 @@ $ cat /etc/clickhouse-server/users.d/alice.xml Для каждого конфигурационного файла, сервер при запуске генерирует также файлы `file-preprocessed.xml`. Эти файлы содержат все выполненные подстановки и переопределения, и предназначены для информационных целей. Если в конфигурационных файлах были использованы ZooKeeper-подстановки, но при старте сервера ZooKeeper недоступен, то сервер загрузит конфигурацию из preprocessed-файла. Сервер следит за изменениями конфигурационных файлов, а также файлов и ZooKeeper-узлов, которые были использованы при выполнении подстановок и переопределений, и перезагружает настройки пользователей и кластеров на лету. То есть, можно изменять кластера, пользователей и их настройки без перезапуска сервера. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/configuration_files/) diff --git a/docs/ru/operations/index.md b/docs/ru/operations/index.md index 75f20597ec3..f16d6b3f8d7 100644 --- a/docs/ru/operations/index.md +++ b/docs/ru/operations/index.md @@ -1 +1,3 @@ # Эксплуатация + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/) diff --git a/docs/ru/operations/quotas.md b/docs/ru/operations/quotas.md index 1a56ff9fe62..93a795973d9 100644 --- a/docs/ru/operations/quotas.md +++ b/docs/ru/operations/quotas.md @@ -103,3 +103,5 @@ При распределённой обработке запроса, накопленные величины хранятся на сервере-инициаторе запроса. То есть, если пользователь пойдёт на другой сервер - там квота будет действовать "с нуля". При перезапуске сервера, квоты сбрасываются. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/quotas/) diff --git a/docs/ru/operations/server_settings/index.md b/docs/ru/operations/server_settings/index.md index 644b1af79c4..c208f9b418b 100644 --- a/docs/ru/operations/server_settings/index.md +++ b/docs/ru/operations/server_settings/index.md @@ -9,3 +9,5 @@ Прочие настройки описаны в разделе "[Настройки](../settings/index.md#settings)". Перед изучением настроек ознакомьтесь с разделом [Конфигурационные файлы](../configuration_files.md#configuration_files), обратите внимание на использование подстановок (атрибуты `incl` и `optional`). + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/server_settings/) diff --git a/docs/ru/operations/server_settings/settings.md b/docs/ru/operations/server_settings/settings.md index 5e71c871615..567551cc394 100644 --- a/docs/ru/operations/server_settings/settings.md +++ b/docs/ru/operations/server_settings/settings.md @@ -719,3 +719,5 @@ ClickHouse использует ZooKeeper для хранения метадан ```xml ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/server_settings/settings/) diff --git a/docs/ru/operations/settings/index.md b/docs/ru/operations/settings/index.md index 3de41c00b3f..1fa2f55bd53 100644 --- a/docs/ru/operations/settings/index.md +++ b/docs/ru/operations/settings/index.md @@ -22,3 +22,5 @@ - При использовании HTTP API передавайте cgi-параметры (`URL?setting_1=value&setting_2=value...`). Настройки, которые можно задать только в конфигурационном файле сервера, в разделе не рассматриваются. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/settings/) diff --git a/docs/ru/operations/settings/query_complexity.md b/docs/ru/operations/settings/query_complexity.md index ccb206a587d..3be6205e5be 100644 --- a/docs/ru/operations/settings/query_complexity.md +++ b/docs/ru/operations/settings/query_complexity.md @@ -196,3 +196,5 @@ ## transfer_overflow_mode Что делать, когда количество данных превысило одно из ограничений: throw или break. По умолчанию: throw. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/settings/query_complexity/) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 9648620be93..91271b2de5f 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -413,3 +413,5 @@ ClickHouse применяет настройку в тех случаях, ко - [insert_quorum](#setting-insert_quorum) - [insert_quorum_timeout](#setting-insert_quorum_timeout) + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/settings/settings/) diff --git a/docs/ru/operations/settings/settings_profiles.md b/docs/ru/operations/settings/settings_profiles.md index 8e30d76107e..600b7c11730 100644 --- a/docs/ru/operations/settings/settings_profiles.md +++ b/docs/ru/operations/settings/settings_profiles.md @@ -9,7 +9,7 @@ Установить профиль `web`. -```sql +``` sql SET profile = 'web' ``` @@ -62,3 +62,5 @@ SET profile = 'web' В примере задано два профиля: `default` и `web`. Профиль `default` имеет специальное значение - он всегда обязан присутствовать и применяется при запуске сервера. То есть, профиль `default` содержит настройки по умолчанию. Профиль `web` - обычный профиль, который может быть установлен с помощью запроса `SET` или с помощью параметра URL при запросе по HTTP. Профили настроек могут наследоваться от друг-друга - это реализуется указанием настройки `profile` перед остальными настройками, перечисленными в профиле. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/settings/settings_profiles/) diff --git a/docs/ru/operations/system_tables.md b/docs/ru/operations/system_tables.md index ee03d00ac34..d92274b3b27 100644 --- a/docs/ru/operations/system_tables.md +++ b/docs/ru/operations/system_tables.md @@ -19,7 +19,7 @@ Содержит информацию о доступных в конфигурационном файле кластерах и серверах, которые в них входят. Столбцы: -```text +``` cluster String - имя кластера shard_num UInt32 - номер шарда в кластере, начиная с 1 shard_weight UInt32 - относительный вес шарда при записи данных @@ -34,7 +34,7 @@ user String - имя пользователя, которого испо Содержит информацию о столбцах всех таблиц. С помощью этой таблицы можно получить информацию аналогично запросу `DESCRIBE TABLE`, но для многих таблиц сразу. -```text +``` database String - имя базы данных, в которой находится таблица table String - имя таблицы name String - имя столбца @@ -159,7 +159,7 @@ default_expression String - выражение для значения по ум Эта системная таблица используется для реализации запроса `SHOW PROCESSLIST`. Столбцы: -```text +``` user String - имя пользователя, который задал запрос. При распределённой обработке запроса, относится к пользователю, с помощью которого сервер-инициатор запроса отправил запрос на данный сервер, а не к имени пользователя, который задал распределённый запрос на сервер-инициатор запроса. address String - IP-адрес, с которого задан запрос. При распределённой обработке запроса, аналогично. @@ -185,14 +185,14 @@ query_id String - идентификатор запроса, если Пример: -```sql +``` sql SELECT * FROM system.replicas WHERE table = 'visits' FORMAT Vertical ``` -```text +``` Row 1: ────── database: merge @@ -218,7 +218,7 @@ active_replicas: 2 Столбцы: -```text +``` database: имя БД table: имя таблицы engine: имя движка таблицы @@ -271,7 +271,7 @@ active_replicas: число реплик этой таблицы, имеющ Например, так можно проверить, что всё хорошо: -```sql +``` sql SELECT database, table, @@ -309,7 +309,7 @@ WHERE Столбцы: -```text +``` name String - имя настройки value String - значение настройки changed UInt8 - была ли настройка явно задана в конфиге или изменена явным образом @@ -317,13 +317,13 @@ changed UInt8 - была ли настройка явно задана в кон Пример: -```sql +``` sql SELECT * FROM system.settings WHERE changed ``` -```text +``` ┌─name───────────────────┬─value───────┬─changed─┐ │ max_threads │ 8 │ 1 │ │ use_uncompressed_cache │ 0 │ 1 │ @@ -368,14 +368,14 @@ WHERE changed Пример: -```sql +``` sql SELECT * FROM system.zookeeper WHERE path = '/clickhouse/tables/01-08/visits/replicas' FORMAT Vertical ``` -```text +``` Row 1: ────── name: example01-08-1.yandex.ru @@ -410,3 +410,5 @@ numChildren: 7 pzxid: 987021252247 path: /clickhouse/tables/01-08/visits/replicas ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/system_tables/) diff --git a/docs/ru/operations/table_engines/aggregatingmergetree.md b/docs/ru/operations/table_engines/aggregatingmergetree.md index 42bff397e5e..92e3ff7eea0 100644 --- a/docs/ru/operations/table_engines/aggregatingmergetree.md +++ b/docs/ru/operations/table_engines/aggregatingmergetree.md @@ -60,7 +60,7 @@ CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] Создаём материализованное представление типа `AggregatingMergeTree`, следящее за таблицей `test.visits`: -```sql +``` sql CREATE MATERIALIZED VIEW test.basic ENGINE = AggregatingMergeTree() PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate) AS SELECT @@ -74,7 +74,7 @@ GROUP BY CounterID, StartDate; Вставляем данные в таблицу `test.visits`: -```sql +``` sql INSERT INTO test.visits ... ``` @@ -82,7 +82,7 @@ INSERT INTO test.visits ... Чтобы получить агрегированные данные, выполним запрос вида `SELECT ... GROUP BY ...` из представления `test.basic`: -```sql +``` sql SELECT StartDate, sumMerge(Visits) AS Visits, @@ -91,3 +91,5 @@ FROM test.basic GROUP BY StartDate ORDER BY StartDate; ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/aggregatingmergetree/) diff --git a/docs/ru/operations/table_engines/buffer.md b/docs/ru/operations/table_engines/buffer.md index 7f947ea91b6..24e456da486 100644 --- a/docs/ru/operations/table_engines/buffer.md +++ b/docs/ru/operations/table_engines/buffer.md @@ -2,7 +2,7 @@ Буферизует записываемые данные в оперативке, периодически сбрасывая их в другую таблицу. При чтении, производится чтение данных одновременно из буфера и из другой таблицы. -```text +``` Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes) ``` @@ -22,7 +22,7 @@ min_bytes, max_bytes - условие на количество байт в бу Пример: -```sql +``` sql CREATE TABLE merge.hits_buffer AS merge.hits ENGINE = Buffer(merge, hits, 16, 10, 100, 10000, 1000000, 10000000, 100000000) ``` @@ -57,3 +57,5 @@ CREATE TABLE merge.hits_buffer AS merge.hits ENGINE = Buffer(merge, hits, 16, 10 Таблицы типа Buffer используются в тех случаях, когда от большого количества серверов поступает слишком много INSERT-ов в единицу времени, и нет возможности заранее самостоятельно буферизовать данные перед вставкой, в результате чего, INSERT-ы не успевают выполняться. Заметим, что даже для таблиц типа Buffer не имеет смысла вставлять данные по одной строке, так как таким образом будет достигнута скорость всего лишь в несколько тысяч строк в секунду, тогда как при вставке более крупными блоками, достижимо более миллиона строк в секунду (смотрите раздел "Производительность"). + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/buffer/) diff --git a/docs/ru/operations/table_engines/collapsingmergetree.md b/docs/ru/operations/table_engines/collapsingmergetree.md index 83dfaf99d72..d283a8dbe04 100644 --- a/docs/ru/operations/table_engines/collapsingmergetree.md +++ b/docs/ru/operations/table_engines/collapsingmergetree.md @@ -14,7 +14,7 @@ CollapsingMergeTree принимает дополнительный параметр - имя столбца типа Int8, содержащего "знак" строки. Пример: -```sql +``` sql CollapsingMergeTree(EventDate, (CounterID, EventDate, intHash32(UniqID), VisitID), 8192, Sign) ``` @@ -35,3 +35,5 @@ CollapsingMergeTree(EventDate, (CounterID, EventDate, intHash32(UniqID), VisitID 1. Написать запрос с GROUP BY и агрегатными функциями, учитывающими знак. Например, чтобы посчитать количество, надо вместо count() написать sum(Sign); чтобы посчитать сумму чего-либо, надо вместо sum(x) написать sum(Sign \* x) и т. п., а также добавить HAVING sum(Sign) `>` 0. Не все величины можно посчитать подобным образом. Например, агрегатные функции min, max не могут быть переписаны. 2. Если необходимо вынимать данные без агрегации (например, проверить наличие строк, самые новые значения которых удовлетворяют некоторым условиям), можно использовать модификатор FINAL для секции FROM. Это вариант существенно менее эффективен. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/collapsingmergetree/) diff --git a/docs/ru/operations/table_engines/custom_partitioning_key.md b/docs/ru/operations/table_engines/custom_partitioning_key.md index dc7abf81811..8d228f5c498 100644 --- a/docs/ru/operations/table_engines/custom_partitioning_key.md +++ b/docs/ru/operations/table_engines/custom_partitioning_key.md @@ -11,7 +11,7 @@ ENGINE [=] Name(...) [PARTITION BY expr] [ORDER BY expr] [SAMPLE BY expr] [SETTI ``` Для MergeTree таблиц выражение партиционирования указывается после `PARTITION BY`, первичный ключ после `ORDER BY`, ключ сэмплирования после `SAMPLE BY`, а в `SETTINGS` можно указать `index_granularity` (не обязательно, значение по умолчанию 8192), а также другие настройки из [MergeTreeSettings.h](https://github.com/yandex/ClickHouse/blob/master/dbms/src/Storages/MergeTree/MergeTreeSettings.h). Остальные параметры движка по-прежнему указываются в скобках после его названия. Пример: -```sql +``` sql ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/name', 'replica1', Sign) PARTITION BY (toMonday(StartDate), EventType) ORDER BY (CounterID, StartDate, intHash32(UserID)) @@ -25,7 +25,7 @@ ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/name', 'replica1', Si После создания такой таблицы слияние кусков будет работать только для кусков с одинаковым значением выражения партиционирования. Замечание: это означает, что нежелательно делать слишком гранулированное партиционирование (более порядка тысячи партиций), иначе производительность SELECT будет неудовлетворительной. Чтобы указать партицию в командах ALTER PARTITION, нужно указать значение выражения партиционирования (или кортежа). Поддерживаются константы и константные выражения. Пример: -```sql +``` sql ALTER TABLE table DROP PARTITION (toMonday(today()), 1) ``` удалит партицию за текущую неделю с типом события 1. То же самое для запроса OPTIMIZE. Чтобы указать единственную партицию непартиционированной таблицы, укажите `PARTITION tuple()`. @@ -41,3 +41,5 @@ ALTER TABLE table DROP PARTITION (toMonday(today()), 1) ID партиции - это её строковый идентификатор (по возможности человекочитаемый), используемый для имён кусков на файловой системе и в ZooKeeper. Его можно указывать в запросах ALTER вместо значения ключа партиционирования. Пример: ключ партиционирования `toYYYYMM(EventDate)`, в ALTER можно указывать либо `PARTITION 201710`, либо `PARTITION ID '201710'`. Больше примеров в тестах [`00502_custom_partitioning_local`](https://github.com/yandex/ClickHouse/blob/master/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.sql) и [`00502_custom_partitioning_replicated_zookeeper`](https://github.com/yandex/ClickHouse/blob/master/dbms/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql). + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/custom_partitioning_key/) diff --git a/docs/ru/operations/table_engines/dictionary.md b/docs/ru/operations/table_engines/dictionary.md index 7d6aedf394f..bc5e8e409ea 100644 --- a/docs/ru/operations/table_engines/dictionary.md +++ b/docs/ru/operations/table_engines/dictionary.md @@ -2,7 +2,7 @@ # Dictionary -Движок `Dictionary` отображает данные [словаря](../../query_language/dicts/external_dicts.md) как таблицу ClickHouse. +Движок `Dictionary` отображает данные [словаря](../../query_language/dicts/external_dicts.md#dicts-external_dicts) как таблицу ClickHouse. Рассмотрим для примера словарь `products` со следующей конфигурацией: @@ -39,7 +39,7 @@ Запрос данных словаря: -```sql +``` sql select name, type, key, attribute.names, attribute.types, bytes_allocated, element_count,source from system.dictionaries where name = 'products'; SELECT @@ -73,7 +73,7 @@ CREATE TABLE %table_name% (%fields%) engine = Dictionary(%dictionary_name%)` Пример использования: -```sql +``` sql create table products (product_id UInt64, title String) Engine = Dictionary(products); CREATE TABLE products @@ -91,7 +91,7 @@ Ok. Проверим что у нас в таблице? -```sql +``` sql select * from products limit 1; SELECT * @@ -106,3 +106,5 @@ LIMIT 1 1 rows in set. Elapsed: 0.006 sec. ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/dictionary/) diff --git a/docs/ru/operations/table_engines/distributed.md b/docs/ru/operations/table_engines/distributed.md index a797677690e..a483e6de8b5 100644 --- a/docs/ru/operations/table_engines/distributed.md +++ b/docs/ru/operations/table_engines/distributed.md @@ -7,7 +7,7 @@ Движок Distributed принимает параметры: имя кластера в конфигурационном файле сервера, имя удалённой базы данных, имя удалённой таблицы, а также (не обязательно) ключ шардирования. Пример: -```text +``` Distributed(logs, default, hits[, sharding_key]) ``` @@ -120,3 +120,5 @@ logs - имя кластера в конфигурационном файле с Если после INSERT-а в Distributed таблицу, сервер перестал существовать или был грубо перезапущен (например, в следствие аппаратного сбоя), то записанные данные могут быть потеряны. Если в директории таблицы обнаружен повреждённый кусок данных, то он переносится в поддиректорию broken и больше не используется. При выставлении опции max_parallel_replicas выполнение запроса распараллеливается по всем репликам внутри одного шарда. Подробнее смотрите раздел "Настройки, max_parallel_replicas". + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/distributed/) diff --git a/docs/ru/operations/table_engines/external_data.md b/docs/ru/operations/table_engines/external_data.md index fc61c77677c..430f90a82ae 100644 --- a/docs/ru/operations/table_engines/external_data.md +++ b/docs/ru/operations/table_engines/external_data.md @@ -59,3 +59,5 @@ curl -F 'passwd=@passwd.tsv;' 'http://localhost:8123/?query=SELECT+shell,+count( ``` При распределённой обработке запроса, временные таблицы передаются на все удалённые серверы. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/external_data/) diff --git a/docs/ru/operations/table_engines/file.md b/docs/ru/operations/table_engines/file.md index f6573676822..2cf9f3ff788 100644 --- a/docs/ru/operations/table_engines/file.md +++ b/docs/ru/operations/table_engines/file.md @@ -31,7 +31,7 @@ File(Format) **1.** Создадим на сервере таблицу `file_engine_table`: -```sql +``` sql CREATE TABLE file_engine_table (name String, value UInt32) ENGINE=File(TabSeparated) ``` @@ -47,11 +47,11 @@ two 2 **3.** Запросим данные: -```sql +``` sql SELECT * FROM file_engine_table ``` -```text +``` ┌─name─┬─value─┐ │ one │ 1 │ │ two │ 2 │ @@ -75,3 +75,5 @@ $ echo -e "1,2\n3,4" | clickhouse-local -q "CREATE TABLE table (a Int64, b Int64 - использование операций `ALTER` и `SELECT...SAMPLE`; - индексы; - репликация. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/file/) diff --git a/docs/ru/operations/table_engines/graphitemergetree.md b/docs/ru/operations/table_engines/graphitemergetree.md index 3617fe40829..fa789857751 100644 --- a/docs/ru/operations/table_engines/graphitemergetree.md +++ b/docs/ru/operations/table_engines/graphitemergetree.md @@ -27,7 +27,7 @@ Graphite хранит в ClickHouse полные данные, а получат Шаблон правил rollup: -```text +``` pattern regexp function @@ -83,3 +83,5 @@ default ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/graphitemergetree/) diff --git a/docs/ru/operations/table_engines/index.md b/docs/ru/operations/table_engines/index.md index 90b14f70094..a5e42c21edb 100644 --- a/docs/ru/operations/table_engines/index.md +++ b/docs/ru/operations/table_engines/index.md @@ -14,3 +14,5 @@ При чтении, движок обязан лишь выдать запрошенные столбцы, но в некоторых случаях движок может частично обрабатывать данные при ответе на запрос. Для большинства серьёзных задач, следует использовать движки семейства `MergeTree`. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/) diff --git a/docs/ru/operations/table_engines/join.md b/docs/ru/operations/table_engines/join.md index 2acabefc491..d9ce49665e9 100644 --- a/docs/ru/operations/table_engines/join.md +++ b/docs/ru/operations/table_engines/join.md @@ -2,7 +2,7 @@ Представляет собой подготовленную структуру данных для JOIN-а, постоянно находящуюся в оперативке. -```text +``` Join(ANY|ALL, LEFT|INNER, k1[, k2, ...]) ``` @@ -14,3 +14,5 @@ Join(ANY|ALL, LEFT|INNER, k1[, k2, ...]) В таблицу можно вставлять данные INSERT-ом, аналогично движку Set. В случае ANY, данные для дублирующихся ключей будут проигнорированы; в случае ALL - будут учитываться. Из таблицы нельзя, непосредственно, делать SELECT. Единственная возможность чтения - использование в качестве "правой" таблицы для JOIN. Хранение данных на диске аналогично движку Set. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/join/) diff --git a/docs/ru/operations/table_engines/kafka.md b/docs/ru/operations/table_engines/kafka.md index 282140d4350..8dc7b55e373 100644 --- a/docs/ru/operations/table_engines/kafka.md +++ b/docs/ru/operations/table_engines/kafka.md @@ -43,7 +43,7 @@ Kafka SETTINGS Примеры: -```sql +``` sql CREATE TABLE queue ( timestamp UInt64, level String, @@ -85,7 +85,7 @@ Kafka SETTINGS Пример: -```sql +``` sql CREATE TABLE queue ( timestamp UInt64, level String, @@ -136,3 +136,5 @@ Kafka SETTINGS ``` В документе [librdkafka configuration reference](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md) можно увидеть список возможных опций конфигурации. Используйте подчёркивания (`_`) вместо точек в конфигурации ClickHouse, например, `check.crcs=true` будет соответствовать `true`. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/kafka/) diff --git a/docs/ru/operations/table_engines/log.md b/docs/ru/operations/table_engines/log.md index cfb050f1640..58aab57f5f7 100644 --- a/docs/ru/operations/table_engines/log.md +++ b/docs/ru/operations/table_engines/log.md @@ -3,3 +3,5 @@ Отличается от TinyLog тем, что вместе с файлами столбцов лежит небольшой файл "засечек". Засечки пишутся на каждый блок данных и содержат смещение - с какого места нужно читать файл, чтобы пропустить заданное количество строк. Это позволяет читать данные из таблицы в несколько потоков. При конкуррентном доступе к данным, чтения могут выполняться одновременно, а записи блокируют чтения и друг друга. Движок Log не поддерживает индексы. Также, если при записи в таблицу произошёл сбой, то таблица станет битой, и чтения из неё будут возвращать ошибку. Движок Log подходит для временных данных, write-once таблиц, а также для тестовых и демонстрационных целей. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/log/) diff --git a/docs/ru/operations/table_engines/materializedview.md b/docs/ru/operations/table_engines/materializedview.md index 8c44eb40c01..843f0678a9a 100644 --- a/docs/ru/operations/table_engines/materializedview.md +++ b/docs/ru/operations/table_engines/materializedview.md @@ -1,3 +1,5 @@ # MaterializedView Используется для реализации материализованных представлений (подробнее см. запрос [CREATE TABLE](../../query_language/create.md#query_language-queries-create_table)). Для хранения данных, использует другой движок, который был указан при создании представления. При чтении из таблицы, просто использует этот движок. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/materializedview/) diff --git a/docs/ru/operations/table_engines/memory.md b/docs/ru/operations/table_engines/memory.md index 7552a270988..70cf0c8db20 100644 --- a/docs/ru/operations/table_engines/memory.md +++ b/docs/ru/operations/table_engines/memory.md @@ -8,3 +8,5 @@ Обычно, использование этого движка таблиц является неоправданным. Тем не менее, он может использоваться для тестов, а также в задачах, где важно достичь максимальной скорости на не очень большом количестве строк (примерно до 100 000 000). Движок Memory используется системой для временных таблиц - внешних данных запроса (смотрите раздел "Внешние данные для обработки запроса"), для реализации `GLOBAL IN` (смотрите раздел "Операторы IN"). + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/memory/) diff --git a/docs/ru/operations/table_engines/merge.md b/docs/ru/operations/table_engines/merge.md index 7aa5ebd3348..3b2174d52fa 100644 --- a/docs/ru/operations/table_engines/merge.md +++ b/docs/ru/operations/table_engines/merge.md @@ -39,3 +39,5 @@ Merge(hits, '^WatchLog') Таблица типа `Merge` содержит виртуальный столбец `_table` типа `String`. (Если в таблице уже есть столбец `_table`, то виртуальный столбец называется `_table1`; если уже есть `_table1`, то `_table2` и т. п.) Он содержит имя таблицы, из которой были прочитаны данные. Если секция `WHERE/PREWHERE` содержит (в качестве одного из элементов конъюнкции или в качестве всего выражения) условия на столбец `_table`, не зависящие от других столбцов таблицы, то эти условия используются как индекс: условия выполняются над множеством имён таблиц, из которых нужно читать данные, и чтение будет производиться только из тех таблиц, для которых условия сработали. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/merge/) diff --git a/docs/ru/operations/table_engines/mergetree.md b/docs/ru/operations/table_engines/mergetree.md index ce6b793ade5..68cd52d145d 100644 --- a/docs/ru/operations/table_engines/mergetree.md +++ b/docs/ru/operations/table_engines/mergetree.md @@ -178,7 +178,7 @@ ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDa В этом случае в запросах: -```sql +``` sql SELECT count() FROM table WHERE EventDate = toDate(now()) AND CounterID = 34 SELECT count() FROM table WHERE EventDate = toDate(now()) AND (CounterID = 34 OR CounterID = 42) SELECT count() FROM table WHERE ((EventDate >= toDate('2014-01-01') AND EventDate <= toDate('2014-01-31')) OR EventDate = toDate('2014-05-01')) AND CounterID IN (101500, 731962, 160656) AND (CounterID = 101500 OR EventDate != toDate('2014-05-01')) @@ -190,7 +190,7 @@ ClickHouse будет использовать индекс по первичн В примере ниже индекс не может использоваться. -```sql +``` sql SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' ``` @@ -204,3 +204,5 @@ SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' Для конкурентного доступа к таблице используется мультиверсионность. То есть, при одновременном чтении и обновлении таблицы, данные будут читаться из набора кусочков, актуального на момент запроса. Длинных блокировок нет. Вставки никак не мешают чтениям. Чтения из таблицы автоматически распараллеливаются. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/mergetree/) diff --git a/docs/ru/operations/table_engines/mysql.md b/docs/ru/operations/table_engines/mysql.md index e08edc4e180..d4444dfe269 100644 --- a/docs/ru/operations/table_engines/mysql.md +++ b/docs/ru/operations/table_engines/mysql.md @@ -25,3 +25,5 @@ MySQL('host:port', 'database', 'table', 'user', 'password'[, replace_query, 'on_ Остальные условия и ограничение выборки `LIMIT` будут выполнены в ClickHouse только после выполнения запроса к MySQL. Движок `MySQL` не поддерживает тип данных [Nullable](../../data_types/nullable.md#data_type-nullable), поэтому при чтении данных из таблиц MySQL `NULL` преобразуются в значения по умолчанию для заданного типа столбца, обычно это 0 или пустая строка. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/mysql/) diff --git a/docs/ru/operations/table_engines/null.md b/docs/ru/operations/table_engines/null.md index 252517b1acd..a67b9bb16a8 100644 --- a/docs/ru/operations/table_engines/null.md +++ b/docs/ru/operations/table_engines/null.md @@ -3,3 +3,5 @@ При записи в таблицу типа Null, данные игнорируются. При чтении из таблицы типа Null, возвращается пустота. Тем не менее, есть возможность создать материализованное представление над таблицей типа Null. Тогда данные, записываемые в таблицу, будут попадать в представление. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/null/) diff --git a/docs/ru/operations/table_engines/replacingmergetree.md b/docs/ru/operations/table_engines/replacingmergetree.md index b2397200abd..d22e931aa63 100644 --- a/docs/ru/operations/table_engines/replacingmergetree.md +++ b/docs/ru/operations/table_engines/replacingmergetree.md @@ -53,3 +53,5 @@ CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] Все параметры, кроме `ver` имеют то же значение, что в и `MergeTree`. - `ver` — столбец с версией. Необязательный параметр. Описание смотрите выше по тексту. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/replacingmergetree/) diff --git a/docs/ru/operations/table_engines/replication.md b/docs/ru/operations/table_engines/replication.md index 1dec0f4f42c..ec40645a8f6 100644 --- a/docs/ru/operations/table_engines/replication.md +++ b/docs/ru/operations/table_engines/replication.md @@ -78,7 +78,7 @@ Пример: -```text +``` ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/hits', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192) ``` @@ -180,3 +180,5 @@ sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data ## Восстановление в случае потери или повреждения метаданных на ZooKeeper кластере Если данные в ZooKeeper оказались утеряны или повреждены, то вы можете сохранить данные, переместив их в нереплицируемую таблицу, как описано в пункте выше. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/replication/) diff --git a/docs/ru/operations/table_engines/set.md b/docs/ru/operations/table_engines/set.md index 0697b32b492..b4371e1a9c2 100644 --- a/docs/ru/operations/table_engines/set.md +++ b/docs/ru/operations/table_engines/set.md @@ -8,3 +8,5 @@ Данные постоянно находятся в оперативке. При INSERT-е, в директорию таблицы на диске, также пишутся блоки вставленных данных. При запуске сервера, эти данные считываются в оперативку. То есть, после перезапуска, данные остаются на месте. При грубом перезапуске сервера, блок данных на диске может быть потерян или повреждён. В последнем случае, может потребоваться вручную удалить файл с повреждёнными данными. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/set/) diff --git a/docs/ru/operations/table_engines/summingmergetree.md b/docs/ru/operations/table_engines/summingmergetree.md index b169ad9833c..3bdf22e62ae 100644 --- a/docs/ru/operations/table_engines/summingmergetree.md +++ b/docs/ru/operations/table_engines/summingmergetree.md @@ -120,7 +120,7 @@ ClickHouse может слить куски данных таким образо Примеры: -```text +``` [(1, 100)] + [(2, 150)] -> [(1, 100), (2, 150)] [(1, 100)] + [(1, 150)] -> [(1, 250)] [(1, 100)] + [(1, 150), (2, 150)] -> [(1, 250), (2, 150)] @@ -130,3 +130,5 @@ ClickHouse может слить куски данных таким образо При запросе данных используйте функцию [sumMap(key, value)](../../query_language/agg_functions/reference.md#agg_function-summap) для агрегации `Map`. Для вложенной структуры данных не нужно указывать её столбцы в кортеже столбцов для суммирования. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/summingmergetree/) diff --git a/docs/ru/operations/table_engines/tinylog.md b/docs/ru/operations/table_engines/tinylog.md index 813eaa56890..3e09668504e 100644 --- a/docs/ru/operations/table_engines/tinylog.md +++ b/docs/ru/operations/table_engines/tinylog.md @@ -16,3 +16,5 @@ **Индексы не поддерживаются.** В Яндекс.Метрике таблицы типа TinyLog используются для промежуточных данных, обрабатываемых маленькими пачками. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/tinylog/) diff --git a/docs/ru/operations/table_engines/url.md b/docs/ru/operations/table_engines/url.md index b3daae06169..a4a0d511e8a 100644 --- a/docs/ru/operations/table_engines/url.md +++ b/docs/ru/operations/table_engines/url.md @@ -23,7 +23,7 @@ **1.** Создадим на сервере таблицу `url_engine_table`: -```sql +``` sql CREATE TABLE url_engine_table (word String, value UInt64) ENGINE=URL('http://127.0.0.1:12345/', CSV) ``` @@ -53,11 +53,11 @@ python3 server.py **3.** Запросим данные: -```sql +``` sql SELECT * FROM url_engine_table ``` -```text +``` ┌─word──┬─value─┐ │ Hello │ 1 │ │ World │ 2 │ @@ -72,3 +72,5 @@ SELECT * FROM url_engine_table - индексы; - репликация. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/url/) diff --git a/docs/ru/operations/table_engines/view.md b/docs/ru/operations/table_engines/view.md index 128986eb733..874569e3af4 100644 --- a/docs/ru/operations/table_engines/view.md +++ b/docs/ru/operations/table_engines/view.md @@ -1,3 +1,5 @@ # View Используется для реализации представлений (подробнее см. запрос `CREATE VIEW`). Не хранит данные, а хранит только указанный запрос `SELECT`. При чтении из таблицы, выполняет его (с удалением из запроса всех ненужных столбцов). + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/view/) diff --git a/docs/ru/operations/tips.md b/docs/ru/operations/tips.md index 9927d78dcf3..af5ab00c31a 100644 --- a/docs/ru/operations/tips.md +++ b/docs/ru/operations/tips.md @@ -178,7 +178,7 @@ dynamicConfigFile=/etc/zookeeper-{{ cluster['name'] }}/conf/zoo.cfg.dynamic Версия Java: -```text +``` Java(TM) SE Runtime Environment (build 1.8.0_25-b17) Java HotSpot(TM) 64-Bit Server VM (build 25.25-b02, mixed mode) ``` @@ -226,7 +226,7 @@ JAVA_OPTS="-Xms{{ cluster.get('xms','128M') }} \ Salt init: -```text +``` description "zookeeper-{{ cluster['name'] }} centralized coordination service" start on runlevel [2345] @@ -254,3 +254,5 @@ script -Dzookeeper.root.logger=${ZOO_LOG4J_PROP} $ZOOMAIN $ZOOCFG end script ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/tips/) diff --git a/docs/ru/operations/utils/clickhouse-copier.md b/docs/ru/operations/utils/clickhouse-copier.md index 849fa532d41..d114c826dc6 100644 --- a/docs/ru/operations/utils/clickhouse-copier.md +++ b/docs/ru/operations/utils/clickhouse-copier.md @@ -157,3 +157,5 @@ clickhouse-copier copier --daemon --config zookeeper.xml --task-path /task/path ``` `clickhouse-copier` отслеживает изменения `/task/path/description` и применяет их "на лету". Если вы поменяете, например, значение `max_workers`, то количество процессов, выполняющих задания, также изменится. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/utils/clickhouse-copier/) diff --git a/docs/ru/operations/utils/clickhouse-local.md b/docs/ru/operations/utils/clickhouse-local.md index ddaa64e0a21..4edbb3abb2b 100644 --- a/docs/ru/operations/utils/clickhouse-local.md +++ b/docs/ru/operations/utils/clickhouse-local.md @@ -66,3 +66,5 @@ Read 186 rows, 4.15 KiB in 0.035 sec., 5302 rows/sec., 118.34 KiB/sec. ├──────────┼──────────┤ ... ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/utils/clickhouse-local/) diff --git a/docs/ru/operations/utils/index.md b/docs/ru/operations/utils/index.md index 75bd574a72e..aa07954d8c6 100644 --- a/docs/ru/operations/utils/index.md +++ b/docs/ru/operations/utils/index.md @@ -2,3 +2,5 @@ * [clickhouse-local](clickhouse-local.md#utils-clickhouse-local) - позволяет выполнять SQL-запросы над данными без установки сервера ClickHouse подобно тому, как это делает `awk`. * [clickhouse-copier](clickhouse-copier.md#utils-clickhouse-copier) - копирует (и перешардирует) данные с одного кластера на другой. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/utils/) diff --git a/docs/ru/query_language/agg_functions/combinators.md b/docs/ru/query_language/agg_functions/combinators.md index 75276b0132c..b823d6b877c 100644 --- a/docs/ru/query_language/agg_functions/combinators.md +++ b/docs/ru/query_language/agg_functions/combinators.md @@ -40,3 +40,5 @@ ## -ForEach Преобразует агрегатную функцию для таблиц в агрегатную функцию для массивов, которая применяет агрегирование для соответствующих элементов массивов и возвращает массив результатов. Например, `sumForEach` для массивов `[1, 2]`, `[3, 4, 5]` и `[6, 7]` даст результат `[10, 13, 5]`, сложив соответственные элементы массивов. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/agg_functions/combinators/) diff --git a/docs/ru/query_language/agg_functions/index.md b/docs/ru/query_language/agg_functions/index.md index 261c3c3700c..e89934fde39 100644 --- a/docs/ru/query_language/agg_functions/index.md +++ b/docs/ru/query_language/agg_functions/index.md @@ -61,3 +61,5 @@ FROM t_null_big ``` `groupArray` не включает `NULL` в результирующий массив. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/agg_functions/) diff --git a/docs/ru/query_language/agg_functions/parametric_functions.md b/docs/ru/query_language/agg_functions/parametric_functions.md index b86b75baf6c..7f7432fb671 100644 --- a/docs/ru/query_language/agg_functions/parametric_functions.md +++ b/docs/ru/query_language/agg_functions/parametric_functions.md @@ -23,7 +23,7 @@ Это вырожденный пример. Его можно записать с помощью других агрегатных функций: -```text +``` minIf(EventTime, URL LIKE '%company%') < maxIf(EventTime, URL LIKE '%cart%'). ``` @@ -123,7 +123,9 @@ ORDER BY level Пример применения: -```text +``` Задача: показывать в отчёте только поисковые фразы, по которым было хотя бы 5 уникальных посетителей. Решение: пишем в запросе GROUP BY SearchPhrase HAVING uniqUpTo(4)(UserID) >= 5 ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/agg_functions/parametric_functions/) diff --git a/docs/ru/query_language/agg_functions/reference.md b/docs/ru/query_language/agg_functions/reference.md index 436633427c1..2cd156591d4 100644 --- a/docs/ru/query_language/agg_functions/reference.md +++ b/docs/ru/query_language/agg_functions/reference.md @@ -39,7 +39,7 @@ anyHeavy(column) Возьмем набор данных [OnTime](../../getting_started/example_datasets/ontime.md#example_datasets-ontime) и выберем произвольное часто встречающееся значение в столбце `AirlineID`. -```sql +``` sql SELECT anyHeavy(AirlineID) AS res FROM ontime ``` @@ -112,7 +112,7 @@ SELECT argMin(user, salary) FROM salary Пример: -```sql +``` sql CREATE TABLE sum_map( date Date, timeslot DateTime, @@ -133,7 +133,7 @@ FROM sum_map GROUP BY timeslot ``` -```text +``` ┌────────────timeslot─┬─sumMap(statusMap.status, statusMap.requests)─┐ │ 2000-01-01 00:00:00 │ ([1,2,3,4,5],[10,10,20,10,10]) │ │ 2000-01-01 00:01:00 │ ([4,5,6,7,8],[10,10,20,10,10]) │ @@ -358,7 +358,7 @@ topK(N)(column) Возьмем набор данных [OnTime](../../getting_started/example_datasets/ontime.md#example_datasets-ontime) и выберем 3 наиболее часто встречающихся значения в столбце `AirlineID`. -```sql +``` sql SELECT topK(3)(AirlineID) AS res FROM ontime ``` @@ -383,3 +383,5 @@ FROM ontime ## corr(x, y) Вычисляет коэффициент корреляции Пирсона: `Σ((x - x̅)(y - y̅)) / sqrt(Σ((x - x̅)^2) * Σ((y - y̅)^2))`. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/agg_functions/reference/) diff --git a/docs/ru/query_language/alter.md b/docs/ru/query_language/alter.md index 4a39af65b95..268e7baa962 100644 --- a/docs/ru/query_language/alter.md +++ b/docs/ru/query_language/alter.md @@ -7,7 +7,7 @@ Изменение структуры таблицы. -```sql +``` sql ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|MODIFY COLUMN ... ``` @@ -16,7 +16,7 @@ ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|MODIFY COLUMN ... Существуют следующие действия: -```sql +``` sql ADD COLUMN name [type] [default_expr] [AFTER name_after] ``` @@ -26,14 +26,14 @@ ADD COLUMN name [type] [default_expr] [AFTER name_after] Такая схема позволяет добиться мгновенной работы запроса ALTER и отсутствия необходимости увеличивать объём старых данных. -```sql +``` sql DROP COLUMN name ``` Удаляет столбец с именем name. Удаляет данные из файловой системы. Так как это представляет собой удаление целых файлов, запрос выполняется почти мгновенно. -```sql +``` sql MODIFY COLUMN name [type] [default_expr] ``` @@ -84,7 +84,7 @@ MODIFY COLUMN name [type] [default_expr] Чтобы посмотреть набор кусков и партиций таблицы, можно воспользоваться системной таблицей `system.parts`: -```sql +``` sql SELECT * FROM system.parts WHERE active ``` @@ -122,7 +122,7 @@ drwxrwxrwx 2 clickhouse clickhouse 4096 May 5 02:55 detached Директория `detached` содержит куски, не используемые сервером - отцепленные от таблицы с помощью запроса `ALTER ... DETACH`. Также в эту директорию переносятся куски, признанные повреждёнными, вместо их удаления. Вы можете в любое время добавлять, удалять, модифицировать данные в директории detached - сервер не будет об этом знать, пока вы не сделаете запрос `ALTER TABLE ... ATTACH`. -```sql +``` sql ALTER TABLE [db.]table DETACH PARTITION 'name' ``` @@ -133,13 +133,13 @@ ALTER TABLE [db.]table DETACH PARTITION 'name' Запрос реплицируется - данные будут перенесены в директорию detached и забыты на всех репликах. Запрос может быть отправлен только на реплику-лидер. Вы можете узнать, является ли реплика лидером, сделав SELECT в системную таблицу system.replicas. Или, проще, вы можете выполнить запрос на всех репликах, и на всех кроме одной, он кинет исключение. -```sql +``` sql ALTER TABLE [db.]table DROP PARTITION 'name' ``` Аналогично операции `DETACH`. Удалить данные из таблицы. Куски с данными будут помечены как неактивные и будут полностью удалены примерно через 10 минут. Запрос реплицируется - данные будут удалены на всех репликах. -```sql +``` sql ALTER TABLE [db.]table ATTACH PARTITION|PART 'name' ``` @@ -151,7 +151,7 @@ ALTER TABLE [db.]table ATTACH PARTITION|PART 'name' То есть, вы можете разместить данные в директории detached на одной реплике и, с помощью запроса ALTER ... ATTACH добавить их в таблицу на всех репликах. -```sql +``` sql ALTER TABLE [db.]table FREEZE PARTITION 'name' ``` @@ -195,7 +195,7 @@ ALTER TABLE [db.]table FREEZE PARTITION 'name' Бэкапы защищают от человеческих ошибок (случайно удалили данные, удалили не те данные или не на том кластере, испортили данные). Для баз данных большого объёма, бывает затруднительно копировать бэкапы на удалённые серверы. В этих случаях, для защиты от человеческой ошибки, можно держать бэкап на том же сервере (он будет лежать в `/var/lib/clickhouse/shadow/`). -```sql +``` sql ALTER TABLE [db.]table FETCH PARTITION 'name' FROM 'path-in-zookeeper' ``` @@ -231,13 +231,13 @@ ALTER TABLE [db.]table FETCH PARTITION 'name' FROM 'path-in-zookeeper' На данный момент доступны команды: -```sql +``` sql ALTER TABLE [db.]table DELETE WHERE filter_expr ``` Выражение `filter_expr` должно иметь тип UInt8. Запрос удаляет строки таблицы, для которых это выражение принимает ненулевое значение. -```sql +``` sql ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr ``` @@ -270,3 +270,5 @@ ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr **parts_to_do** - Количество кусков таблицы, которые ещё предстоит изменить. **is_done** - Завершена ли мутация. Замечание: даже если `parts_to_do = 0`, для реплицированной таблицы возможна ситуация, когда мутация ещё не завершена из-за долго выполняющейся вставки, которая добавляет данные, которые нужно будет мутировать. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/alter/) diff --git a/docs/ru/query_language/create.md b/docs/ru/query_language/create.md index 1b819830a5f..f1d8f8471de 100644 --- a/docs/ru/query_language/create.md +++ b/docs/ru/query_language/create.md @@ -1,7 +1,7 @@ ## CREATE DATABASE Создание базы данных db_name -```sql +``` sql CREATE DATABASE [IF NOT EXISTS] db_name ``` @@ -14,7 +14,7 @@ CREATE DATABASE [IF NOT EXISTS] db_name ## CREATE TABLE Запрос `CREATE TABLE` может иметь несколько форм. -```sql +``` sql CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] ( name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], @@ -29,13 +29,13 @@ CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] Описание столбца, это `name type`, в простейшем случае. Пример: `RegionID UInt32`. Также могут быть указаны выражения для значений по умолчанию - смотрите ниже. -```sql +``` sql CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name AS [db2.]name2 [ENGINE = engine] ``` Создаёт таблицу с такой же структурой, как другая таблица. Можно указать другой движок для таблицы. Если движок не указан, то будет выбран такой же движок, как у таблицы `db2.name2`. -```sql +``` sql CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name ENGINE = engine AS SELECT ... ``` @@ -99,7 +99,7 @@ CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name ENGINE = engine AS SELECT ... Запросы `CREATE`, `DROP`, `ALTER`, `RENAME` поддерживают возможность распределенного выполнения на кластере. Например, следующий запрос создает `Distributed`-таблицу `all_hits` на каждом хосте кластера `cluster`: -```sql +``` sql CREATE TABLE IF NOT EXISTS all_hits ON CLUSTER cluster (p Date, i Int32) ENGINE = Distributed(cluster, default, hits) ``` @@ -109,7 +109,7 @@ CREATE TABLE IF NOT EXISTS all_hits ON CLUSTER cluster (p Date, i Int32) ENGINE ## CREATE VIEW -```sql +``` sql CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... ``` @@ -123,19 +123,19 @@ CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [TO[db.]name] [ENGINE = eng Для примера, пусть вы создали представление: -```sql +``` sql CREATE VIEW view AS SELECT ... ``` и написали запрос: -```sql +``` sql SELECT a, b, c FROM view ``` Этот запрос полностью эквивалентен использованию подзапроса: -```sql +``` sql SELECT a, b, c FROM (SELECT ...) ``` @@ -154,3 +154,6 @@ SELECT a, b, c FROM (SELECT ...) Представления выглядят так же, как обычные таблицы. Например, они перечисляются в результате запроса `SHOW TABLES`. Отсутствует отдельный запрос для удаления представлений. Чтобы удалить представление, следует использовать `DROP TABLE`. + + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/create/) diff --git a/docs/ru/query_language/dicts/external_dicts.md b/docs/ru/query_language/dicts/external_dicts.md index 6fdd4d2d266..d1df2c4990c 100644 --- a/docs/ru/query_language/dicts/external_dicts.md +++ b/docs/ru/query_language/dicts/external_dicts.md @@ -48,3 +48,5 @@ ClickHouse: - [Источники внешних словарей](external_dicts_dict_sources.md#dicts-external_dicts_dict_sources) - [Ключ и поля словаря](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure) - [Функции для работы с внешними словарями](../functions/ext_dict_functions.md#ext_dict_functions) + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/dicts/external_dicts/) diff --git a/docs/ru/query_language/dicts/external_dicts_dict.md b/docs/ru/query_language/dicts/external_dicts_dict.md index 7ba8588dcc1..0c56593fdd1 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict.md +++ b/docs/ru/query_language/dicts/external_dicts_dict.md @@ -31,3 +31,5 @@ - [layout](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout) - Размещение словаря в памяти. - [structure](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure) - Структура словаря. Ключ и атрибуты, которые можно получить по ключу. - [lifetime](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime) - Периодичность обновления словарей. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/dicts/external_dicts_dict/) diff --git a/docs/ru/query_language/dicts/external_dicts_dict_layout.md b/docs/ru/query_language/dicts/external_dicts_dict_layout.md index d2ee91d5d3d..199b6926f7a 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict_layout.md +++ b/docs/ru/query_language/dicts/external_dicts_dict_layout.md @@ -290,3 +290,5 @@ dictGetString('prefix', 'asn', tuple(IPv6StringToNum('2001:db8::1'))) Никакие другие типы не поддерживаются. Функция возвращает атрибут для префикса, соответствующего данному IP-адресу. Если есть перекрывающиеся префиксы, возвращается наиболее специфический. Данные хранятся в побитовом дереве (`trie`), он должены полностью помещаться в оперативной памяти. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/dicts/external_dicts_dict_layout/) diff --git a/docs/ru/query_language/dicts/external_dicts_dict_lifetime.md b/docs/ru/query_language/dicts/external_dicts_dict_lifetime.md index f8898684353..4e78a838165 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict_lifetime.md +++ b/docs/ru/query_language/dicts/external_dicts_dict_lifetime.md @@ -56,3 +56,5 @@ ClickHouse периодически обновляет словари. Инте ... ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/dicts/external_dicts_dict_lifetime/) diff --git a/docs/ru/query_language/dicts/external_dicts_dict_sources.md b/docs/ru/query_language/dicts/external_dicts_dict_sources.md index 6b2c455b5bc..e1fa4578ac8 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict_sources.md +++ b/docs/ru/query_language/dicts/external_dicts_dict_sources.md @@ -111,7 +111,7 @@ ```xml DatabaseName - TableName
+ ShemaName.TableName
DSN=some_parameters SQL_QUERY
@@ -119,11 +119,13 @@ Поля настройки: -- `db` - имя базы данных. Не указывать, если имя базы задано в параметрах ``. -- `table` - имя таблицы. +- `db` - имя базы данных. Не указывать, если имя базы задано в параметрах. ``. +- `table` - имя таблицы и схемы, если она есть. - `connection_string` - строка соединения. - `invalidate_query` - запрос для проверки статуса словаря. Необязательный параметр. Читайте подробнее в разделе [Обновление словарей](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime). +ClickHouse получает от ODBC-драйвера информацию о квотировании и квотирует настройки в запросах к драйверу, поэтому имя таблицы нужно указывать в соответствии с регистром имени таблицы в базе данных. + ### Выявленная уязвимость в функционировании ODBC словарей !!! attention @@ -421,3 +423,5 @@ MySQL можно подключить на локальном хосте чер - `password` - пароль пользователя MongoDB. - `db` - имя базы данных. - `collection` - имя коллекции. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/dicts/external_dicts_dict_sources/) diff --git a/docs/ru/query_language/dicts/external_dicts_dict_structure.md b/docs/ru/query_language/dicts/external_dicts_dict_structure.md index 15952024c06..e26eb0282bc 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict_structure.md +++ b/docs/ru/query_language/dicts/external_dicts_dict_structure.md @@ -116,3 +116,5 @@ ClickHouse поддерживает следующие виды ключей: - `injective` - Признак инъективности отображения `id -> attribute`. Если `true`, то можно оптимизировать `GROUP BY`. По умолчанию, `false`. - `is_object_id` - Признак того, что запрос выполняется к документу MongoDB по `ObjectID`. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/dicts/external_dicts_dict_structure/) diff --git a/docs/ru/query_language/dicts/index.md b/docs/ru/query_language/dicts/index.md index 30d5d705b1e..2ea9f67ce26 100644 --- a/docs/ru/query_language/dicts/index.md +++ b/docs/ru/query_language/dicts/index.md @@ -10,3 +10,5 @@ ClickHouse поддерживает: - [Встроенные словари](internal_dicts.md#internal_dicts) со специфическим [набором функций](../functions/ym_dict_functions.md#ym_dict_functions). - [Подключаемые (внешние) словари](external_dicts.md#dicts-external_dicts) с [набором функций](../functions/ext_dict_functions.md#ext_dict_functions). + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/dicts/) diff --git a/docs/ru/query_language/dicts/internal_dicts.md b/docs/ru/query_language/dicts/internal_dicts.md index 0a61652dd0d..da4e9846e55 100644 --- a/docs/ru/query_language/dicts/internal_dicts.md +++ b/docs/ru/query_language/dicts/internal_dicts.md @@ -46,3 +46,5 @@ ClickHouse содержит встроенную возможность рабо Рекомендуется периодически обновлять словари с геобазой. При обновлении, генерируйте новые файлы, записывая их в отдельное место, а только когда всё готово - переименовывайте в файлы, которые использует сервер. Также имеются функции для работы с идентификаторами операционных систем и поисковых систем Яндекс.Метрики, пользоваться которыми не нужно. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/dicts/internal_dicts/) diff --git a/docs/ru/query_language/functions/arithmetic_functions.md b/docs/ru/query_language/functions/arithmetic_functions.md index 70c6826d8b3..8707e0dda08 100644 --- a/docs/ru/query_language/functions/arithmetic_functions.md +++ b/docs/ru/query_language/functions/arithmetic_functions.md @@ -4,11 +4,11 @@ Пример: -```sql +``` sql SELECT toTypeName(0), toTypeName(0 + 0), toTypeName(0 + 0 + 0), toTypeName(0 + 0 + 0 + 0) ``` -```text +``` ┌─toTypeName(0)─┬─toTypeName(plus(0, 0))─┬─toTypeName(plus(plus(0, 0), 0))─┬─toTypeName(plus(plus(plus(0, 0), 0), 0))─┐ │ UInt8 │ UInt16 │ UInt32 │ UInt64 │ └───────────────┴────────────────────────┴─────────────────────────────────┴──────────────────────────────────────────┘ @@ -71,3 +71,5 @@ SELECT toTypeName(0), toTypeName(0 + 0), toTypeName(0 + 0 + 0), toTypeName(0 + 0 ## lcm(a, b) Вычисляет наименьшее общее кратное чисел. При делении на ноль или при делении минимального отрицательного числа на минус единицу, кидается исключение. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/arithmetic_functions/) diff --git a/docs/ru/query_language/functions/array_functions.md b/docs/ru/query_language/functions/array_functions.md index 4fab96f32e7..53bf3b94af6 100644 --- a/docs/ru/query_language/functions/array_functions.md +++ b/docs/ru/query_language/functions/array_functions.md @@ -53,7 +53,7 @@ arrayConcat(arrays) **Пример** -```sql +``` sql SELECT arrayConcat([1, 2], [3, 4], [5, 6]) AS res ``` ``` @@ -123,7 +123,7 @@ SELECT countEqual([1, 2, NULL, NULL], NULL) Эта функция обычно используется совместно с ARRAY JOIN. Она позволяет, после применения ARRAY JOIN, посчитать что-либо только один раз для каждого массива. Пример: -```sql +``` sql SELECT count() AS Reaches, countIf(num = 1) AS Hits @@ -135,7 +135,7 @@ WHERE CounterID = 160656 LIMIT 10 ``` -```text +``` ┌─Reaches─┬──Hits─┐ │ 95606 │ 31406 │ └─────────┴───────┘ @@ -143,7 +143,7 @@ LIMIT 10 В этом примере, Reaches - число достижений целей (строк, получившихся после применения ARRAY JOIN), а Hits - число хитов (строк, которые были до ARRAY JOIN). В данном случае, тот же результат можно получить проще: -```sql +``` sql SELECT sum(length(GoalsReached)) AS Reaches, count() AS Hits @@ -151,7 +151,7 @@ FROM test.hits WHERE (CounterID = 160656) AND notEmpty(GoalsReached) ``` -```text +``` ┌─Reaches─┬──Hits─┐ │ 95606 │ 31406 │ └─────────┴───────┘ @@ -166,7 +166,7 @@ WHERE (CounterID = 160656) AND notEmpty(GoalsReached) Эта функция полезна при использовании ARRAY JOIN и агрегации по элементам массива. Пример: -```sql +``` sql SELECT Goals.ID AS GoalID, sum(Sign) AS Reaches, @@ -181,7 +181,7 @@ ORDER BY Reaches DESC LIMIT 10 ``` -```text +``` ┌──GoalID─┬─Reaches─┬─Visits─┐ │ 53225 │ 3214 │ 1097 │ │ 2825062 │ 3188 │ 1097 │ @@ -200,11 +200,11 @@ LIMIT 10 Функция arrayEnumerateUniq может принимать несколько аргументов - массивов одинаковых размеров. В этом случае, уникальность считается для кортежей элементов на одинаковых позициях всех массивов. -```sql +``` sql SELECT arrayEnumerateUniq([1, 1, 1, 2, 2, 2], [1, 1, 2, 1, 1, 2]) AS res ``` -```text +``` ┌─res───────────┐ │ [1,2,1,1,2,1] │ └───────────────┘ @@ -226,7 +226,7 @@ arrayPopBack(array) **Пример** -```sql +``` sql SELECT arrayPopBack([1, 2, 3]) AS res ``` ``` @@ -249,7 +249,7 @@ arrayPopFront(array) **Пример** -```sql +``` sql SELECT arrayPopFront([1, 2, 3]) AS res ``` ``` @@ -273,7 +273,7 @@ arrayPushBack(array, single_value) **Пример** -```sql +``` sql SELECT arrayPushBack(['a'], 'b') AS res ``` ``` @@ -297,7 +297,7 @@ arrayPushFront(array, single_value) **Пример** -```sql +``` sql SELECT arrayPushBack(['b'], 'a') AS res ``` ``` @@ -359,7 +359,7 @@ arraySlice(array, offset[, length]) **Пример** -```sql +``` sql SELECT arraySlice([1, 2, NULL, 4, 5], 2, 3) AS res ``` ``` @@ -378,3 +378,5 @@ SELECT arraySlice([1, 2, NULL, 4, 5], 2, 3) AS res ## arrayJoin(arr) Особенная функция. Смотрите раздел ["Функция arrayJoin"](array_join.md#functions_arrayjoin). + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/array_functions/) diff --git a/docs/ru/query_language/functions/array_join.md b/docs/ru/query_language/functions/array_join.md index 40878eee027..57bdbe94c75 100644 --- a/docs/ru/query_language/functions/array_join.md +++ b/docs/ru/query_language/functions/array_join.md @@ -17,14 +17,16 @@ Пример: -```sql +``` sql SELECT arrayJoin([1, 2, 3] AS src) AS dst, 'Hello', src ``` -```text +``` ┌─dst─┬─\'Hello\'─┬─src─────┐ │ 1 │ Hello │ [1,2,3] │ │ 2 │ Hello │ [1,2,3] │ │ 3 │ Hello │ [1,2,3] │ └─────┴───────────┴─────────┘ ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/array_join/) diff --git a/docs/ru/query_language/functions/bit_functions.md b/docs/ru/query_language/functions/bit_functions.md index b2fc83b6e0f..7ba32ad6ba8 100644 --- a/docs/ru/query_language/functions/bit_functions.md +++ b/docs/ru/query_language/functions/bit_functions.md @@ -15,3 +15,5 @@ ## bitShiftLeft(a, b) ## bitShiftRight(a, b) + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/bit_functions/) diff --git a/docs/ru/query_language/functions/comparison_functions.md b/docs/ru/query_language/functions/comparison_functions.md index 99879e2dc7d..392c6c2573c 100644 --- a/docs/ru/query_language/functions/comparison_functions.md +++ b/docs/ru/query_language/functions/comparison_functions.md @@ -29,3 +29,5 @@ ## lessOrEquals, оператор `<=` ## greaterOrEquals, оператор `>=` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/comparison_functions/) diff --git a/docs/ru/query_language/functions/conditional_functions.md b/docs/ru/query_language/functions/conditional_functions.md index 0393094fb4a..0140a008123 100644 --- a/docs/ru/query_language/functions/conditional_functions.md +++ b/docs/ru/query_language/functions/conditional_functions.md @@ -46,3 +46,5 @@ multiIf(cond_1, then_1, cond_2, then_2...else) │ ᴺᵁᴸᴸ │ └────────────────────────────────────────────┘ ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/conditional_functions/) diff --git a/docs/ru/query_language/functions/date_time_functions.md b/docs/ru/query_language/functions/date_time_functions.md index 40f12cea07a..c0a654a0507 100644 --- a/docs/ru/query_language/functions/date_time_functions.md +++ b/docs/ru/query_language/functions/date_time_functions.md @@ -4,7 +4,7 @@ Все функции по работе с датой и временем, для которых это имеет смысл, могут принимать второй, необязательный аргумент - имя часового пояса. Пример: Asia/Yekaterinburg. В этом случае, они используют не локальный часовой пояс (по умолчанию), а указанный. -```sql +``` sql SELECT toDateTime('2016-06-15 23:00:00') AS time, toDate(time) AS date_local, @@ -12,7 +12,7 @@ SELECT toString(time, 'US/Samoa') AS time_samoa ``` -```text +``` ┌────────────────time─┬─date_local─┬─date_yekat─┬─time_samoa──────────┐ │ 2016-06-15 23:00:00 │ 2016-06-15 │ 2016-06-16 │ 2016-06-15 09:00:00 │ └─────────────────────┴────────────┴────────────┴─────────────────────┘ @@ -153,3 +153,5 @@ SELECT |%y|год, последние 2 цифры (00-99)|18| |%Y|год, 4 цифры|2018| |%%|символ %|%| + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/date_time_functions/) diff --git a/docs/ru/query_language/functions/encoding_functions.md b/docs/ru/query_language/functions/encoding_functions.md index f0961822ec3..c3825cd22af 100644 --- a/docs/ru/query_language/functions/encoding_functions.md +++ b/docs/ru/query_language/functions/encoding_functions.md @@ -19,3 +19,5 @@ ## bitmaskToArray(num) Принимает целое число. Возвращает массив чисел типа UInt64, содержащий степени двойки, в сумме дающих исходное число; числа в массиве идут по возрастанию. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/encoding_functions/) diff --git a/docs/ru/query_language/functions/ext_dict_functions.md b/docs/ru/query_language/functions/ext_dict_functions.md index 67702c538a4..d80d309a02c 100644 --- a/docs/ru/query_language/functions/ext_dict_functions.md +++ b/docs/ru/query_language/functions/ext_dict_functions.md @@ -38,3 +38,5 @@ ## dictHas `dictHas('dict_name', id)` - проверить наличие ключа в словаре. Возвращает значение типа UInt8, равное 0, если ключа нет и 1, если ключ есть. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/ext_dict_functions/) diff --git a/docs/ru/query_language/functions/functions_for_nulls.md b/docs/ru/query_language/functions/functions_for_nulls.md index 5d5314efb6b..9173b466f5b 100644 --- a/docs/ru/query_language/functions/functions_for_nulls.md +++ b/docs/ru/query_language/functions/functions_for_nulls.md @@ -287,3 +287,5 @@ SELECT toTypeName(toNullable(10)) │ Nullable(UInt8) │ └────────────────────────────┘ ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/functions_for_nulls/) diff --git a/docs/ru/query_language/functions/geo.md b/docs/ru/query_language/functions/geo.md index 0b0c6fa24c3..f1b460a4c41 100644 --- a/docs/ru/query_language/functions/geo.md +++ b/docs/ru/query_language/functions/geo.md @@ -25,11 +25,11 @@ greatCircleDistance(lon1Deg, lat1Deg, lon2Deg, lat2Deg) **Пример** -```sql +``` sql SELECT greatCircleDistance(55.755831, 37.617673, -55.755831, -37.617673) ``` -```text +``` ┌─greatCircleDistance(55.755831, 37.617673, -55.755831, -37.617673)─┐ │ 14132374.194975413 │ └───────────────────────────────────────────────────────────────────┘ @@ -59,11 +59,11 @@ pointInEllipses(x, y, x₀, y₀, a₀, b₀,...,xₙ, yₙ, aₙ, bₙ) **Пример** -```sql +``` sql SELECT pointInEllipses(55.755831, 37.617673, 55.755831, 37.617673, 1.0, 2.0) ``` -```text +``` ┌─pointInEllipses(55.755831, 37.617673, 55.755831, 37.617673, 1., 2.)─┐ │ 1 │ └─────────────────────────────────────────────────────────────────────┘ @@ -91,7 +91,7 @@ pointInPolygon((x, y), [(a, b), (c, d) ...], ...) **Пример** -```sql +``` sql SELECT pointInPolygon((3., 3.), [(6, 0), (8, 4), (5, 8), (0, 2)]) AS res ``` ``` @@ -99,3 +99,5 @@ SELECT pointInPolygon((3., 3.), [(6, 0), (8, 4), (5, 8), (0, 2)]) AS res │ 1 │ └─────┘ ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/geo/) diff --git a/docs/ru/query_language/functions/hash_functions.md b/docs/ru/query_language/functions/hash_functions.md index 905b14182c4..62699d38d5b 100644 --- a/docs/ru/query_language/functions/hash_functions.md +++ b/docs/ru/query_language/functions/hash_functions.md @@ -55,3 +55,5 @@ SipHash - криптографическая хэш-функция. Работа `URLHash(s)` - вычислить хэш от строки без одного завершающего символа `/`, `?` или `#` на конце, если такой там есть. `URLHash(s, N)` - вычислить хэш от строки до N-го уровня в иерархии URL, без одного завершающего символа `/`, `?` или `#` на конце, если такой там есть. Уровни аналогичные URLHierarchy. Функция специфична для Яндекс.Метрики. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/hash_functions/) diff --git a/docs/ru/query_language/functions/higher_order_functions.md b/docs/ru/query_language/functions/higher_order_functions.md index 2dfe08066bc..e9cb1d0de17 100644 --- a/docs/ru/query_language/functions/higher_order_functions.md +++ b/docs/ru/query_language/functions/higher_order_functions.md @@ -22,17 +22,17 @@ Примеры: -```sql +``` sql SELECT arrayFilter(x -> x LIKE '%World%', ['Hello', 'abc World']) AS res ``` -```text +``` ┌─res───────────┐ │ ['abc World'] │ └───────────────┘ ``` -```sql +``` sql SELECT arrayFilter( (i, x) -> x LIKE '%World%', @@ -41,7 +41,7 @@ SELECT AS res ``` -```text +``` ┌─res─┐ │ [2] │ └─────┘ @@ -71,11 +71,11 @@ SELECT Пример: -```sql +``` sql SELECT arrayCumSum([1, 1, 1, 1]) AS res ``` -```text +``` ┌─res──────────┐ │ [1, 2, 3, 4] │ └──────────────┘ @@ -90,11 +90,11 @@ SELECT arrayCumSum([1, 1, 1, 1]) AS res Пример: -```sql +``` sql SELECT arraySort((x, y) -> y, ['hello', 'world'], [2, 1]); ``` -```text +``` ┌─res────────────────┐ │ ['world', 'hello'] │ └────────────────────┘ @@ -103,3 +103,5 @@ SELECT arraySort((x, y) -> y, ['hello', 'world'], [2, 1]); ### arrayReverseSort(\[func,\] arr1, ...) Возвращает отсортированный в нисходящем порядке массив `arr1`. Если задана функция `func`, то порядок сортировки определяется результатом применения функции `func` на элементы массива (массивов). + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/higher_order_functions/) diff --git a/docs/ru/query_language/functions/in_functions.md b/docs/ru/query_language/functions/in_functions.md index 71aa8187e4f..606867c85bb 100644 --- a/docs/ru/query_language/functions/in_functions.md +++ b/docs/ru/query_language/functions/in_functions.md @@ -12,3 +12,5 @@ Функция, позволяющая достать столбец из кортежа. N - индекс столбца начиная с 1. N должно быть константой. N должно быть целым строго положительным числом не большим размера кортежа. Выполнение функции ничего не стоит. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/in_functions/) diff --git a/docs/ru/query_language/functions/index.md b/docs/ru/query_language/functions/index.md index 870eccb24ab..c24e020097d 100644 --- a/docs/ru/query_language/functions/index.md +++ b/docs/ru/query_language/functions/index.md @@ -61,3 +61,5 @@ Другой пример - функция `hostName` вернёт имя сервера, на котором она выполняется, и это можно использовать для служебных целей - чтобы в запросе `SELECT` сделать `GROUP BY` по серверам. Если функция в запросе выполняется на сервере-инициаторе запроса, а вам нужно, чтобы она выполнялась на удалённых серверах, вы можете обернуть её в агрегатную функцию any или добавить в ключ в `GROUP BY`. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/) diff --git a/docs/ru/query_language/functions/ip_address_functions.md b/docs/ru/query_language/functions/ip_address_functions.md index 5d1945d5545..a0fe39a77b0 100644 --- a/docs/ru/query_language/functions/ip_address_functions.md +++ b/docs/ru/query_language/functions/ip_address_functions.md @@ -11,7 +11,7 @@ Пример: -```sql +``` sql SELECT IPv4NumToStringClassC(ClientIP) AS k, count() AS c @@ -21,7 +21,7 @@ ORDER BY c DESC LIMIT 10 ``` -```text +``` ┌─k──────────────┬─────c─┐ │ 83.149.9.xxx │ 26238 │ │ 217.118.81.xxx │ 26074 │ @@ -42,17 +42,17 @@ LIMIT 10 Принимает значение типа FixedString(16), содержащее IPv6-адрес в бинарном виде. Возвращает строку, содержащую этот адрес в текстовом виде. IPv6-mapped IPv4 адреса выводится в формате ::ffff:111.222.33.44. Примеры: -```sql +``` sql SELECT IPv6NumToString(toFixedString(unhex('2A0206B8000000000000000000000011'), 16)) AS addr ``` -```text +``` ┌─addr─────────┐ │ 2a02:6b8::11 │ └──────────────┘ ``` -```sql +``` sql SELECT IPv6NumToString(ClientIP6 AS k), count() AS c @@ -63,7 +63,7 @@ ORDER BY c DESC LIMIT 10 ``` -```text +``` ┌─IPv6NumToString(ClientIP6)──────────────┬─────c─┐ │ 2a02:2168:aaa:bbbb::2 │ 24695 │ │ 2a02:2698:abcd:abcd:abcd:abcd:8888:5555 │ 22408 │ @@ -78,7 +78,7 @@ LIMIT 10 └─────────────────────────────────────────┴───────┘ ``` -```sql +``` sql SELECT IPv6NumToString(ClientIP6 AS k), count() AS c @@ -89,7 +89,7 @@ ORDER BY c DESC LIMIT 10 ``` -```text +``` ┌─IPv6NumToString(ClientIP6)─┬──────c─┐ │ ::ffff:94.26.111.111 │ 747440 │ │ ::ffff:37.143.222.4 │ 529483 │ @@ -107,3 +107,5 @@ LIMIT 10 ## IPv6StringToNum(s) Функция, обратная к IPv6NumToString. Если IPv6 адрес в неправильном формате, то возвращает строку из нулевых байт. HEX может быть в любом регистре. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/ip_address_functions/) diff --git a/docs/ru/query_language/functions/json_functions.md b/docs/ru/query_language/functions/json_functions.md index 302bed008ef..6958f33e2ba 100644 --- a/docs/ru/query_language/functions/json_functions.md +++ b/docs/ru/query_language/functions/json_functions.md @@ -29,7 +29,7 @@ Примеры: -```text +``` visitParamExtractRaw('{"abc":"\\n\\u0000"}', 'abc') = '"\\n\\u0000"' visitParamExtractRaw('{"abc":{"def":[1,2,3]}}', 'abc') = '{"def":[1,2,3]}' ``` @@ -39,7 +39,7 @@ visitParamExtractRaw('{"abc":{"def":[1,2,3]}}', 'abc') = '{"def":[1,2,3]}' Примеры: -```text +``` visitParamExtractString('{"abc":"\\n\\u0000"}', 'abc') = '\n\0' visitParamExtractString('{"abc":"\\u263a"}', 'abc') = '☺' visitParamExtractString('{"abc":"\\u263"}', 'abc') = '' @@ -47,3 +47,5 @@ visitParamExtractString('{"abc":"hello}', 'abc') = '' ``` На данный момент, не поддерживаются записанные в формате `\uXXXX\uYYYY` кодовые точки не из basic multilingual plane (они переводятся не в UTF-8, а в CESU-8). + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/json_functions/) diff --git a/docs/ru/query_language/functions/logical_functions.md b/docs/ru/query_language/functions/logical_functions.md index 458356b5cc9..6fa2504f933 100644 --- a/docs/ru/query_language/functions/logical_functions.md +++ b/docs/ru/query_language/functions/logical_functions.md @@ -11,3 +11,5 @@ ## not, оператор NOT ## xor + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/logical_functions/) diff --git a/docs/ru/query_language/functions/math_functions.md b/docs/ru/query_language/functions/math_functions.md index bd75e481960..0470a469780 100644 --- a/docs/ru/query_language/functions/math_functions.md +++ b/docs/ru/query_language/functions/math_functions.md @@ -39,11 +39,11 @@ Пример (правило трёх сигм): -```sql +``` sql SELECT erf(3 / sqrt(2)) ``` -```text +``` ┌─erf(divide(3, sqrt(2)))─┐ │ 0.9973002039367398 │ └─────────────────────────┘ @@ -78,3 +78,5 @@ SELECT erf(3 / sqrt(2)) ## pow(x, y) Принимает два числовых аргумента x и y. Возвращает число типа Float64, близкое к x в степени y. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/math_functions/) diff --git a/docs/ru/query_language/functions/other_functions.md b/docs/ru/query_language/functions/other_functions.md index fb41049f23c..d7c70d61d52 100644 --- a/docs/ru/query_language/functions/other_functions.md +++ b/docs/ru/query_language/functions/other_functions.md @@ -72,7 +72,7 @@ SELECT visibleWidth(NULL) Пример: -```sql +``` sql SELECT toHour(EventTime) AS h, count() AS c, @@ -82,7 +82,7 @@ GROUP BY h ORDER BY h ASC ``` -```text +``` ┌──h─┬──────c─┬─bar────────────────┐ │ 0 │ 292907 │ █████████▋ │ │ 1 │ 180563 │ ██████ │ @@ -141,7 +141,7 @@ ORDER BY h ASC Пример: -```sql +``` sql SELECT transform(SearchEngineID, [2, 3], ['Yandex', 'Google'], 'Other') AS title, count() AS c @@ -151,7 +151,7 @@ GROUP BY title ORDER BY c DESC ``` -```text +``` ┌─title─────┬──────c─┐ │ Yandex │ 498635 │ │ Google │ 229872 │ @@ -170,7 +170,7 @@ ORDER BY c DESC Пример: -```sql +``` sql SELECT transform(domain(Referer), ['yandex.ru', 'google.ru', 'vk.com'], ['www.yandex', 'example.com']) AS s, count() AS c @@ -180,7 +180,7 @@ ORDER BY count() DESC LIMIT 10 ``` -```text +``` ┌─s──────────────┬───────c─┐ │ │ 2906259 │ │ www.yandex │ 867767 │ @@ -199,13 +199,13 @@ LIMIT 10 Пример: -```sql +``` sql SELECT arrayJoin([1, 1024, 1024*1024, 192851925]) AS filesize_bytes, formatReadableSize(filesize_bytes) AS filesize ``` -```text +``` ┌─filesize_bytes─┬─filesize───┐ │ 1 │ 1.00 B │ │ 1024 │ 1.00 KiB │ @@ -238,7 +238,7 @@ SELECT Пример: -```sql +``` sql SELECT EventID, EventTime, @@ -255,7 +255,7 @@ FROM ) ``` -```text +``` ┌─EventID─┬───────────EventTime─┬─delta─┐ │ 1106 │ 2016-11-24 00:00:04 │ 0 │ │ 1107 │ 2016-11-24 00:00:05 │ 1 │ @@ -540,3 +540,5 @@ SELECT replicate(1, ['a', 'b', 'c']) │ [1,1,1] │ └───────────────────────────────┘ ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/other_functions/) diff --git a/docs/ru/query_language/functions/random_functions.md b/docs/ru/query_language/functions/random_functions.md index 5e74ac10b56..b29f50dfd4e 100644 --- a/docs/ru/query_language/functions/random_functions.md +++ b/docs/ru/query_language/functions/random_functions.md @@ -13,3 +13,5 @@ ## rand64 Возвращает псевдослучайное число типа UInt64, равномерно распределённое среди всех чисел типа UInt64. Используется linear congruential generator. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/random_functions/) diff --git a/docs/ru/query_language/functions/rounding_functions.md b/docs/ru/query_language/functions/rounding_functions.md index 849c35013b9..269beccffa1 100644 --- a/docs/ru/query_language/functions/rounding_functions.md +++ b/docs/ru/query_language/functions/rounding_functions.md @@ -31,7 +31,7 @@ N может быть отрицательным. **Пример** -```sql +``` sql SELECT number / 2 AS x, round(x) @@ -61,3 +61,5 @@ LIMIT 10 ## roundAge(num) Принимает число. Если число меньше 18 - возвращает 0. Иначе округляет число вниз до чисел из набора: 18, 25, 35, 45, 55. Эта функция специфична для Яндекс.Метрики и предназначена для реализации отчёта по возрасту посетителей. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/rounding_functions/) diff --git a/docs/ru/query_language/functions/splitting_merging_functions.md b/docs/ru/query_language/functions/splitting_merging_functions.md index 1f4d38ac79d..8561048b864 100644 --- a/docs/ru/query_language/functions/splitting_merging_functions.md +++ b/docs/ru/query_language/functions/splitting_merging_functions.md @@ -25,4 +25,5 @@ SELECT alphaTokens('abca1abc') ┌─alphaTokens('abca1abc')─┐ │ ['abca','abc'] │ └─────────────────────────┘ -``` \ No newline at end of file +``` +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/splitting_merging_functions/) diff --git a/docs/ru/query_language/functions/string_functions.md b/docs/ru/query_language/functions/string_functions.md index 43842b59468..a917fb4d91d 100644 --- a/docs/ru/query_language/functions/string_functions.md +++ b/docs/ru/query_language/functions/string_functions.md @@ -58,3 +58,5 @@ ## convertCharset(s, from, to) Возвращает сконвертированную из кодировки from в кодировку to строку s. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/string_functions/) diff --git a/docs/ru/query_language/functions/string_replace_functions.md b/docs/ru/query_language/functions/string_replace_functions.md index 7669b598b9f..498b321fb1b 100644 --- a/docs/ru/query_language/functions/string_replace_functions.md +++ b/docs/ru/query_language/functions/string_replace_functions.md @@ -17,7 +17,7 @@ Пример 1. Переведём дату в американский формат: -```sql +``` sql SELECT DISTINCT EventDate, replaceRegexpOne(toString(EventDate), '(\\d{4})-(\\d{2})-(\\d{2})', '\\2/\\3/\\1') AS res @@ -26,7 +26,7 @@ LIMIT 7 FORMAT TabSeparated ``` -```text +``` 2014-03-17 03/17/2014 2014-03-18 03/18/2014 2014-03-19 03/19/2014 @@ -38,11 +38,11 @@ FORMAT TabSeparated Пример 2. Размножить строку десять раз: -```sql +``` sql SELECT replaceRegexpOne('Hello, World!', '.*', '\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0') AS res ``` -```text +``` ┌─res────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ │ Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World! │ └────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ @@ -51,11 +51,11 @@ SELECT replaceRegexpOne('Hello, World!', '.*', '\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0') ## replaceRegexpAll(haystack, pattern, replacement) То же самое, но делается замена всех вхождений. Пример: -```sql +``` sql SELECT replaceRegexpAll('Hello, World!', '.', '\\0\\0') AS res ``` -```text +``` ┌─res────────────────────────┐ │ HHeelllloo,, WWoorrlldd!! │ └────────────────────────────┘ @@ -64,12 +64,14 @@ SELECT replaceRegexpAll('Hello, World!', '.', '\\0\\0') AS res В качестве исключения, если регулярное выражение сработало на пустой подстроке, то замена делается не более одного раза. Пример: -```sql +``` sql SELECT replaceRegexpAll('Hello, World!', '^', 'here: ') AS res ``` -```text +``` ┌─res─────────────────┐ │ here: Hello, World! │ └─────────────────────┘ ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/string_replace_functions/) diff --git a/docs/ru/query_language/functions/string_search_functions.md b/docs/ru/query_language/functions/string_search_functions.md index 125c57a6138..031082c6659 100644 --- a/docs/ru/query_language/functions/string_search_functions.md +++ b/docs/ru/query_language/functions/string_search_functions.md @@ -44,3 +44,5 @@ ## notLike(haystack, pattern), оператор haystack NOT LIKE pattern То же, что like, но с отрицанием. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/string_search_functions/) diff --git a/docs/ru/query_language/functions/type_conversion_functions.md b/docs/ru/query_language/functions/type_conversion_functions.md index ea39c4f6160..6b512574f6e 100644 --- a/docs/ru/query_language/functions/type_conversion_functions.md +++ b/docs/ru/query_language/functions/type_conversion_functions.md @@ -27,7 +27,7 @@ Форматы даты и даты-с-временем для функций toDate/toDateTime определены следующим образом: -```text +``` YYYY-MM-DD YYYY-MM-DD hh:mm:ss ``` @@ -40,13 +40,13 @@ YYYY-MM-DD hh:mm:ss Дополнительно, функция toString от аргумента типа DateTime может принимать второй аргумент String - имя тайм-зоны. Пример: `Asia/Yekaterinburg` В этом случае, форматирование времени производится согласно указанной тайм-зоне. -```sql +``` sql SELECT now() AS now_local, toString(now(), 'Asia/Yekaterinburg') AS now_yekat ``` -```text +``` ┌───────────now_local─┬─now_yekat───────────┐ │ 2016-06-15 00:11:21 │ 2016-06-15 02:11:21 │ └─────────────────────┴─────────────────────┘ @@ -63,21 +63,21 @@ SELECT Пример: -```sql +``` sql SELECT toFixedString('foo', 8) AS s, toStringCutToZero(s) AS s_cut ``` -```text +``` ┌─s─────────────┬─s_cut─┐ │ foo\0\0\0\0\0 │ foo │ └───────────────┴───────┘ ``` -```sql +``` sql SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut ``` -```text +``` ┌─s──────────┬─s_cut─┐ │ foo\0bar\0 │ foo │ └────────────┴───────┘ @@ -101,7 +101,7 @@ SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut Пример: -```sql +``` sql SELECT '2016-06-15 23:00:00' AS timestamp, CAST(timestamp AS DateTime) AS datetime, @@ -110,7 +110,7 @@ SELECT CAST(timestamp, 'FixedString(22)') AS fixed_string ``` -```text +``` ┌─timestamp───────────┬────────────datetime─┬───────date─┬─string──────────────┬─fixed_string──────────────┐ │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00 │ 2016-06-15 │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00\0\0\0 │ └─────────────────────┴─────────────────────┴────────────┴─────────────────────┴───────────────────────────┘ @@ -135,3 +135,5 @@ SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null │ Nullable(UInt16) │ └─────────────────────────────────────────┘ ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/type_conversion_functions/) diff --git a/docs/ru/query_language/functions/url_functions.md b/docs/ru/query_language/functions/url_functions.md index f4697b9537f..4b4fdc9adda 100644 --- a/docs/ru/query_language/functions/url_functions.md +++ b/docs/ru/query_language/functions/url_functions.md @@ -57,7 +57,7 @@ То же самое, но без протокола и хоста в результате. Элемент / (корень) не включается. Пример: Функция используется для реализации древовидных отчётов по URL в Яндекс.Метрике. -```text +``` URLPathHierarchy('https://example.com/browse/CONV-6788') = [ '/browse/', @@ -69,11 +69,11 @@ URLPathHierarchy('https://example.com/browse/CONV-6788') = Возвращает декодированный URL. Пример: -```sql +``` sql SELECT decodeURLComponent('http://127.0.0.1:8123/?query=SELECT%201%3B') AS DecodedURL; ``` -```text +``` ┌─DecodedURL─────────────────────────────┐ │ http://127.0.0.1:8123/?query=SELECT 1; │ └────────────────────────────────────────┘ @@ -97,3 +97,5 @@ SELECT decodeURLComponent('http://127.0.0.1:8123/?query=SELECT%201%3B') AS Decod ### cutURLParameter(URL, name) Удаляет параметр URL с именем name, если такой есть. Функция работает при допущении, что имя параметра закодировано в URL в точности таким же образом, что и в переданном аргументе. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/url_functions/) diff --git a/docs/ru/query_language/functions/ym_dict_functions.md b/docs/ru/query_language/functions/ym_dict_functions.md index 1761a21a7dd..7d37e1b0c87 100644 --- a/docs/ru/query_language/functions/ym_dict_functions.md +++ b/docs/ru/query_language/functions/ym_dict_functions.md @@ -23,7 +23,7 @@ ClickHouse поддерживает работу одновременно с н Во все функции по работе с регионами, в конце добавлен один необязательный аргумент - ключ словаря. Далее он обозначен как geobase. Пример: -```text +``` regionToCountry(RegionID) - использует словарь по умолчанию: /opt/geo/regions_hierarchy.txt; regionToCountry(RegionID, '') - использует словарь по умолчанию: /opt/geo/regions_hierarchy.txt; regionToCountry(RegionID, 'ua') - использует словарь для ключа ua: /opt/geo/regions_hierarchy_ua.txt; @@ -35,13 +35,13 @@ regionToCountry(RegionID, 'ua') - использует словарь для к ### regionToArea(id\[, geobase\]) Переводит регион в область (тип в геобазе - 5). В остальном, аналогично функции regionToCity. -```sql +``` sql SELECT DISTINCT regionToName(regionToArea(toUInt32(number), 'ua')) FROM system.numbers LIMIT 15 ``` -```text +``` ┌─regionToName(regionToArea(toUInt32(number), \'ua\'))─┐ │ │ │ Москва и Московская область │ @@ -64,13 +64,13 @@ LIMIT 15 ### regionToDistrict(id\[, geobase\]) Переводит регион в федеральный округ (тип в геобазе - 4). В остальном, аналогично функции regionToCity. -```sql +``` sql SELECT DISTINCT regionToName(regionToDistrict(toUInt32(number), 'ua')) FROM system.numbers LIMIT 15 ``` -```text +``` ┌─regionToName(regionToDistrict(toUInt32(number), \'ua\'))─┐ │ │ │ Центральный федеральный округ │ @@ -116,3 +116,5 @@ LIMIT 15 Принимает число типа UInt32 - идентификатор региона из геобазы Яндекса. Вторым аргументом может быть передана строка - название языка. Поддерживаются языки ru, en, ua, uk, by, kz, tr. Если второй аргумент отсутствует - используется язык ru. Если язык не поддерживается - кидается исключение. Возвращает строку - название региона на соответствующем языке. Если региона с указанным идентификатором не существует - возвращается пустая строка. `ua` и `uk` обозначают одно и то же - украинский язык. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/ym_dict_functions/) diff --git a/docs/ru/query_language/index.md b/docs/ru/query_language/index.md index 01666d3e0e7..59313b15f8a 100644 --- a/docs/ru/query_language/index.md +++ b/docs/ru/query_language/index.md @@ -5,3 +5,5 @@ * [CREATE](create.md#create-database) * [ALTER](alter.md#query_language_queries_alter) * [Прочие виды запросов](misc.md#miscellaneous-queries) + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/) diff --git a/docs/ru/query_language/insert_into.md b/docs/ru/query_language/insert_into.md index 9d2e6280e3a..70e4d5529f2 100644 --- a/docs/ru/query_language/insert_into.md +++ b/docs/ru/query_language/insert_into.md @@ -6,7 +6,7 @@ Базовый формат запроса: -```sql +``` sql INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... ``` @@ -19,13 +19,13 @@ INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), . В INSERT можно передавать данные любого [формата](../interfaces/formats.md#formats), который поддерживает ClickHouse. Для этого формат необходимо указать в запросе в явном виде: -```sql +``` sql INSERT INTO [db.]table [(c1, c2, c3)] FORMAT format_name data_set ``` Например, следующий формат запроса идентичен базовому варианту INSERT ... VALUES: -```sql +``` sql INSERT INTO [db.]table [(c1, c2, c3)] FORMAT Values (v11, v12, v13), (v21, v22, v23), ... ``` @@ -33,7 +33,7 @@ ClickHouse отсекает все пробелы и один перенос с Пример: -```sql +``` sql INSERT INTO t FORMAT TabSeparated 11 Hello, world! 22 Qwerty @@ -43,7 +43,7 @@ INSERT INTO t FORMAT TabSeparated ### Вставка результатов `SELECT` -```sql +``` sql INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... ``` @@ -65,3 +65,5 @@ INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... - Данные поступают в режиме реального времени. - Вы загружаете данные, которые как правило отсортированы по времени. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/insert_into/) diff --git a/docs/ru/query_language/misc.md b/docs/ru/query_language/misc.md index 86080a7360c..5bfddb64b43 100644 --- a/docs/ru/query_language/misc.md +++ b/docs/ru/query_language/misc.md @@ -12,7 +12,7 @@ Если таблица перед этим была отсоединена (`DETACH`), т.е. её структура известна, то можно использовать сокращенную форму записи без определения структуры. -```sql +``` sql ATTACH TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] ``` @@ -21,14 +21,14 @@ ATTACH TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] ## DROP Запрос имеет два вида: `DROP DATABASE` и `DROP TABLE`. -```sql +``` sql DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster] ``` Удаляет все таблицы внутри базы данных db, а затем саму базу данных db. Если указано `IF EXISTS` - не выдавать ошибку, если база данных не существует. -```sql +``` sql DROP [TEMPORARY] TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] ``` @@ -38,7 +38,7 @@ DROP [TEMPORARY] TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] ## DETACH Удаляет из сервера информацию о таблице name. Сервер перестаёт знать о существовании таблицы. -```sql +``` sql DETACH TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] ``` @@ -50,7 +50,7 @@ DETACH TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] ## RENAME Переименовывает одну или несколько таблиц. -```sql +``` sql RENAME TABLE [db11.]name11 TO [db12.]name12, [db21.]name21 TO [db22.]name22, ... [ON CLUSTER cluster] ``` @@ -58,7 +58,7 @@ RENAME TABLE [db11.]name11 TO [db12.]name12, [db21.]name21 TO [db22.]name22, ... ## SHOW DATABASES -```sql +``` sql SHOW DATABASES [INTO OUTFILE filename] [FORMAT format] ``` @@ -69,7 +69,7 @@ SHOW DATABASES [INTO OUTFILE filename] [FORMAT format] ## SHOW TABLES -```sql +``` sql SHOW [TEMPORARY] TABLES [FROM db] [LIKE 'pattern'] [INTO OUTFILE filename] [FORMAT format] ``` @@ -84,7 +84,7 @@ SHOW [TEMPORARY] TABLES [FROM db] [LIKE 'pattern'] [INTO OUTFILE filename] [FORM ## SHOW PROCESSLIST -```sql +``` sql SHOW PROCESSLIST [INTO OUTFILE filename] [FORMAT format] ``` @@ -116,7 +116,7 @@ watch -n1 "clickhouse-client --query='SHOW PROCESSLIST'" ## SHOW CREATE TABLE -```sql +``` sql SHOW CREATE [TEMPORARY] TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] ``` @@ -124,7 +124,7 @@ SHOW CREATE [TEMPORARY] TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] ## DESCRIBE TABLE -```sql +``` sql DESC|DESCRIBE TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] ``` @@ -134,7 +134,7 @@ DESC|DESCRIBE TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] ## EXISTS -```sql +``` sql EXISTS [TEMPORARY] TABLE [db.]name [INTO OUTFILE filename] [FORMAT format] ``` @@ -142,7 +142,7 @@ EXISTS [TEMPORARY] TABLE [db.]name [INTO OUTFILE filename] [FORMAT format] ## USE -```sql +``` sql USE db ``` @@ -152,7 +152,7 @@ USE db ## SET -```sql +``` sql SET param = value ``` @@ -165,7 +165,7 @@ SET param = value ## OPTIMIZE -```sql +``` sql OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition] [FINAL] ``` @@ -179,7 +179,7 @@ OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition] [FINAL] ## KILL QUERY -```sql +``` sql KILL QUERY [ON CLUSTER cluster] WHERE [SYNC|ASYNC|TEST] @@ -190,7 +190,7 @@ KILL QUERY [ON CLUSTER cluster] Запросы для принудительной остановки выбираются из таблицы system.processes с помощью условия, указанного в секции `WHERE` запроса `KILL`. Примеры: -```sql +``` sql -- Принудительно останавливает все запросы с указанным query_id: KILL QUERY WHERE query_id='2-857d-4a57-9ee0-327da5d60a90' @@ -210,3 +210,5 @@ Readonly-пользователи могут останавливать толь 3. остальные значения описывают причину невозможности остановки запроса. Тестовый вариант запроса (`TEST`) только проверяет права пользователя и выводит список запросов для остановки. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/misc/) diff --git a/docs/ru/query_language/operators.md b/docs/ru/query_language/operators.md index c5f90588faa..3814e1699d4 100644 --- a/docs/ru/query_language/operators.md +++ b/docs/ru/query_language/operators.md @@ -87,7 +87,7 @@ ## Условное выражение -```sql +``` sql CASE [x] WHEN a THEN b [WHEN ... THEN ...] @@ -173,3 +173,5 @@ WHERE isNotNull(y) 1 rows in set. Elapsed: 0.002 sec. ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/operators/) diff --git a/docs/ru/query_language/select.md b/docs/ru/query_language/select.md index 81537d1573b..4bf5e1020fb 100644 --- a/docs/ru/query_language/select.md +++ b/docs/ru/query_language/select.md @@ -2,7 +2,7 @@ `SELECT` осуществляет выборку данных. -```sql +``` sql SELECT [DISTINCT] expr_list [FROM [db.]table | (subquery) | table_function] [FINAL] [SAMPLE sample_coeff] @@ -59,7 +59,7 @@ SELECT [DISTINCT] expr_list Пример: -```sql +``` sql SELECT Title, count() * 10 AS PageViews @@ -90,7 +90,7 @@ ORDER BY PageViews DESC LIMIT 1000 `ARRAY JOIN` - это, по сути, `INNER JOIN` с массивом. Пример: -```text +``` :) CREATE TABLE arrays_test (s String, arr Array(UInt8)) ENGINE = Memory CREATE TABLE arrays_test @@ -143,7 +143,7 @@ ARRAY JOIN arr Для массива в секции ARRAY JOIN может быть указан алиас. В этом случае, элемент массива будет доступен под этим алиасом, а сам массив - под исходным именем. Пример: -```text +``` :) SELECT s, arr, a FROM arrays_test ARRAY JOIN arr AS a SELECT s, arr, a @@ -163,7 +163,7 @@ ARRAY JOIN arr AS a В секции ARRAY JOIN может быть указано несколько массивов одинаковых размеров через запятую. В этом случае, JOIN делается с ними одновременно (прямая сумма, а не прямое произведение). Пример: -```text +``` :) SELECT s, arr, a, num, mapped FROM arrays_test ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num, arrayMap(x -> x + 1, arr) AS mapped SELECT s, arr, a, num, mapped @@ -199,7 +199,7 @@ ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num ARRAY JOIN также работает с вложенными структурами данных. Пример: -```text +``` :) CREATE TABLE nested_test (s String, nest Nested(x UInt8, y UInt32)) ENGINE = Memory CREATE TABLE nested_test @@ -254,7 +254,7 @@ ARRAY JOIN nest При указании имени вложенной структуры данных в ARRAY JOIN, смысл такой же, как ARRAY JOIN со всеми элементами-массивами, из которых она состоит. Пример: -```text +``` :) SELECT s, nest.x, nest.y FROM nested_test ARRAY JOIN nest.x, nest.y SELECT s, `nest.x`, `nest.y` @@ -274,7 +274,7 @@ ARRAY JOIN `nest.x`, `nest.y` Такой вариант тоже имеет смысл: -```text +``` :) SELECT s, nest.x, nest.y FROM nested_test ARRAY JOIN nest.x SELECT s, `nest.x`, `nest.y` @@ -294,7 +294,7 @@ ARRAY JOIN `nest.x` Алиас для вложенной структуры данных можно использовать, чтобы выбрать как результат JOIN-а, так и исходный массив. Пример: -```text +``` :) SELECT s, n.x, n.y, nest.x, nest.y FROM nested_test ARRAY JOIN nest AS n SELECT s, `n.x`, `n.y`, `nest.x`, `nest.y` @@ -314,7 +314,7 @@ ARRAY JOIN nest AS n Пример использования функции arrayEnumerate: -```text +``` :) SELECT s, n.x, n.y, nest.x, nest.y, num FROM nested_test ARRAY JOIN nest AS n, arrayEnumerate(nest.x) AS num SELECT s, `n.x`, `n.y`, `nest.x`, `nest.y`, num @@ -343,7 +343,7 @@ ARRAY JOIN nest AS n, arrayEnumerate(`nest.x`) AS num Обычный JOIN, не имеет отношения к ARRAY JOIN, который описан выше. -```sql +``` sql [GLOBAL] ANY|ALL INNER|LEFT [OUTER] JOIN (subquery)|table USING columns_list ``` @@ -380,7 +380,7 @@ JOIN-ы бывают нескольких видов: Пример: -```sql +``` sql SELECT CounterID, hits, @@ -404,7 +404,7 @@ ORDER BY hits DESC LIMIT 10 ``` -```text +``` ┌─CounterID─┬───hits─┬─visits─┐ │ 1143050 │ 523264 │ 13665 │ │ 731962 │ 475698 │ 102716 │ @@ -500,7 +500,7 @@ WHERE isNull(y) Пример: -```sql +``` sql SELECT count(), median(FetchTiming > 60 ? 60 : FetchTiming), @@ -514,7 +514,7 @@ FROM hits Пример: -```sql +``` sql SELECT domainWithoutWWW(URL) AS domain, count(), @@ -610,7 +610,7 @@ GROUP BY вычисляет для каждого встретившегося Пример: -```sql +``` sql SELECT domainWithoutWWW(URL) AS domain, domainWithoutWWW(REFERRER_URL) AS referrer, @@ -731,7 +731,7 @@ n и m должны быть неотрицательными целыми чи Произвольное количество запросов может быть объединено с помощью `UNION ALL`. Пример: -```sql +``` sql SELECT CounterID, 1 AS table, toInt64(count()) AS c FROM test.hits GROUP BY CounterID @@ -779,7 +779,7 @@ SELECT CounterID, 2 AS table, sum(Sign) AS c Примеры: -```sql +``` sql SELECT UserID IN (123, 456) FROM ... SELECT (CounterID, UserID) IN ((34, 123), (101500, 456)) FROM ... ``` @@ -798,7 +798,7 @@ SELECT (CounterID, UserID) IN ((34, 123), (101500, 456)) FROM ... В подзапросе может быть указано более одного столбца для фильтрации кортежей. Пример: -```sql +``` sql SELECT (CounterID, UserID) IN (SELECT CounterID, UserID FROM ...) FROM ... ``` @@ -807,7 +807,7 @@ SELECT (CounterID, UserID) IN (SELECT CounterID, UserID FROM ...) FROM ... Оператор IN и подзапрос могут встречаться в любой части запроса, в том числе в агрегатных и лямбда функциях. Пример: -```sql +``` sql SELECT EventDate, avg(UserID IN @@ -821,7 +821,7 @@ GROUP BY EventDate ORDER BY EventDate ASC ``` -```text +``` ┌──EventDate─┬────ratio─┐ │ 2014-03-17 │ 1 │ │ 2014-03-18 │ 0.807696 │ @@ -893,13 +893,13 @@ FROM t_null Например, запрос -```sql +``` sql SELECT uniq(UserID) FROM distributed_table ``` будет отправлен на все удалённые серверы в виде -```sql +``` sql SELECT uniq(UserID) FROM local_table ``` @@ -907,7 +907,7 @@ SELECT uniq(UserID) FROM local_table Теперь рассмотрим запрос с IN-ом: -```sql +``` sql SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM local_table WHERE CounterID = 34) ``` @@ -915,7 +915,7 @@ SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID I Этот запрос будет отправлен на все удалённые серверы в виде -```sql +``` sql SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM local_table WHERE CounterID = 34) ``` @@ -925,19 +925,19 @@ SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID IN (SEL Чтобы исправить работу запроса, когда данные размазаны по серверам кластера произвольным образом, можно было бы указать **distributed_table** внутри подзапроса. Запрос будет выглядеть так: -```sql +``` sql SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) ``` Этот запрос будет отправлен на все удалённые серверы в виде -```sql +``` sql SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) ``` На каждом удалённом сервере начнёт выполняться подзапрос. Так как в подзапросе используется распределённая таблица, то подзапрос будет, на каждом удалённом сервере, снова отправлен на каждый удалённый сервер, в виде -```sql +``` sql SELECT UserID FROM local_table WHERE CounterID = 34 ``` @@ -945,19 +945,19 @@ SELECT UserID FROM local_table WHERE CounterID = 34 В таких случаях всегда следует использовать GLOBAL IN вместо IN. Рассмотрим его работу для запроса -```sql +``` sql SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID GLOBAL IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) ``` На сервере-инициаторе запроса будет выполнен подзапрос -```sql +``` sql SELECT UserID FROM distributed_table WHERE CounterID = 34 ``` , и результат будет сложен во временную таблицу в оперативке. Затем запрос будет отправлен на каждый удалённый сервер в виде -```sql +``` sql SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID GLOBAL IN _data1 ``` @@ -999,3 +999,5 @@ SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID GLOBAL - в подзапросах (так как из подзапросов выкидываются столбцы, не нужные для внешнего запроса). В других случаях использование звёздочки является издевательством над системой, так как вместо преимуществ столбцовой СУБД вы получаете недостатки. То есть использовать звёздочку не рекомендуется. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/select/) diff --git a/docs/ru/query_language/syntax.md b/docs/ru/query_language/syntax.md index 95609b37109..a09074a5681 100644 --- a/docs/ru/query_language/syntax.md +++ b/docs/ru/query_language/syntax.md @@ -4,7 +4,7 @@ Во всех случаях кроме запроса INSERT, используется только полноценный парсер SQL. В запросе INSERT используется оба парсера: -```sql +``` sql INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') ``` @@ -100,7 +100,7 @@ INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') В запросе SELECT, в выражениях могут быть указаны синонимы с помощью ключевого слова AS. Слева от AS стоит любое выражение. Справа от AS стоит идентификатор - имя для синонима. В отличие от стандартного SQL, синонимы могут объявляться не только на верхнем уровне выражений: -```sql +``` sql SELECT (1 AS n) + 2, n ``` @@ -115,3 +115,5 @@ SELECT (1 AS n) + 2, n Выражение представляет собой функцию, идентификатор, литерал, применение оператора, выражение в скобках, подзапрос, звёздочку; и может содержать синоним. Список выражений - одно выражение или несколько выражений через запятую. Функции и операторы, в свою очередь, в качестве аргументов, могут иметь произвольные выражения. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/syntax/) diff --git a/docs/ru/query_language/table_functions/file.md b/docs/ru/query_language/table_functions/file.md index 9e029a6b729..f0fe3d4687b 100644 --- a/docs/ru/query_language/table_functions/file.md +++ b/docs/ru/query_language/table_functions/file.md @@ -34,7 +34,7 @@ $ cat /var/lib/clickhouse/user_files/test.csv Таблица из `test.csv` и выборка первых двух строк из неё: -```sql +``` sql SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') LIMIT 2 @@ -45,3 +45,5 @@ LIMIT 2 │ 3 │ 2 │ 1 │ └─────────┴─────────┴─────────┘ ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/table_functions/file/) diff --git a/docs/ru/query_language/table_functions/index.md b/docs/ru/query_language/table_functions/index.md index 6649ae3978f..704c9fa7123 100644 --- a/docs/ru/query_language/table_functions/index.md +++ b/docs/ru/query_language/table_functions/index.md @@ -3,3 +3,5 @@ Табличные функции могут указываться в секции FROM вместо имени БД и таблицы. Табличные функции можно использовать только если не выставлена настройка readonly. Табличные функции не имеют отношения к другим функциям. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/table_functions/) diff --git a/docs/ru/query_language/table_functions/jdbc.md b/docs/ru/query_language/table_functions/jdbc.md index 8a7ca2c4acd..77b1da780ab 100644 --- a/docs/ru/query_language/table_functions/jdbc.md +++ b/docs/ru/query_language/table_functions/jdbc.md @@ -5,19 +5,21 @@ `jdbc(jdbc_connection_uri, schema, table)` - возвращает таблицу, соединение с которой происходит через JDBC-драйвер. Для работы этой табличной функциии требуется отдельно запускать приложение clickhouse-jdbc-bridge. -В отличии от табличной функции `odbc`, данная функция поддерживает Nullable типы (на основании DDL таблицы к которой происходит запрос). +Данная функция поддерживает Nullable типы (на основании DDL таблицы к которой происходит запрос). **Пример** -```sql +``` sql SELECT * FROM jdbc('jdbc:mysql://localhost:3306/?user=root&password=root', 'schema', 'table') ``` -```sql +``` sql SELECT * FROM jdbc('mysql://localhost:3306/?user=root&password=root', 'schema', 'table') ``` -```sql +``` sql SELECT * FROM jdbc('datasource://mysql-local', 'schema', 'table') -``` \ No newline at end of file +``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/table_functions/jdbc/) diff --git a/docs/ru/query_language/table_functions/merge.md b/docs/ru/query_language/table_functions/merge.md index 092c9243fc5..04a3ba98223 100644 --- a/docs/ru/query_language/table_functions/merge.md +++ b/docs/ru/query_language/table_functions/merge.md @@ -3,3 +3,5 @@ `merge(db_name, 'tables_regexp')` - создаёт временную таблицу типа Merge. Подробнее смотрите раздел "Движки таблиц, Merge". Структура таблицы берётся из первой попавшейся таблицы, подходящей под регулярное выражение. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/table_functions/merge/) diff --git a/docs/ru/query_language/table_functions/numbers.md b/docs/ru/query_language/table_functions/numbers.md index 8ff5ae3ac97..a5e4ba58ed9 100644 --- a/docs/ru/query_language/table_functions/numbers.md +++ b/docs/ru/query_language/table_functions/numbers.md @@ -7,13 +7,15 @@ Следующие запросы эквивалентны: -```sql +``` sql SELECT * FROM numbers(10); SELECT * FROM numbers(0,10); SELECT * FROM system.numbers LIMIT 10; ``` Примеры: -```sql +``` sql -- генерация последовательности всех дат от 2010-01-01 до 2010-12-31 select toDate('2010-01-01') + number as d FROM numbers(365); ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/table_functions/numbers/) diff --git a/docs/ru/query_language/table_functions/remote.md b/docs/ru/query_language/table_functions/remote.md index a2366607fe7..acd623a583e 100644 --- a/docs/ru/query_language/table_functions/remote.md +++ b/docs/ru/query_language/table_functions/remote.md @@ -6,7 +6,7 @@ Сигнатуры: -```sql +``` sql remote('addresses_expr', db, table[, 'user'[, 'password']]) remote('addresses_expr', db.table[, 'user'[, 'password']]) ``` @@ -18,7 +18,7 @@ remote('addresses_expr', db.table[, 'user'[, 'password']]) Примеры: -```text +``` example01-01-1 example01-01-1:9000 localhost @@ -31,19 +31,19 @@ localhost Пример: -```text +``` example01-01-1,example01-02-1 ``` Часть выражения может быть указана в фигурных скобках. Предыдущий пример может быть записан следующим образом: -```text +``` example01-0{1,2}-1 ``` В фигурных скобках может быть указан диапазон (неотрицательных целых) чисел через две точки. В этом случае, диапазон раскрывается в множество значений, генерирующих адреса шардов. Если запись первого числа начинается с нуля, то значения формируются с таким же выравниванием нулями. Предыдущий пример может быть записан следующим образом: -```text +``` example01-{01..02}-1 ``` @@ -53,7 +53,7 @@ example01-{01..02}-1 Пример: -```text +``` example01-{01..02}-{1|2} ``` @@ -72,3 +72,5 @@ example01-{01..02}-{1|2} Если пользователь не задан,то используется `default`. Если пароль не задан, то используется пустой пароль. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/table_functions/remote/) diff --git a/docs/ru/query_language/table_functions/url.md b/docs/ru/query_language/table_functions/url.md index 7c5068b3caa..fccaf1fe11e 100644 --- a/docs/ru/query_language/table_functions/url.md +++ b/docs/ru/query_language/table_functions/url.md @@ -14,7 +14,9 @@ structure - структура таблицы в форме `'UserID UInt64, Nam **Пример** -```sql +``` sql -- получение 3-х строк таблицы, состоящей из двух колонк типа String и UInt32 от сервера, отдающего данные в формате CSV SELECT * FROM url('http://127.0.0.1:12345/', CSV, 'column1 String, column2 UInt32') LIMIT 3 ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/table_functions/url/) diff --git a/docs/ru/roadmap.md b/docs/ru/roadmap.md index afd492c4b60..5418cf8a8b1 100644 --- a/docs/ru/roadmap.md +++ b/docs/ru/roadmap.md @@ -1,20 +1,14 @@ # Roadmap -## Q3 2018 - -- `ALTER UPDATE` для массового изменения данных с использованием подхода, аналогичного `ALTER DELETE` -- Добавление Protobuf и Parquet к ассортименту поддерживаемых форматов ввода-вывода -- Улучшением совместимости с Tableau и другими инструментами бизнес-аналитики - ## Q4 2018 - Соответствующий SQL стандарту синтаксис JOIN: - Несколько `JOIN`ов в одном `SELECT` - - Указание связи между таблицами через `ON` - - Возможность сослаться на имя таблицы вместо обязательного использования подзапроса - Улучшения в исполнении JOIN: - Распределённый JOIN, не ограниченный оперативной памятью - - Перенос зависящих только от одной стороны предикатов сквозь JOIN +- Добавление Protobuf и Parquet к ассортименту поддерживаемых форматов ввода-вывода - Пулы ресурсов для более точного распределения мощностей кластера между его пользователями + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/roadmap/) diff --git a/docs/ru/security_changelog.md b/docs/ru/security_changelog.md index de3e3b2cbd9..7bf286a4e45 100644 --- a/docs/ru/security_changelog.md +++ b/docs/ru/security_changelog.md @@ -18,4 +18,5 @@ Некоррректная конфигурация в deb пакете могла привести к неавторизованному доступу к базе данных. -Обнаружено благодаря: the UK's National Cyber Security Centre (NCSC) \ No newline at end of file +Обнаружено благодаря: the UK's National Cyber Security Centre (NCSC) +[Оригинальная статья](https://clickhouse.yandex/docs/ru/security_changelog/) diff --git a/docs/toc_en.yml b/docs/toc_en.yml index 8b191b165f9..e6e63095dd8 100644 --- a/docs/toc_en.yml +++ b/docs/toc_en.yml @@ -100,6 +100,7 @@ nav: - 'numbers': 'query_language/table_functions/numbers.md' - 'remote': 'query_language/table_functions/remote.md' - 'url': 'query_language/table_functions/url.md' + - 'jdbc': 'query_language/table_functions/jdbc.md' - 'Dictionaries': - 'Introduction': 'query_language/dicts/index.md' - 'External dictionaries': diff --git a/docs/toc_fa.yml b/docs/toc_fa.yml index 9970325f495..780d9747931 100644 --- a/docs/toc_fa.yml +++ b/docs/toc_fa.yml @@ -97,6 +97,7 @@ nav: - 'numbers': 'query_language/table_functions/numbers.md' - 'remote': 'query_language/table_functions/remote.md' - 'url': 'query_language/table_functions/url.md' + - 'jdbc': 'query_language/table_functions/jdbc.md' - 'Dictionaries': - 'Introduction': 'query_language/dicts/index.md' - 'External dictionaries': diff --git a/docs/toc_ru.yml b/docs/toc_ru.yml index 8272e1b1186..9c5ee2db74f 100644 --- a/docs/toc_ru.yml +++ b/docs/toc_ru.yml @@ -101,6 +101,7 @@ nav: - 'numbers': 'query_language/table_functions/numbers.md' - 'remote': 'query_language/table_functions/remote.md' - 'url': 'query_language/table_functions/url.md' + - 'jdbc': 'query_language/table_functions/jdbc.md' - 'Словари': - 'Введение': 'query_language/dicts/index.md' - 'Внешние словари': diff --git a/docs/toc_zh.yml b/docs/toc_zh.yml index 8b191b165f9..7a3bf5de7f0 100644 --- a/docs/toc_zh.yml +++ b/docs/toc_zh.yml @@ -1,15 +1,15 @@ nav: -- 'Introduction': - - 'Overview': 'index.md' - - 'Distinctive features of ClickHouse': 'introduction/distinctive_features.md' - - 'ClickHouse features that can be considered disadvantages': 'introduction/features_considered_disadvantages.md' - - 'Performance': 'introduction/performance.md' - - 'The Yandex.Metrica task': 'introduction/ya_metrika_task.md' +- '介绍': + - '概貌': 'index.md' + - 'ClickHouse的独特功能': 'introduction/distinctive_features.md' + - 'ClickHouse功能可被视为缺点': 'introduction/features_considered_disadvantages.md' + - '性能': 'introduction/performance.md' + - 'Yandex.Metrica使用案例': 'introduction/ya_metrika_task.md' -- 'Getting started': - - 'Deploying and running': 'getting_started/index.md' - - 'Example datasets': +- '起步': + - '部署运行': 'getting_started/index.md' + - '示例数据集': - 'OnTime': 'getting_started/example_datasets/ontime.md' - 'New York Taxi data': 'getting_started/example_datasets/nyc_taxi.md' - 'AMPLab Big Data Benchmark': 'getting_started/example_datasets/amplab_benchmark.md' @@ -17,18 +17,18 @@ nav: - 'Terabyte click logs from Criteo': 'getting_started/example_datasets/criteo.md' - 'Star Schema Benchmark': 'getting_started/example_datasets/star_schema.md' -- 'Interfaces': - - 'Introduction': 'interfaces/index.md' - - 'Command-line client': 'interfaces/cli.md' - - 'HTTP interface': 'interfaces/http_interface.md' - - 'JDBC driver': 'interfaces/jdbc.md' - - 'Native interface (TCP)': 'interfaces/tcp.md' - - 'Libraries from third-party developers': 'interfaces/third-party_client_libraries.md' - - 'Visual interfaces from third-party developers': 'interfaces/third-party_gui.md' - - 'Input and output formats': 'interfaces/formats.md' +- '客户端': + - '介绍': 'interfaces/index.md' + - '命令行客户端接口': 'interfaces/cli.md' + - 'HTTP 客户端接口': 'interfaces/http_interface.md' + - 'JDBC 驱动': 'interfaces/jdbc.md' + - '原生客户端接口 (TCP)': 'interfaces/tcp.md' + - '第三方开发的库': 'interfaces/third-party_client_libraries.md' + - '第三方开发的可视化界面': 'interfaces/third-party_gui.md' + - '输入输出格式': 'interfaces/formats.md' -- 'Data types': - - 'Introduction': 'data_types/index.md' +- '数据类型': + - '介绍': 'data_types/index.md' - 'UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64': 'data_types/int_uint.md' - 'Float32, Float64': 'data_types/float.md' - 'Decimal': 'data_types/decimal.md' @@ -51,7 +51,7 @@ nav: - 'Set': 'data_types/special_data_types/set.md' - 'Nothing': 'data_types/special_data_types/nothing.md' -- 'SQL reference': +- 'SQL语法': - 'hidden': 'query_language/index.md' - 'SELECT': 'query_language/select.md' - 'INSERT INTO': 'query_language/insert_into.md' @@ -100,6 +100,7 @@ nav: - 'numbers': 'query_language/table_functions/numbers.md' - 'remote': 'query_language/table_functions/remote.md' - 'url': 'query_language/table_functions/url.md' + - 'jdbc': 'query_language/table_functions/jdbc.md' - 'Dictionaries': - 'Introduction': 'query_language/dicts/index.md' - 'External dictionaries': @@ -113,7 +114,7 @@ nav: - 'Operators': 'query_language/operators.md' - 'General syntax': 'query_language/syntax.md' -- 'Operations': +- '运维': - 'hidden': 'operations/index.md' - 'Table engines': - 'Introduction': 'operations/table_engines/index.md' @@ -165,10 +166,10 @@ nav: - 'clickhouse-copier': 'operations/utils/clickhouse-copier.md' - 'clickhouse-local': 'operations/utils/clickhouse-local.md' -- 'F.A.Q.': - - 'General questions': 'faq/general.md' +- '常见问题': + - '一般的问题': 'faq/general.md' -- 'Development': +- '开发者指南': - 'hidden': 'development/index.md' - 'Overview of ClickHouse architecture': 'development/architecture.md' - 'How to build ClickHouse on Linux': 'development/build.md' @@ -176,7 +177,7 @@ nav: - 'How to write C++ code': 'development/style.md' - 'How to run ClickHouse tests': 'development/tests.md' -- 'What''s new': - - 'Roadmap': 'roadmap.md' - - 'Changelog': 'changelog.md' - - 'Security changelog': 'security_changelog.md' +- '新功能特性': + - '路线图': 'roadmap.md' + - '更新日志': 'changelog.md' + - '安全更改日志': 'security_changelog.md' diff --git a/docs/tools/README.md b/docs/tools/README.md index 1017aa93e9b..a0b853c0308 100644 --- a/docs/tools/README.md +++ b/docs/tools/README.md @@ -16,7 +16,7 @@ Usually those have some way to preview how Markdown will look like, which allows It'll take some effort to go through, but the result will be very close to production documentation. -For the first time you'll need to set up virtualenv: +For the first time you'll need to install [wkhtmltopdf](https://wkhtmltopdf.org/) and set up virtualenv: ``` bash $ cd ClickHouse/docs/tools diff --git a/docs/tools/build.py b/docs/tools/build.py index f3b05be97d4..96a2a74ea70 100755 --- a/docs/tools/build.py +++ b/docs/tools/build.py @@ -8,8 +8,13 @@ import datetime import logging import os import shutil +import subprocess import sys import tempfile +import time + +import markdown.extensions +import markdown.util from mkdocs import config from mkdocs import exceptions @@ -34,6 +39,17 @@ def autoremoved_file(path): finally: os.unlink(path) +class ClickHouseMarkdown(markdown.extensions.Extension): + class ClickHousePreprocessor(markdown.util.Processor): + def run(self, lines): + for line in lines: + if '' not in line: + yield line + + def extendMarkdown(self, md): + md.preprocessors.register(self.ClickHousePreprocessor(), 'clickhouse_preprocessor', 31) + +markdown.extensions.ClickHouseMarkdown = ClickHouseMarkdown def build_for_lang(lang, args): logging.info('Building %s docs' % lang) @@ -61,10 +77,8 @@ def build_for_lang(lang, args): 'static_templates': ['404.html'], 'extra': { 'single_page': False, - 'opposite_lang': 'ru' if lang == 'en' else 'en', - 'now': datetime.datetime.now() # TODO better way to avoid caching + 'now': int(time.mktime(datetime.datetime.now().timetuple())) # TODO better way to avoid caching } - } site_names = { @@ -89,6 +103,7 @@ def build_for_lang(lang, args): edit_uri='edit/master/docs/%s' % lang, extra_css=['assets/stylesheets/custom.css'], markdown_extensions=[ + 'clickhouse', 'admonition', 'attr_list', 'codehilite', @@ -134,8 +149,7 @@ def build_single_page_version(lang, args, cfg): 'docs_dir': docs_temp_lang, 'site_dir': site_temp, 'extra': { - 'single_page': True, - 'opposite_lang': 'en' if lang == 'ru' else 'ru' + 'single_page': True }, 'nav': [ {cfg.data.get('site_name'): 'single.md'} @@ -154,6 +168,12 @@ def build_single_page_version(lang, args, cfg): single_page_output_path ) + single_page_index_html = os.path.abspath(os.path.join(single_page_output_path, 'index.html')) + single_page_pdf = single_page_index_html.replace('index.html', 'clickhouse_%s.pdf' % lang) + create_pdf_command = ['wkhtmltopdf', '--print-media-type', single_page_index_html, single_page_pdf] + logging.debug(' '.join(create_pdf_command)) + subprocess.check_call(' '.join(create_pdf_command), shell=True) + def build_redirects(args): lang_re_fragment = args.lang.replace(',', '|') diff --git a/docs/tools/mdx_clickhouse.py b/docs/tools/mdx_clickhouse.py new file mode 100755 index 00000000000..c38d6ddcf16 --- /dev/null +++ b/docs/tools/mdx_clickhouse.py @@ -0,0 +1,22 @@ +#!/usr/bin/env python +# -*- coding: utf-8 -*- +from __future__ import unicode_literals + +import markdown.extensions +import markdown.util + + +class ClickHousePreprocessor(markdown.util.Processor): + def run(self, lines): + for line in lines: + if '' not in line: + yield line + +class ClickHouseMarkdown(markdown.extensions.Extension): + + def extendMarkdown(self, md, md_globals): + md.preprocessors['clickhouse'] = ClickHousePreprocessor() + + +def makeExtension(**kwargs): + return ClickHouseMarkdown(**kwargs) diff --git a/docs/tools/mkdocs-material-theme/assets/flags/en.svg b/docs/tools/mkdocs-material-theme/assets/flags/en.svg new file mode 100644 index 00000000000..7cabb5e9071 --- /dev/null +++ b/docs/tools/mkdocs-material-theme/assets/flags/en.svg @@ -0,0 +1,10 @@ + + + + + + + + + + diff --git a/docs/tools/mkdocs-material-theme/assets/flags/fa.svg b/docs/tools/mkdocs-material-theme/assets/flags/fa.svg new file mode 100644 index 00000000000..de467f133d9 --- /dev/null +++ b/docs/tools/mkdocs-material-theme/assets/flags/fa.svg @@ -0,0 +1,31 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/docs/tools/mkdocs-material-theme/assets/flags/ru.svg b/docs/tools/mkdocs-material-theme/assets/flags/ru.svg new file mode 100644 index 00000000000..76e92a1015e --- /dev/null +++ b/docs/tools/mkdocs-material-theme/assets/flags/ru.svg @@ -0,0 +1,5 @@ + + + + + diff --git a/docs/tools/mkdocs-material-theme/assets/flags/zh.svg b/docs/tools/mkdocs-material-theme/assets/flags/zh.svg new file mode 100644 index 00000000000..7681b350400 --- /dev/null +++ b/docs/tools/mkdocs-material-theme/assets/flags/zh.svg @@ -0,0 +1,11 @@ + + + + + + + + + + + diff --git a/docs/tools/mkdocs-material-theme/assets/javascripts/lunr/lunr.fa.js b/docs/tools/mkdocs-material-theme/assets/javascripts/lunr/lunr.fa.js new file mode 100644 index 00000000000..e69de29bb2d diff --git a/docs/tools/mkdocs-material-theme/assets/javascripts/lunr/lunr.zh.js b/docs/tools/mkdocs-material-theme/assets/javascripts/lunr/lunr.zh.js new file mode 100644 index 00000000000..1125b3efa89 --- /dev/null +++ b/docs/tools/mkdocs-material-theme/assets/javascripts/lunr/lunr.zh.js @@ -0,0 +1,146 @@ +i/** + * lunr对中文分词的支持 + */ +; +(function(root, factory) { + if (typeof define === 'function' && define.amd) { + // AMD. Register as an anonymous module. + define(factory) + } else if (typeof exports === 'object') { + /** + * Node. Does not work with strict CommonJS, but + * only CommonJS-like environments that support module.exports, + * like Node. + */ + module.exports = factory() + } else { + // Browser globals (root is window) + factory()(root.lunr); + } +}(this, function() { + /** + * Just return a value to define the module export. + * This example returns an object, but the module + * can return a function as the exported value. + */ + return function(lunr) { + + /* + Thai tokenization is the same to Japanense, which does not take into account spaces. + So, it uses the same logic to assign tokenization function due to different Lunr versions. + */ + var isLunr2 = lunr.version[0] == "2"; + + /* register specific locale function */ + lunr.zhcn = function() { + this.pipeline.reset(); + this.pipeline.add( + lunr.zhcn.trimmer, + lunr.zhcn.stopWordFilter, + lunr.zhcn.stemmer + ); + + if (isLunr2) { // for lunr version 2.0.0 + this.tokenizer = lunr.zhcn.tokenizer; + } else { + if (lunr.tokenizer) { // for lunr version 0.6.0 + lunr.tokenizer = lunr.zhcn.tokenizer; + } + if (this.tokenizerFn) { // for lunr version 0.7.0 -> 1.0.0 + this.tokenizerFn = lunr.zhcn.tokenizer; + } + } + }; + + + var segmenter = new lunr.TinySegmenter(); + + lunr.zhcn.tokenizer = function(obj) { + var i; + var str; + var len; + var segs; + var tokens; + var char; + var sliceLength; + var sliceStart; + var sliceEnd; + var segStart; + + if (!arguments.length || obj == null || obj == undefined) + return []; + + if (Array.isArray(obj)) { + return obj.map( + function(t) { + return isLunr2 ? new lunr.Token(t.toLowerCase()) : t.toLowerCase(); + } + ); + } + + str = obj.toString().toLowerCase().replace(/^\s+/, ''); + for (i = str.length - 1; i >= 0; i--) { + if (/\S/.test(str.charAt(i))) { + str = str.substring(0, i + 1); + break; + } + } + + tokens = []; + len = str.length; + for (sliceEnd = 0, sliceStart = 0; sliceEnd <= len; sliceEnd++) { + char = str.charAt(sliceEnd); + sliceLength = sliceEnd - sliceStart; + + if ((char.match(/\s/) || sliceEnd == len)) { + if (sliceLength > 0) { + segs = segmenter.segment(str.slice(sliceStart, sliceEnd)).filter( + function(token) { + return !!token; + } + ); + + segStart = sliceStart; + for (i = 0; i < segs.length; i++) { + if (isLunr2) { + tokens.push( + new lunr.Token( + segs[i], { + position: [segStart, segs[i].length], + index: tokens.length + } + ) + ); + } else { + tokens.push(segs[i]); + } + segStart += segs[i].length; + } + } + + sliceStart = sliceEnd + 1; + } + } + + return tokens; + } + + lunr.zhcn.stemmer = (function(){ + return function(word) { + return word; + } + })(); + + lunr.Pipeline.registerFunction(lunr.zhcn.stemmer, 'stemmer-zhcn'); + + /* lunr trimmer function */ + lunr.zhcn.wordCharacters = "一二三四五六七八九十百千万億兆一-龠々〆ヵヶぁ-んァ-ヴーア-ン゙a-zA-Za-zA-Z0-90-9"; + lunr.zhcn.trimmer = lunr.trimmerSupport.generateTrimmer(lunr.zhcn.wordCharacters); + lunr.Pipeline.registerFunction(lunr.zhcn.trimmer, 'trimmer-zhcn'); + + + /* lunr stop word filter. see https://www.ranks.nl/stopwords/chinese-stopwords */ + lunr.zhcn.stopWordFilter = lunr.generateStopWordFilter('的 一 不 在 人 有 是 为 以 于 上 他 而 后 之 来 及 了 因 下 可 到 由 这 与 也 此 但 并 个 其 已 无 小 我 们 起 最 再 今 去 好 只 又 或 很 亦 某 把 那 你 乃 它 吧 被 比 别 趁 当 从 到 得 打 凡 儿 尔 该 各 给 跟 和 何 还 即 几 既 看 据 距 靠 啦 了 另 么 每 们 嘛 拿 哪 那 您 凭 且 却 让 仍 啥 如 若 使 谁 虽 随 同 所 她 哇 嗡 往 哪 些 向 沿 哟 用 于 咱 则 怎 曾 至 致 着 诸 自'.split(' ')); + lunr.Pipeline.registerFunction(lunr.zhcn.stopWordFilter, 'stopWordFilter-zhcn'); + }; +})) diff --git a/docs/tools/mkdocs-material-theme/assets/stylesheets/custom.css b/docs/tools/mkdocs-material-theme/assets/stylesheets/custom.css index 2fcf11ce447..0024345554b 100644 --- a/docs/tools/mkdocs-material-theme/assets/stylesheets/custom.css +++ b/docs/tools/mkdocs-material-theme/assets/stylesheets/custom.css @@ -1,70 +1,74 @@ @font-face { - font-family: 'Yandex Sans Text Web'; - src: url(https://yastatic.net/adv-www/_/yy5JveR58JFkc97waf-xp0i6_jM.eot); - src: url(https://yastatic.net/adv-www/_/yy5JveR58JFkc97waf-xp0i6_jM.eot?#iefix) format('embedded-opentype'), - url(https://yastatic.net/adv-www/_/CYblzLEXzCqQIvrYs7QKQe2omRk.woff2) format('woff2'), - url(https://yastatic.net/adv-www/_/pUcnOdRwl83MvPPzrNomhyletnA.woff) format('woff'), - url(https://yastatic.net/adv-www/_/vNFEmXOcGYKJ4AAidUprHWoXrLU.ttf) format('truetype'), - url(https://yastatic.net/adv-www/_/0w7OcWZM_QLP8x-LQUXFOgXO6dE.svg#YandexSansTextWeb-Bold) format('svg'); - font-weight: 700; - font-style: normal; - font-stretch: normal + font-family: 'Yandex Sans Text Web'; + src: url(https://yastatic.net/adv-www/_/yy5JveR58JFkc97waf-xp0i6_jM.eot); + src: url(https://yastatic.net/adv-www/_/yy5JveR58JFkc97waf-xp0i6_jM.eot?#iefix) format('embedded-opentype'), + url(https://yastatic.net/adv-www/_/CYblzLEXzCqQIvrYs7QKQe2omRk.woff2) format('woff2'), + url(https://yastatic.net/adv-www/_/pUcnOdRwl83MvPPzrNomhyletnA.woff) format('woff'), + url(https://yastatic.net/adv-www/_/vNFEmXOcGYKJ4AAidUprHWoXrLU.ttf) format('truetype'), + url(https://yastatic.net/adv-www/_/0w7OcWZM_QLP8x-LQUXFOgXO6dE.svg#YandexSansTextWeb-Bold) format('svg'); + font-weight: 700; + font-style: normal; + font-stretch: normal } @font-face { - font-family: 'Yandex Sans Text Web'; - src: url(https://yastatic.net/adv-www/_/LI6l3L2RqcgxBe2pXmuUha37czQ.eot); - src: url(https://yastatic.net/adv-www/_/LI6l3L2RqcgxBe2pXmuUha37czQ.eot?#iefix) format('embedded-opentype'), - url(https://yastatic.net/adv-www/_/z3MYElcut0R2MF_Iw1RDNrstgYs.woff2) format('woff2'), - url(https://yastatic.net/adv-www/_/1jvKJ_-hCXl3s7gmFl-y_-UHTaI.woff) format('woff'), - url(https://yastatic.net/adv-www/_/9nzjfpCR2QHvK1EzHpDEIoVFGuY.ttf) format('truetype'), - url(https://yastatic.net/adv-www/_/gwyBTpxSwkFCF1looxqs6JokKls.svg#YandexSansTextWeb-Regular) format('svg'); - font-weight: 400; - font-style: normal; - font-stretch: normal + font-family: 'Yandex Sans Text Web'; + src: url(https://yastatic.net/adv-www/_/LI6l3L2RqcgxBe2pXmuUha37czQ.eot); + src: url(https://yastatic.net/adv-www/_/LI6l3L2RqcgxBe2pXmuUha37czQ.eot?#iefix) format('embedded-opentype'), + url(https://yastatic.net/adv-www/_/z3MYElcut0R2MF_Iw1RDNrstgYs.woff2) format('woff2'), + url(https://yastatic.net/adv-www/_/1jvKJ_-hCXl3s7gmFl-y_-UHTaI.woff) format('woff'), + url(https://yastatic.net/adv-www/_/9nzjfpCR2QHvK1EzHpDEIoVFGuY.ttf) format('truetype'), + url(https://yastatic.net/adv-www/_/gwyBTpxSwkFCF1looxqs6JokKls.svg#YandexSansTextWeb-Regular) format('svg'); + font-weight: 400; + font-style: normal; + font-stretch: normal } @font-face { - font-family: 'Yandex Sans Text Web'; - src: url(https://yastatic.net/adv-www/_/ayAFYoY8swgBLhq_I56tKj2JftU.eot); - src: url(https://yastatic.net/adv-www/_/ayAFYoY8swgBLhq_I56tKj2JftU.eot?#iefix) format('embedded-opentype'), - url(https://yastatic.net/adv-www/_/lGQcYklLVV0hyvz1HFmFsUTj8_0.woff2) format('woff2'), - url(https://yastatic.net/adv-www/_/f0AAJ9GJ4iiwEmhG-7PWMHk6vUY.woff) format('woff'), - url(https://yastatic.net/adv-www/_/4UDe4nlVvgEJ-VmLWNVq3SxCsA.ttf) format('truetype'), - url(https://yastatic.net/adv-www/_/EKLr1STNokPqxLAQa_RyN82pL98.svg#YandexSansTextWeb-Light) format('svg'); - font-weight: 300; - font-style: normal; - font-stretch: normal + font-family: 'Yandex Sans Text Web'; + src: url(https://yastatic.net/adv-www/_/ayAFYoY8swgBLhq_I56tKj2JftU.eot); + src: url(https://yastatic.net/adv-www/_/ayAFYoY8swgBLhq_I56tKj2JftU.eot?#iefix) format('embedded-opentype'), + url(https://yastatic.net/adv-www/_/lGQcYklLVV0hyvz1HFmFsUTj8_0.woff2) format('woff2'), + url(https://yastatic.net/adv-www/_/f0AAJ9GJ4iiwEmhG-7PWMHk6vUY.woff) format('woff'), + url(https://yastatic.net/adv-www/_/4UDe4nlVvgEJ-VmLWNVq3SxCsA.ttf) format('truetype'), + url(https://yastatic.net/adv-www/_/EKLr1STNokPqxLAQa_RyN82pL98.svg#YandexSansTextWeb-Light) format('svg'); + font-weight: 300; + font-style: normal; + font-stretch: normal } @font-face { - font-family: 'Yandex Sans Display Web'; - src: url(https://yastatic.net/adv-www/_/H63jN0veW07XQUIA2317lr9UIm8.eot); - src: url(https://yastatic.net/adv-www/_/H63jN0veW07XQUIA2317lr9UIm8.eot?#iefix) format('embedded-opentype'), - url(https://yastatic.net/adv-www/_/sUYVCPUAQE7ExrvMS7FoISoO83s.woff2) format('woff2'), - url(https://yastatic.net/adv-www/_/v2Sve_obH3rKm6rKrtSQpf-eB7U.woff) format('woff'), - url(https://yastatic.net/adv-www/_/PzD8hWLMunow5i3RfJ6WQJAL7aI.ttf) format('truetype'), - url(https://yastatic.net/adv-www/_/lF_KG5g4tpQNlYIgA0e77fBSZ5s.svg#YandexSansDisplayWeb-Regular) format('svg'); - font-weight: 400; - font-style: normal; - font-stretch: normal + font-family: 'Yandex Sans Display Web'; + src: url(https://yastatic.net/adv-www/_/H63jN0veW07XQUIA2317lr9UIm8.eot); + src: url(https://yastatic.net/adv-www/_/H63jN0veW07XQUIA2317lr9UIm8.eot?#iefix) format('embedded-opentype'), + url(https://yastatic.net/adv-www/_/sUYVCPUAQE7ExrvMS7FoISoO83s.woff2) format('woff2'), + url(https://yastatic.net/adv-www/_/v2Sve_obH3rKm6rKrtSQpf-eB7U.woff) format('woff'), + url(https://yastatic.net/adv-www/_/PzD8hWLMunow5i3RfJ6WQJAL7aI.ttf) format('truetype'), + url(https://yastatic.net/adv-www/_/lF_KG5g4tpQNlYIgA0e77fBSZ5s.svg#YandexSansDisplayWeb-Regular) format('svg'); + font-weight: 400; + font-style: normal; + font-stretch: normal } @font-face { - font-family: 'Yandex Sans Display Web'; - src: url(https://yastatic.net/adv-www/_/g8_MyyKVquSZ3xEL6tarK__V9Vw.eot); - src: url(https://yastatic.net/adv-www/_/g8_MyyKVquSZ3xEL6tarK__V9Vw.eot?#iefix) format('embedded-opentype'), - url(https://yastatic.net/adv-www/_/LGiRvlfqQHlWR9YKLhsw5e7KGNA.woff2) format('woff2'), - url(https://yastatic.net/adv-www/_/40vXwNl4eYYMgteIVgLP49dwmfc.woff) format('woff'), - url(https://yastatic.net/adv-www/_/X6zG5x_wO8-AtwJ-vDLJcKC5228.ttf) format('truetype'), - url(https://yastatic.net/adv-www/_/ZKhaR0m08c8CRRL77GtFKoHcLYA.svg#YandexSansDisplayWeb-Light) format('svg'); - font-weight: 300; - font-style: normal; - font-stretch: normal + font-family: 'Yandex Sans Display Web'; + src: url(https://yastatic.net/adv-www/_/g8_MyyKVquSZ3xEL6tarK__V9Vw.eot); + src: url(https://yastatic.net/adv-www/_/g8_MyyKVquSZ3xEL6tarK__V9Vw.eot?#iefix) format('embedded-opentype'), + url(https://yastatic.net/adv-www/_/LGiRvlfqQHlWR9YKLhsw5e7KGNA.woff2) format('woff2'), + url(https://yastatic.net/adv-www/_/40vXwNl4eYYMgteIVgLP49dwmfc.woff) format('woff'), + url(https://yastatic.net/adv-www/_/X6zG5x_wO8-AtwJ-vDLJcKC5228.ttf) format('truetype'), + url(https://yastatic.net/adv-www/_/ZKhaR0m08c8CRRL77GtFKoHcLYA.svg#YandexSansDisplayWeb-Light) format('svg'); + font-weight: 300; + font-style: normal; + font-stretch: normal } body { - font: 300 14pt/200% 'Yandex Sans Text Web', Arial, sans-serif; + font: 400 14pt/200% 'Yandex Sans Text Web', Arial, sans-serif; +} + +body.md-lang-zh { + font: 400 14pt/200% 'Yandex Sans Text Web', -apple-system, BlinkMacSystemFont, "Segoe UI", Helvetica, Arial, sans-serif, "Apple Color Emoji", "Segoe UI Emoji", "Segoe UI Symbol"; } a:link, a:visited { @@ -73,7 +77,7 @@ a:link, a:visited { } .md-nav__link { - color: #000 !important; + color: #000 !important; } .md-nav__link[data-md-state='blur'] { @@ -100,11 +104,11 @@ a:hover, a:active { h1, h2, h3, .md-logo { font-family: 'Yandex Sans Display Web', Arial, sans-serif; - color: #000 !important; + color: #000 !important; } .md-logo { - padding: 0; + padding: 0; } .md-header { @@ -130,10 +134,34 @@ h1, h2, h3, .md-logo { } #md-extra-nav { - background: #efefef; - padding-top: 0.5rem; + background: #efefef; + padding: 0.5rem 0; } .grey { - color: #666; + color: #666; +} + +.md-alt-lang:hover { + text-decoration: none; +} + +.md-alt-lang>svg { + width: 36px; + height: 24px; + margin: 1.3rem 0; + filter: brightness(96%) grayscale(90%); +} + +.md-alt-lang:hover>svg { + filter: brightness(96%) grayscale(5%); +} + +.md-current-lang>svg { + filter: brightness(96%) grayscale(0%) !important; +} +@media only screen and (min-width: 60em) { + #md-sidebar-flags { + display: none; + } } diff --git a/docs/tools/mkdocs-material-theme/base.html b/docs/tools/mkdocs-material-theme/base.html index 53e855a19ed..b3d3ae2d0e4 100644 --- a/docs/tools/mkdocs-material-theme/base.html +++ b/docs/tools/mkdocs-material-theme/base.html @@ -63,7 +63,7 @@ {% endblock %} {% for path in extra_css %} - + {% endfor %} {% block extrahead %}{% endblock %} @@ -71,9 +71,9 @@ {% if palette.primary or palette.accent %} {% set primary = palette.primary | replace(" ", "-") | lower %} {% set accent = palette.accent | replace(" ", "-") | lower %} - + {% else %} - + {% endif %} @@ -231,6 +231,20 @@ base: "{{ base_url }}" } }); + var beforePrint = function() { + var details = document.getElementsByTagName("details"); + for (var i = 0; i < details.length; ++i) { + details[i].open = 1; + } + }; + if (window.matchMedia) { + window.matchMedia('print').addListener(function(q) { + if (q.matches) { + beforePrint(); + } + }); + } + window.onbeforeprint = beforePrint; }); diff --git a/docs/tools/mkdocs-material-theme/partials/flags.html b/docs/tools/mkdocs-material-theme/partials/flags.html new file mode 100644 index 00000000000..ae0200b602b --- /dev/null +++ b/docs/tools/mkdocs-material-theme/partials/flags.html @@ -0,0 +1,9 @@ +{% set alt_langs = [['en', 'English'], ['ru', 'Russian'], ['zh', 'Chinese'], ['fa', 'Farsi']] %} +{% for alt_lang, alt_title in alt_langs %} + + {% include "assets/flags/" + alt_lang + ".svg" %} + +{% endfor %} + diff --git a/docs/tools/mkdocs-material-theme/partials/header.html b/docs/tools/mkdocs-material-theme/partials/header.html index 7adbbeb8d7e..5c70a96ac41 100644 --- a/docs/tools/mkdocs-material-theme/partials/header.html +++ b/docs/tools/mkdocs-material-theme/partials/header.html @@ -43,24 +43,7 @@ {% if page %}
- {% if config.theme.extra.opposite_lang == 'en' %} - - - - - - - - - {% else %} - - - - - - - - {% endif %} + {% include "partials/flags.html" %}
{% endif %} diff --git a/docs/tools/mkdocs-material-theme/partials/nav.html b/docs/tools/mkdocs-material-theme/partials/nav.html index 66366e6ad40..14f5d507eea 100644 --- a/docs/tools/mkdocs-material-theme/partials/nav.html +++ b/docs/tools/mkdocs-material-theme/partials/nav.html @@ -29,6 +29,13 @@ {% endif %} {% endif %} +
  • + {% if config.theme.language == 'ru' %} + PDF версия + {% else %} + PDF version + {% endif %} +
  • {% if config.repo_url %}
  • @@ -40,5 +47,10 @@
  • {% endif %} +
  • + + {% include "partials/flags.html" %} + +
  • diff --git a/docs/zh/faq/general.md b/docs/zh/faq/general.md index a45670232fa..adc0b5791b4 100644 --- a/docs/zh/faq/general.md +++ b/docs/zh/faq/general.md @@ -8,3 +8,5 @@ 大多数MapReduce系统允许您在集群上执行任意代码。但是,声明性查询语言更适合OLAP,以便快速运行实验。例如,Hadoop包含Hive和Pig,Cloudera Impala或Shark(过时)for Spark,以及Spark SQL、Presto和Apache Drill。与专业系统相比,运行此类任务时的性能非常不理想,所以将这些系统用作Web接口的后端服务是不现实的,因为延迟相对较高。 + +[来源文章](https://clickhouse.yandex/docs/zh/faq/general/) diff --git a/docs/zh/index.md b/docs/zh/index.md index 3eb9756a64e..6c16c93eafb 100644 --- a/docs/zh/index.md +++ b/docs/zh/index.md @@ -78,9 +78,8 @@ ClickHouse是一个用于联机分析(OLAP)的列式数据库管理系统(DBMS) 例如,查询“统计每个广告平台的记录数量”需要读取“广告平台ID”这一列,它在未压缩的情况下需要1个字节进行存储。如果大部分流量不是来自广告平台,那么这一列至少可以以十倍的压缩率被压缩。当采用快速压缩算法,它的解压速度最少在十亿字节(未压缩数据)每秒。换句话说,这个查询可以在单个服务器上以每秒大约几十亿行的速度进行处理。这实际上是当前实现的速度。 -
    示例 -

    -

    +
    示例 +``` $ clickhouse-client ClickHouse client version 0.0.52053. Connecting to localhost:9000. @@ -122,8 +121,7 @@ LIMIT 20 20 rows in set. Elapsed: 0.153 sec. Processed 1.00 billion rows, 4.00 GB (6.53 billion rows/s., 26.10 GB/s.) :) -
    -

    +```
    ### CPU @@ -139,3 +137,5 @@ LIMIT 20 这是不应该在一个通用数据库中实现的,因为这在运行简单查询时是没有意义的。但是也有例外,例如,MemSQL使用代码生成来减少处理SQL查询的延迟(只是为了比较,分析型数据库通常需要优化的是吞吐而不是延迟)。 请注意,为了提高CPU效率,查询语言必须是声明型的(SQL或MDX), 或者至少一个向量(J,K)。 查询应该只包含隐式循环,允许进行优化。 + +[来源文章](https://clickhouse.yandex/docs/zh/) diff --git a/docs/zh/interfaces/cli.md b/docs/zh/interfaces/cli.md index 134132bf87a..c9a287a1efd 100644 --- a/docs/zh/interfaces/cli.md +++ b/docs/zh/interfaces/cli.md @@ -111,3 +111,5 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA ``` + +[来源文章](https://clickhouse.yandex/docs/zh/interfaces/cli/) diff --git a/docs/zh/interfaces/formats.md b/docs/zh/interfaces/formats.md deleted file mode 120000 index 41a65ebe579..00000000000 --- a/docs/zh/interfaces/formats.md +++ /dev/null @@ -1 +0,0 @@ -../../en/interfaces/formats.md \ No newline at end of file diff --git a/docs/zh/interfaces/formats.md b/docs/zh/interfaces/formats.md new file mode 100644 index 00000000000..ec905713b1c --- /dev/null +++ b/docs/zh/interfaces/formats.md @@ -0,0 +1,625 @@ + + +# 输入输出格式 + +ClickHouse 可以接受多种数据格式,可以在 (`INSERT`) 以及 (`SELECT`) 请求中使用。 + +下列表格列出了支持的数据格式以及在 (`INSERT`) 以及 (`SELECT`) 请求中使用它们的方式。 + +| 格式 | INSERT | SELECT | +| ------- | -------- | -------- | +| [TabSeparated](#tabseparated) | ✔ | ✔ | +| [TabSeparatedRaw](#tabseparatedraw) | ✗ | ✔ | +| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ | +| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ | +| [CSV](#csv) | ✔ | ✔ | +| [CSVWithNames](#csvwithnames) | ✔ | ✔ | +| [Values](#values) | ✔ | ✔ | +| [Vertical](#vertical) | ✗ | ✔ | +| [VerticalRaw](#verticalraw) | ✗ | ✔ | +| [JSON](#json) | ✗ | ✔ | +| [JSONCompact](#jsoncompact) | ✗ | ✔ | +| [JSONEachRow](#jsoneachrow) | ✔ | ✔ | +| [TSKV](#tskv) | ✔ | ✔ | +| [Pretty](#pretty) | ✗ | ✔ | +| [PrettyCompact](#prettycompact) | ✗ | ✔ | +| [PrettyCompactMonoBlock](#prettycompactmonoblock) | ✗ | ✔ | +| [PrettyNoEscapes](#prettynoescapes) | ✗ | ✔ | +| [PrettySpace](#prettyspace) | ✗ | ✔ | +| [RowBinary](#rowbinary) | ✔ | ✔ | +| [Native](#native) | ✔ | ✔ | +| [Null](#null) | ✗ | ✔ | +| [XML](#xml) | ✗ | ✔ | +| [CapnProto](#capnproto) | ✔ | ✔ | + + + +## TabSeparated + +在 TabSeparated 格式中,数据按行写入。每行包含由制表符分隔的值。除了行中的最后一个值(后面紧跟换行符)之外,每个值都跟随一个制表符。 在任何地方都可以使用严格的 Unix 命令行。最后一行还必须在最后包含换行符。值以文本格式编写,不包含引号,并且要转义特殊字符。 + +这种格式也可以用 `TSV` 来表示。 + +TabSeparated 格式非常方便用于自定义程序或脚本处理数据。HTTP 客户端接口默认会用这种格式,命令行客户端批量模式下也会用这种格式。这种格式允许在不同数据库之间传输数据。例如,从 MYSQL 中导出数据然后导入到 ClickHouse 中,反之亦然。 + +TabSeparated 格式支持输出数据总值(当使用 WITH TOTALS) 以及极值(当 'extremes' 设置是1)。这种情况下,总值和极值输出在主数据的后面。主要的数据,总值,极值会以一个空行隔开,例如: + +``` sql +SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT TabSeparated`` +``` + +``` +2014-03-17 1406958 +2014-03-18 1383658 +2014-03-19 1405797 +2014-03-20 1353623 +2014-03-21 1245779 +2014-03-22 1031592 +2014-03-23 1046491 + +0000-00-00 8873898 + +2014-03-17 1031592 +2014-03-23 1406958 +``` + +### 数据解析方式 + +整数以十进制形式写入。数字在开头可以包含额外的 `+` 字符(解析时忽略,格式化时不记录)。非负数不能包含负号。 读取时,允许将空字符串解析为零,或者(对于带符号的类型)将仅包含负号的字符串解析为零。 不符合相应数据类型的数字可能会被解析为不同的数字,而不会显示错误消息。 + +浮点数以十进制形式写入。点号用作小数点分隔符。支持指数等符号,如'inf','+ inf','-inf'和'nan'。 浮点数的输入可以以小数点开始或结束。 +格式化的时候,浮点数的精确度可能会丢失。 +解析的时候,没有严格需要去读取与机器可以表示的最接近的数值。 + +日期会以 YYYY-MM-DD 格式写入和解析,但会以任何字符作为分隔符。 +带时间的日期会以 YYYY-MM-DD hh:mm:ss 格式写入和解析,但会以任何字符作为分隔符。 +这一切都发生在客户端或服务器启动时的系统时区(取决于哪一种格式的数据)。对于具有时间的日期,夏时制时间未指定。 因此,如果转储在夏令时中有时间,则转储不会明确地匹配数据,解析将选择两者之一。 +在读取操作期间,不正确的日期和具有时间的日期可以使用自然溢出或空日期和时间进行分析,而不会出现错误消息。 + +有个例外情况,Unix 时间戳格式(10个十进制数字)也支持使用时间解析日期。结果不是时区相关的。格式 YYYY-MM-DD hh:mm:ss和 NNNNNNNNNN 会自动区分。 + +字符串以反斜线转义的特殊字符输出。 以下转义序列用于输出:`\b`,`\f`,`\r`,`\n`,`\t`,`\0`,`\'`,`\\`。 解析还支持`\a`,`\v`和`\xHH`(十六进制转义字符)和任何`\c`字符,其中`c`是任何字符(这些序列被转换为`c`)。 因此,读取数据支持可以将换行符写为`\n`或`\`的格式,或者换行。例如,字符串 `Hello world` 在单词之间换行而不是空格可以解析为以下任何形式: + +``` +Hello\nworld + +Hello\ +world +``` + +第二种形式是支持的,因为 MySQL 读取 tab-separated 格式数据集的时候也会使用它。 + +在 TabSeparated 格式中传递数据时需要转义的最小字符集为:Tab,换行符(LF)和反斜杠。 + +只有一小组符号会被转义。你可以轻易地找到一个字符串值,但这不会正常在你的终端显示。 + +数组写在方括号内的逗号分隔值列表中。 通常情况下,数组中的数字项目会被拼凑,但日期,带时间的日期以及字符串将使用与上面相同的转义规则用单引号引起来。 + +[NULL](../query_language/syntax.md#null-literal) 将输出为 `\N`。 + + + +## TabSeparatedRaw + +与 `TabSeparated` 格式不一样的是,行数据是不会被转义的。 +该格式仅适用于输出查询结果,但不适用于解析输入(将数据插入到表中)。 + +这种格式也可以使用名称 `TSVRaw` 来表示。 + + +## TabSeparatedWithNames + +与 `TabSeparated` 格式不一样的是,第一行会显示列的名称。 +在解析过程中,第一行完全被忽略。您不能使用列名来确定其位置或检查其正确性。 +(未来可能会加入解析头行的功能) + +这种格式也可以使用名称 ` TSVWithNames` 来表示。 + + +## TabSeparatedWithNamesAndTypes + +与 `TabSeparated` 格式不一样的是,第一行会显示列的名称,第二行会显示列的类型。 +在解析过程中,第一行和第二行完全被忽略。 + +这种格式也可以使用名称 ` TSVWithNamesAndTypes` 来表示。 + + +## TSKV + +与 `TabSeparated` 格式类似,但它输出的是 `name=value` 的格式。名称会和 `TabSeparated` 格式一样被转义,`=` 字符也会被转义。 + +``` +SearchPhrase= count()=8267016 +SearchPhrase=bathroom interior design count()=2166 +SearchPhrase=yandex count()=1655 +SearchPhrase=2014 spring fashion count()=1549 +SearchPhrase=freeform photos count()=1480 +SearchPhrase=angelina jolie count()=1245 +SearchPhrase=omsk count()=1112 +SearchPhrase=photos of dog breeds count()=1091 +SearchPhrase=curtain designs count()=1064 +SearchPhrase=baku count()=1000 +``` + +[NULL](../query_language/syntax.md#null-literal) 输出为 `\N`。 + +``` sql +SELECT * FROM t_null FORMAT TSKV +``` + +``` +x=1 y=\N +``` + +当有大量的小列时,这种格式是低效的,通常没有理由使用它。它被用于 Yandex 公司的一些部门。 + +数据的输出和解析都支持这种格式。对于解析,任何顺序都支持不同列的值。可以省略某些值,用 `-` 表示, 它们被视为等于它们的默认值。在这种情况下,零和空行被用作默认值。作为默认值,不支持表中指定的复杂值。 + +对于不带等号或值,可以用附加字段 `tskv` 来表示,这种在解析上是被允许的。这样的话该字段被忽略。 + + +## CSV + +按逗号分隔的数据格式([RFC](https://tools.ietf.org/html/rfc4180))。 + +格式化的时候,行是用双引号括起来的。字符串中的双引号会以两个双引号输出,除此之外没有其他规则来做字符转义了。日期和时间也会以双引号包括。数字的输出不带引号。值由一个单独的字符隔开,这个字符默认是 `,`。行使用 Unix 换行符(LF)分隔。 数组序列化成 CSV 规则如下:首先将数组序列化为 TabSeparated 格式的字符串,然后将结果字符串用双引号包括输出到 CSV。CSV 格式的元组被序列化为单独的列(即它们在元组中的嵌套关系会丢失)。 + + +``` +clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv +``` + +*默认情况下间隔符是 `,` ,在[format_csv_delimiter](../operations/settings/settings.md#format_csv_delimiter)中可以了解更多间隔符配置。 + +解析的时候,可以使用或不使用引号来解析所有值。支持双引号和单引号。行也可以不用引号排列。 在这种情况下,它们被解析为逗号或换行符(CR 或 LF)。在解析不带引号的行时,若违反 RFC 规则,会忽略前导和尾随的空格和制表符。 对于换行,全部支持 Unix(LF),Windows(CR LF)和 Mac OS Classic(CR LF)。 + +`NULL` is formatted as `\N`. + +CSV 格式是和 TabSeparated 一样的方式输出总数和极值。 + +## CSVWithNames + +会输出带头部行,和 `TabSeparatedWithNames` 一样。 + + +## JSON + +以 JSON 格式输出数据。除了数据表之外,它还输出列名称和类型以及一些附加信息:输出行的总数以及在没有 LIMIT 时可以输出的行数。 例: + +``` sql +SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTALS ORDER BY c DESC LIMIT 5 FORMAT JSON +``` + +```json +{ + "meta": + [ + { + "name": "SearchPhrase", + "type": "String" + }, + { + "name": "c", + "type": "UInt64" + } + ], + + "data": + [ + { + "SearchPhrase": "", + "c": "8267016" + }, + { + "SearchPhrase": "bathroom interior design", + "c": "2166" + }, + { + "SearchPhrase": "yandex", + "c": "1655" + }, + { + "SearchPhrase": "spring 2014 fashion", + "c": "1549" + }, + { + "SearchPhrase": "freeform photos", + "c": "1480" + } + ], + + "totals": + { + "SearchPhrase": "", + "c": "8873898" + }, + + "extremes": + { + "min": + { + "SearchPhrase": "", + "c": "1480" + }, + "max": + { + "SearchPhrase": "", + "c": "8267016" + } + }, + + "rows": 5, + + "rows_before_limit_at_least": 141137 +} +``` + +JSON 与 JavaScript 兼容。为了确保这一点,一些字符被另外转义:斜线`/`被转义为`\/`; 替代的换行符 `U+2028` 和 `U+2029` 会打断一些浏览器解析,它们会被转义为 `\uXXXX`。 ASCII 控制字符被转义:退格,换页,换行,回车和水平制表符被替换为`\b`,`\f`,`\n`,`\r`,`\t` 作为使用`\uXXXX`序列的00-1F范围内的剩余字节。 无效的 UTF-8 序列更改为替换字符 ,因此输出文本将包含有效的 UTF-8 序列。 为了与 JavaScript 兼容,默认情况下,Int64 和 UInt64 整数用双引号引起来。要除去引号,可以将配置参数 output_format_json_quote_64bit_integers 设置为0。 + +`rows` – 结果输出的行数。 + +`rows_before_limit_at_least` 去掉 LIMIT 过滤后的最小行总数。 只会在查询包含 LIMIT 条件时输出。 +若查询包含 GROUP BY,rows_before_limit_at_least 就是去掉 LIMIT 后过滤后的准确行数。 + +`totals` – 总值 (当使用 TOTALS 条件时)。 + +`extremes` – 极值 (当 extremes 设置为 1时)。 + +该格式仅适用于输出查询结果,但不适用于解析输入(将数据插入到表中)。 + +ClickHouse 支持 [NULL](../query_language/syntax.md#null-literal), 在 JSON 格式中以 `null` 输出来表示. + +参考 JSONEachRow 格式。 + + + +## JSONCompact + +与 JSON 格式不同的是它以数组的方式输出结果,而不是以结构体。 + +示例: + +```json +{ + "meta": + [ + { + "name": "SearchPhrase", + "type": "String" + }, + { + "name": "c", + "type": "UInt64" + } + ], + + "data": + [ + ["", "8267016"], + ["bathroom interior design", "2166"], + ["yandex", "1655"], + ["fashion trends spring 2014", "1549"], + ["freeform photo", "1480"] + ], + + "totals": ["","8873898"], + + "extremes": + { + "min": ["","1480"], + "max": ["","8267016"] + }, + + "rows": 5, + + "rows_before_limit_at_least": 141137 +} +``` + +这种格式仅仅适用于输出结果集,而不适用于解析(将数据插入到表中)。 +参考 `JSONEachRow` 格式。 + + +## JSONEachRow + +将数据结果每一行以 JSON 结构体输出(换行分割 JSON 结构体)。 + +```json +{"SearchPhrase":"","count()":"8267016"} +{"SearchPhrase": "bathroom interior design","count()": "2166"} +{"SearchPhrase":"yandex","count()":"1655"} +{"SearchPhrase":"2014 spring fashion","count()":"1549"} +{"SearchPhrase":"freeform photo","count()":"1480"} +{"SearchPhrase":"angelina jolie","count()":"1245"} +{"SearchPhrase":"omsk","count()":"1112"} +{"SearchPhrase":"photos of dog breeds","count()":"1091"} +{"SearchPhrase":"curtain designs","count()":"1064"} +{"SearchPhrase":"baku","count()":"1000"} +``` + +与 JSON 格式不同的是,没有替换无效的UTF-8序列。任何一组字节都可以在行中输出。这是必要的,因为这样数据可以被格式化而不会丢失任何信息。值的转义方式与JSON相同。 + +对于解析,任何顺序都支持不同列的值。可以省略某些值 - 它们被视为等于它们的默认值。在这种情况下,零和空行被用作默认值。 作为默认值,不支持表中指定的复杂值。元素之间的空白字符被忽略。如果在对象之后放置逗号,它将被忽略。对象不一定必须用新行分隔。 + + +## Native + +最高性能的格式。 据通过二进制格式的块进行写入和读取。对于每个块,该块中的行数,列数,列名称和类型以及列的部分将被相继记录。 换句话说,这种格式是 “列式”的 - 它不会将列转换为行。 这是用于在服务器之间进行交互的本地界面中使用的格式,用于使用命令行客户端和 C++ 客户端。 + +您可以使用此格式快速生成只能由 ClickHouse DBMS 读取的格式。但自己处理这种格式是没有意义的。 + + +## Null + +没有输出。但是,查询已处理完毕,并且在使用命令行客户端时,数据将传输到客户端。这仅用于测试,包括生产力测试。 +显然,这种格式只适用于输出,不适用于解析。 + + +## Pretty + +将数据以表格形式输出,也可以使用 ANSI 转义字符在终端中设置颜色。 +它会绘制一个完整的表格,每行数据在终端中占用两行。 +每一个结果块都会以单独的表格输出。这是很有必要的,以便结果块不用缓冲结果输出(缓冲在可以预见结果集宽度的时候是很有必要的)。 + +[NULL](../query_language/syntax.md#null-literal) 输出为 `ᴺᵁᴸᴸ`。 + +``` sql +SELECT * FROM t_null +``` + +``` +┌─x─┬────y─┐ +│ 1 │ ᴺᵁᴸᴸ │ +└───┴──────┘ +``` + +为避免将太多数据传输到终端,只打印前10,000行。 如果行数大于或等于10,000,则会显示消息“Showed first 10 000”。 +该格式仅适用于输出查询结果,但不适用于解析输入(将数据插入到表中)。 + +Pretty格式支持输出总值(当使用 WITH TOTALS 时)和极值(当 `extremes` 设置为1时)。 在这些情况下,总数值和极值在主数据之后以单独的表格形式输出。 示例(以 PrettyCompact 格式显示): + +``` sql +SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT PrettyCompact +``` + +``` +┌──EventDate─┬───────c─┐ +│ 2014-03-17 │ 1406958 │ +│ 2014-03-18 │ 1383658 │ +│ 2014-03-19 │ 1405797 │ +│ 2014-03-20 │ 1353623 │ +│ 2014-03-21 │ 1245779 │ +│ 2014-03-22 │ 1031592 │ +│ 2014-03-23 │ 1046491 │ +└────────────┴─────────┘ + +Totals: +┌──EventDate─┬───────c─┐ +│ 0000-00-00 │ 8873898 │ +└────────────┴─────────┘ + +Extremes: +┌──EventDate─┬───────c─┐ +│ 2014-03-17 │ 1031592 │ +│ 2014-03-23 │ 1406958 │ +└────────────┴─────────┘ +``` + + + +## PrettyCompact + +与 `Pretty` 格式不一样的是,`PrettyCompact` 去掉了行之间的表格分割线,这样使得结果更加紧凑。这种格式会在交互命令行客户端下默认使用。 + + +## PrettyCompactMonoBlock + +与 `PrettyCompact` 格式不一样的是,它支持 10,000 行数据缓冲,然后输出在一个表格中,不会按照块来区分 + + +## PrettyNoEscapes + +与 `Pretty` 格式不一样的是,它不使用 ANSI 字符转义, 这在浏览器显示数据以及在使用 `watch` 命令行工具是有必要的。 + +示例: + +```bash +watch -n1 "clickhouse-client --query='SELECT event, value FROM system.events FORMAT PrettyCompactNoEscapes'" +``` + +您可以使用 HTTP 接口来获取数据,显示在浏览器中。 + +### PrettyCompactNoEscapes + +用法类似上述。 + +### PrettySpaceNoEscapes + +用法类似上述。 + + +## PrettySpace + +与 `PrettyCompact`(#prettycompact) 格式不一样的是,它使用空格来代替网格来显示数据。 + + +## RowBinary + +以二进制格式逐行格式化和解析数据。行和值连续列出,没有分隔符。 +这种格式比 Native 格式效率低,因为它是基于行的。 + +整数使用固定长度的小端表示法。 例如,UInt64 使用8个字节。 +DateTime 被表示为 UInt32 类型的Unix 时间戳值。 +Date 被表示为 UInt16 对象,它的值为 1970-01-01以来的天数。 +字符串表示为 varint 长度(无符号[LEB128](https://en.wikipedia.org/wiki/LEB128)),后跟字符串的字节数。 +FixedString 被简单地表示为一个字节序列。 + +数组表示为 varint 长度(无符号[LEB128](https://en.wikipedia.org/wiki/LEB128)),后跟有序的数组元素。 + +对于 [NULL](../query_language/syntax.md#null-literal) 的支持, 一个为 1 或 0 的字节会加在每个 [Nullable](../data_types/nullable.md#data_type-nullable) 值前面。如果为 1, 那么该值就是 `NULL`。 如果为 0,则不为 `NULL`。 + +## Values + +在括号中打印每一行。行由逗号分隔。最后一行之后没有逗号。括号内的值也用逗号分隔。数字以十进制格式输出,不含引号。 数组以方括号输出。带有时间的字符串,日期和时间用引号包围输出。转义字符的解析规则与 [TabSeparated](#tabseparated) 格式类似。 在格式化过程中,不插入额外的空格,但在解析过程中,空格是被允许并跳过的(除了数组值之外的空格,这是不允许的)。[NULL](../query_language/syntax.md#null-literal) 为 `NULL`。 + +以 Values 格式传递数据时需要转义的最小字符集是:单引号和反斜线。 + +这是 `INSERT INTO t VALUES ...` 中可以使用的格式,但您也可以将其用于查询结果。 + + + +## Vertical + +使用指定的列名在单独的行上打印每个值。如果每行都包含大量列,则此格式便于打印一行或几行。 + +[NULL](../query_language/syntax.md#null-literal) 输出为 `ᴺᵁᴸᴸ`。 + +示例: + +``` sql +SELECT * FROM t_null FORMAT Vertical +``` + +``` +Row 1: +────── +x: 1 +y: ᴺᵁᴸᴸ +``` + +该格式仅适用于输出查询结果,但不适用于解析输入(将数据插入到表中)。 + + + +## VerticalRaw + +和 `Vertical` 格式不同点在于,行是不会被转义的。 +这种格式仅仅适用于输出,但不适用于解析输入(将数据插入到表中)。 + +示例: + +``` +:) SHOW CREATE TABLE geonames FORMAT VerticalRaw; +Row 1: +────── +statement: CREATE TABLE default.geonames ( geonameid UInt32, date Date DEFAULT CAST('2017-12-08' AS Date)) ENGINE = MergeTree(date, geonameid, 8192) + +:) SELECT 'string with \'quotes\' and \t with some special \n characters' AS test FORMAT VerticalRaw; +Row 1: +────── +test: string with 'quotes' and with some special + characters +``` + +Compare with the Vertical format: + +``` +:) SELECT 'string with \'quotes\' and \t with some special \n characters' AS test FORMAT Vertical; +Row 1: +────── +test: string with \'quotes\' and \t with some special \n characters +``` + + + +## XML + +该格式仅适用于输出查询结果,但不适用于解析输入,示例: + + +```xml + + + + + + SearchPhrase + String + + + count() + UInt64 + + + + + + + 8267016 + + + bathroom interior design + 2166 + + + yandex + 1655 + + + 2014 spring fashion + 1549 + + + freeform photos + 1480 + + + angelina jolie + 1245 + + + omsk + 1112 + + + photos of dog breeds + 1091 + + + curtain designs + 1064 + + + baku + 1000 + + + 10 + 141137 + +``` + +如果列名称没有可接受的格式,则仅使用 `field` 作为元素名称。 通常,XML 结构遵循 JSON 结构。 +就像JSON一样,将无效的 UTF-8 字符都作替换,以便输出文本将包含有效的 UTF-8 字符序列。 + +在字符串值中,字符 `<` 和 `&` 被转义为 `<` 和 `&`。 + +数组输出为 ` Hello World ... `,元组输出为 ` Hello World ... ` 。 + + + +## CapnProto + +Cap'n Proto 是一种二进制消息格式,类似 Protocol Buffers 和 Thriftis,但与 JSON 或 MessagePack 格式不一样。 + +Cap'n Proto 消息格式是严格类型的,而不是自我描述,这意味着它们不需要外部的描述。这种格式可以实时地应用,并针对每个查询进行缓存。 + +``` sql +SELECT SearchPhrase, count() AS c FROM test.hits + GROUP BY SearchPhrase FORMAT CapnProto SETTINGS schema = 'schema:Message' +``` + +其中 `schema.capnp` 描述如下: + +``` +struct Message { + SearchPhrase @0 :Text; + c @1 :Uint64; +} +``` + +格式文件存储的目录可以在服务配置中的[ format_schema_path ](../operations/server_settings/settings.md#server_settings-format_schema_path) 指定。 + +Cap'n Proto 反序列化是很高效的,通常不会增加系统的负载。 + +[来源文章](https://clickhouse.yandex/docs/zh/interfaces/formats/) diff --git a/docs/zh/interfaces/http_interface.md b/docs/zh/interfaces/http_interface.md index 853b31b4545..8b150d059a9 100644 --- a/docs/zh/interfaces/http_interface.md +++ b/docs/zh/interfaces/http_interface.md @@ -1,4 +1,4 @@ -# HTTP 接口 +# HTTP 客户端 HTTP 接口可以让你通过任何平台和编程语言来使用 ClickHouse。我们用 Java 和 Perl 以及 shell 脚本来访问它。在其他的部门中,HTTP 接口会用在 Perl,Python 以及 Go 中。HTTP 接口比 TCP 原生接口更为局限,但是却有更好的兼容性。 @@ -213,3 +213,5 @@ curl -sS 'http://localhost:8123/?max_result_bytes=4000000&buffer_size=3000000&wa ``` 查询请求响应状态码和 HTTP 头被发送到客户端后,若发生查询处理出错,使用缓冲区可以避免这种情况的发生。在这种情况下,响应主体的结尾会写入一条错误消息,而在客户端,只能在解析阶段检测到该错误。 + +[来源文章](https://clickhouse.yandex/docs/zh/interfaces/http_interface/) diff --git a/docs/zh/interfaces/index.md b/docs/zh/interfaces/index.md index 4ff97552792..b5603f81e6f 100644 --- a/docs/zh/interfaces/index.md +++ b/docs/zh/interfaces/index.md @@ -1,7 +1,9 @@ -# 接口 +# 客户端 为了探索 ClickHouse 的能力,如导入数据到表中,或做一些手动的查询,可以使用 clickhouse-client 命令行程序来完成 + +[来源文章](https://clickhouse.yandex/docs/zh/interfaces/) diff --git a/docs/zh/interfaces/jdbc.md b/docs/zh/interfaces/jdbc.md index f4c8139ef63..41bf39ab2f7 100644 --- a/docs/zh/interfaces/jdbc.md +++ b/docs/zh/interfaces/jdbc.md @@ -4,3 +4,5 @@ - 三方提供的 JDBC 驱动 [ClickHouse-Native-JDBC](https://github.com/housepower/ClickHouse-Native-JDBC). + +[来源文章](https://clickhouse.yandex/docs/zh/interfaces/jdbc/) diff --git a/docs/zh/interfaces/tcp.md b/docs/zh/interfaces/tcp.md index 1ff4270d690..742e252a25f 100644 --- a/docs/zh/interfaces/tcp.md +++ b/docs/zh/interfaces/tcp.md @@ -1,4 +1,6 @@ -# 原生接口(TCP) +# 原生客户端接口(TCP) TCP 原生接口用于 `clickhouse-client` 命令行,它可以在分布式查询执行中和服务器进行交互,并且可以用在 C++ 程序中。我们讲解只覆盖命令行客户端。 + +[来源文章](https://clickhouse.yandex/docs/zh/interfaces/tcp/) diff --git a/docs/zh/interfaces/third-party_client_libraries.md b/docs/zh/interfaces/third-party_client_libraries.md index 7e9203beb01..c3cabeaca80 100644 --- a/docs/zh/interfaces/third-party_client_libraries.md +++ b/docs/zh/interfaces/third-party_client_libraries.md @@ -45,3 +45,5 @@ - Nim - [nim-clickhouse](https://github.com/leonardoce/nim-clickhouse) + +[来源文章](https://clickhouse.yandex/docs/zh/interfaces/third-party_client_libraries/) diff --git a/docs/zh/interfaces/third-party_gui.md b/docs/zh/interfaces/third-party_gui.md index db2a0f58243..e9f20405234 100644 --- a/docs/zh/interfaces/third-party_gui.md +++ b/docs/zh/interfaces/third-party_gui.md @@ -28,7 +28,6 @@ ClickHouse Web 界面 [Tabix](https://github.com/tabixio/tabix). - 快速查看列占用的空间。 - 服务配置。 -The following features are planned for development: 以下功能正在计划开发: - 数据库管理 - 用户管理 @@ -37,3 +36,5 @@ The following features are planned for development: - 集群管理 - 监控副本情况以及 Kafka 引擎表 + +[来源文章](https://clickhouse.yandex/docs/zh/interfaces/third-party_gui/) diff --git a/docs/zh/query_language/table_functions/jdbc.md b/docs/zh/query_language/table_functions/jdbc.md new file mode 120000 index 00000000000..73bec80ca58 --- /dev/null +++ b/docs/zh/query_language/table_functions/jdbc.md @@ -0,0 +1 @@ +../../../en/query_language/table_functions/jdbc.md \ No newline at end of file diff --git a/libs/libmemcpy/impl/README.md b/libs/libmemcpy/impl/README.md index 91e01d4a5a0..ac807ecd925 100644 --- a/libs/libmemcpy/impl/README.md +++ b/libs/libmemcpy/impl/README.md @@ -20,7 +20,7 @@ Reference [Using Block Prefetch for Optimized Memory Performance](http://files.rsdn.ru/23380/AMD_block_prefetch_paper.pdf) -The artical only focused on aligned huge memory copy. You need handle other conditions by your self. +The article only focused on aligned huge memory copy. You need handle other conditions by your self. Results diff --git a/utils/travis/normal.sh b/utils/travis/normal.sh index 16482c3f184..7f45641d45f 100755 --- a/utils/travis/normal.sh +++ b/utils/travis/normal.sh @@ -32,7 +32,7 @@ cmake $CUR_DIR/../.. -DCMAKE_CXX_COMPILER=`which $DEB_CXX $CXX` -DCMAKE_C_COMPIL `# Use all possible contrib libs from system` \ -DUNBUNDLED=1 \ `# Disable all features` \ - -DENABLE_CAPNP=0 -DENABLE_RDKAFKA=0 -DENABLE_EMBEDDED_COMPILER=0 -DENABLE_JEMALLOC=0 -DENABLE_UNWIND=0 -DENABLE_MYSQL=0 -DENABLE_POCO_ODBC=0 -DENABLE_ODBC=0 -DUSE_INTERNAL_LLVM_LIBRARY=0 $CMAKE_FLAGS + -DENABLE_CAPNP=0 -DENABLE_RDKAFKA=0 -DENABLE_EMBEDDED_COMPILER=0 -DUSE_INTERNAL_LLVM_LIBRARY=0 -DENABLE_JEMALLOC=0 -DENABLE_UNWIND=0 -DENABLE_MYSQL=0 -DENABLE_POCO_ODBC=0 -DENABLE_ODBC=0 $CMAKE_FLAGS ninja clickhouse-bundle diff --git a/website/gulpfile.js b/website/gulpfile.js index 09c617eccad..ca254bf681f 100644 --- a/website/gulpfile.js +++ b/website/gulpfile.js @@ -26,6 +26,7 @@ var paths = { docstxt: ['docs/**/*.txt', 'docs/redirects.conf'], docsjson: ['docs/**/*.json'], docsxml: ['docs/**/*.xml'], + docspdf: ['docs/**/*.pdf'], docssitemap: ['sitemap.xml', 'sitemap_static.xml'], scripts: [ '**/*.js', @@ -78,6 +79,11 @@ gulp.task('docsxml', ['docs'], function () { .pipe(gulp.dest(outputDir + '/docs')) }); +gulp.task('docspdf', ['docs'], function () { + return gulp.src(paths.docspdf) + .pipe(gulp.dest(outputDir + '/docs')) +}); + gulp.task('docssitemap', [], function () { return gulp.src(paths.docssitemap) .pipe(gulp.dest(outputDir + '/docs')) @@ -93,7 +99,7 @@ gulp.task('robotstxt', [], function () { .pipe(gulp.dest(outputDir)) }); -gulp.task('htmls', ['docs', 'docstxt', 'docsjson', 'docsxml', 'docssitemap'], function () { +gulp.task('htmls', ['docs', 'docstxt', 'docsjson', 'docsxml', 'docspdf', 'docssitemap'], function () { return gulp.src(paths.htmls) .pipe(htmlmin({collapseWhitespace: true})) .pipe(minifyInline()) diff --git a/website/index.css b/website/index.css index fb626a2d802..8a51c5de0ed 100644 --- a/website/index.css +++ b/website/index.css @@ -65,7 +65,7 @@ body { background: #fff; - font: 300 14pt/200% 'Yandex Sans Text Web', Arial, sans-serif; + font: 400 14pt/200% 'Yandex Sans Text Web', Arial, sans-serif; margin: 0; padding: 0; } diff --git a/website/nginx/nginx.conf b/website/nginx/nginx.conf index 0227b86114f..24bfc1cca3b 100644 --- a/website/nginx/nginx.conf +++ b/website/nginx/nginx.conf @@ -32,6 +32,10 @@ http { gzip_comp_level 5; gzip_min_length 256; + add_header X-Content-Type-Options nosniff; + add_header X-Frame-Options DENY; + add_header X-XSS-Protection "1; mode=block"; + include /etc/nginx/conf.d/*.conf; include /etc/nginx/sites-enabled/*; }