diff --git a/.gitmodules b/.gitmodules index a7061ececc6..923554a1532 100644 --- a/.gitmodules +++ b/.gitmodules @@ -36,7 +36,7 @@ url = https://github.com/ClickHouse-Extras/llvm [submodule "contrib/mariadb-connector-c"] path = contrib/mariadb-connector-c - url = https://github.com/MariaDB/mariadb-connector-c.git + url = https://github.com/ClickHouse-Extras/mariadb-connector-c.git [submodule "contrib/jemalloc"] path = contrib/jemalloc url = https://github.com/jemalloc/jemalloc.git diff --git a/contrib/mariadb-connector-c b/contrib/mariadb-connector-c index a0fd36cc5a5..d85d0e98999 160000 --- a/contrib/mariadb-connector-c +++ b/contrib/mariadb-connector-c @@ -1 +1 @@ -Subproject commit a0fd36cc5a5313414a5a2ebe9322577a29b4782a +Subproject commit d85d0e98999cd9e28ceb66645999b4a9ce85370e diff --git a/dbms/src/AggregateFunctions/AggregateFunctionArray.h b/dbms/src/AggregateFunctions/AggregateFunctionArray.h index ccf5f787309..5dfebf13d52 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionArray.h @@ -85,7 +85,7 @@ public: const ColumnArray & first_array_column = static_cast(*columns[0]); const IColumn::Offsets & offsets = first_array_column.getOffsets(); - size_t begin = row_num == 0 ? 0 : offsets[row_num - 1]; + size_t begin = offsets[row_num - 1]; size_t end = offsets[row_num]; /// Sanity check. NOTE We can implement specialization for a case with single argument, if the check will hurt performance. diff --git a/dbms/src/AggregateFunctions/AggregateFunctionForEach.h b/dbms/src/AggregateFunctions/AggregateFunctionForEach.h index ddd94f8c918..79932c7744f 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionForEach.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionForEach.h @@ -146,7 +146,7 @@ public: const ColumnArray & first_array_column = static_cast(*columns[0]); const IColumn::Offsets & offsets = first_array_column.getOffsets(); - size_t begin = row_num == 0 ? 0 : offsets[row_num - 1]; + size_t begin = offsets[row_num - 1]; size_t end = offsets[row_num]; /// Sanity check. NOTE We can implement specialization for a case with single argument, if the check will hurt performance. diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h index c6a347aaf3b..094cae05615 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -119,7 +119,7 @@ public: ColumnArray & arr_to = static_cast(to); ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); - offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + size); + offsets_to.push_back(offsets_to.back() + size); typename ColumnVector::Container & data_to = static_cast &>(arr_to.getData()).getData(); data_to.insert(this->data(place).value.begin(), this->data(place).value.end()); @@ -370,7 +370,7 @@ public: auto & column_array = static_cast(to); auto & offsets = column_array.getOffsets(); - offsets.push_back((offsets.size() == 0 ? 0 : offsets.back()) + data(place).elems); + offsets.push_back(offsets.back() + data(place).elems); auto & column_data = column_array.getData(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h index e24f20b1769..9a96425a240 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h @@ -83,7 +83,7 @@ public: const typename State::Set & set = this->data(place).value; size_t size = set.size(); - offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + size); + offsets_to.push_back(offsets_to.back() + size); typename ColumnVector::Container & data_to = static_cast &>(arr_to.getData()).getData(); size_t old_size = data_to.size(); @@ -207,7 +207,7 @@ public: IColumn & data_to = arr_to.getData(); auto & set = this->data(place).value; - offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + set.size()); + offsets_to.push_back(offsets_to.back() + set.size()); for (auto & elem : set) { diff --git a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h index 21d871c8e79..3d420d13af0 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -138,7 +138,7 @@ public: ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); size_t size = levels.size(); - offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + size); + offsets_to.push_back(offsets_to.back() + size); if (!size) return; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index b79670eb3de..c3997b37192 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -83,7 +83,7 @@ public: const ColumnArray & array_column = static_cast(*columns[0]); const IColumn::Offsets & offsets = array_column.getOffsets(); const auto & keys_vec = static_cast(array_column.getData()); - const size_t keys_vec_offset = row_num == 0 ? 0 : offsets[row_num - 1]; + const size_t keys_vec_offset = offsets[row_num - 1]; const size_t keys_vec_size = (offsets[row_num] - keys_vec_offset); // Columns 1..n contain arrays of numeric values to sum @@ -93,7 +93,7 @@ public: Field value; const ColumnArray & array_column = static_cast(*columns[col + 1]); const IColumn::Offsets & offsets = array_column.getOffsets(); - const size_t values_vec_offset = row_num == 0 ? 0 : offsets[row_num - 1]; + const size_t values_vec_offset = offsets[row_num - 1]; const size_t values_vec_size = (offsets[row_num] - values_vec_offset); // Expect key and value arrays to be of same length diff --git a/dbms/src/AggregateFunctions/AggregateFunctionTopK.h b/dbms/src/AggregateFunctions/AggregateFunctionTopK.h index ac20f48dd12..4d1238a12cc 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionTopK.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionTopK.h @@ -93,7 +93,7 @@ public: auto result_vec = set.topK(threshold); size_t size = result_vec.size(); - offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + size); + offsets_to.push_back(offsets_to.back() + size); typename ColumnVector::Container & data_to = static_cast &>(arr_to.getData()).getData(); size_t old_size = data_to.size(); @@ -212,7 +212,7 @@ public: IColumn & data_to = arr_to.getData(); auto result_vec = this->data(place).value.topK(threshold); - offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + result_vec.size()); + offsets_to.push_back(offsets_to.back() + result_vec.size()); for (auto & elem : result_vec) { diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index eb497ea8f31..82830281373 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -166,7 +166,7 @@ void ColumnArray::insertData(const char * pos, size_t length) if (pos != end) throw Exception("Incorrect length argument for method ColumnArray::insertData", ErrorCodes::BAD_ARGUMENTS); - getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + elems); + getOffsets().push_back(getOffsets().back() + elems); } @@ -194,7 +194,7 @@ const char * ColumnArray::deserializeAndInsertFromArena(const char * pos) for (size_t i = 0; i < array_size; ++i) pos = getData().deserializeAndInsertFromArena(pos); - getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + array_size); + getOffsets().push_back(getOffsets().back() + array_size); return pos; } @@ -216,7 +216,7 @@ void ColumnArray::insert(const Field & x) size_t size = array.size(); for (size_t i = 0; i < size; ++i) getData().insert(array[i]); - getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + size); + getOffsets().push_back(getOffsets().back() + size); } @@ -227,13 +227,13 @@ void ColumnArray::insertFrom(const IColumn & src_, size_t n) size_t offset = src.offsetAt(n); getData().insertRangeFrom(src.getData(), offset, size); - getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + size); + getOffsets().push_back(getOffsets().back() + size); } void ColumnArray::insertDefault() { - getOffsets().push_back(getOffsets().size() == 0 ? 0 : getOffsets().back()); + getOffsets().push_back(getOffsets().back()); } diff --git a/dbms/src/Columns/ColumnArray.h b/dbms/src/Columns/ColumnArray.h index c2c17c17ed7..0caf6cbb87e 100644 --- a/dbms/src/Columns/ColumnArray.h +++ b/dbms/src/Columns/ColumnArray.h @@ -124,8 +124,8 @@ private: ColumnPtr data; ColumnPtr offsets; - size_t ALWAYS_INLINE offsetAt(size_t i) const { return i == 0 ? 0 : getOffsets()[i - 1]; } - size_t ALWAYS_INLINE sizeAt(size_t i) const { return i == 0 ? getOffsets()[0] : (getOffsets()[i] - getOffsets()[i - 1]); } + size_t ALWAYS_INLINE offsetAt(size_t i) const { return getOffsets()[i - 1]; } + size_t ALWAYS_INLINE sizeAt(size_t i) const { return getOffsets()[i] - getOffsets()[i - 1]; } /// Multiply values if the nested column is ColumnVector. diff --git a/dbms/src/Columns/ColumnString.cpp b/dbms/src/Columns/ColumnString.cpp index a92f62a2b08..86521e3dfb5 100644 --- a/dbms/src/Columns/ColumnString.cpp +++ b/dbms/src/Columns/ColumnString.cpp @@ -148,7 +148,7 @@ ColumnPtr ColumnString::permute(const Permutation & perm, size_t limit) const for (size_t i = 0; i < limit; ++i) { size_t j = perm[i]; - size_t string_offset = j == 0 ? 0 : offsets[j - 1]; + size_t string_offset = offsets[j - 1]; size_t string_size = offsets[j] - string_offset; memcpySmallAllowReadWriteOverflow15(&res_chars[current_new_offset], &chars[string_offset], string_size); @@ -219,7 +219,7 @@ ColumnPtr ColumnString::indexImpl(const PaddedPODArray & indexes, size_t l for (size_t i = 0; i < limit; ++i) { size_t j = indexes[i]; - size_t string_offset = j == 0 ? 0 : offsets[j - 1]; + size_t string_offset = offsets[j - 1]; size_t string_size = offsets[j] - string_offset; memcpySmallAllowReadWriteOverflow15(&res_chars[current_new_offset], &chars[string_offset], string_size); diff --git a/dbms/src/Columns/ColumnString.h b/dbms/src/Columns/ColumnString.h index 863db4f4eff..645f9dcac47 100644 --- a/dbms/src/Columns/ColumnString.h +++ b/dbms/src/Columns/ColumnString.h @@ -31,10 +31,10 @@ private: /// For convenience, every string ends with terminating zero byte. Note that strings could contain zero bytes in the middle. Chars chars; - size_t ALWAYS_INLINE offsetAt(size_t i) const { return i == 0 ? 0 : offsets[i - 1]; } + size_t ALWAYS_INLINE offsetAt(size_t i) const { return offsets[i - 1]; } /// Size of i-th element, including terminating zero. - size_t ALWAYS_INLINE sizeAt(size_t i) const { return i == 0 ? offsets[0] : (offsets[i] - offsets[i - 1]); } + size_t ALWAYS_INLINE sizeAt(size_t i) const { return offsets[i] - offsets[i - 1]; } template struct less; @@ -203,7 +203,7 @@ public: void insertDefault() override { chars.push_back(0); - offsets.push_back(offsets.size() == 0 ? 1 : (offsets.back() + 1)); + offsets.push_back(offsets.back() + 1); } int compareAt(size_t n, size_t m, const IColumn & rhs_, int /*nan_direction_hint*/) const override diff --git a/dbms/src/Columns/IColumnDummy.h b/dbms/src/Columns/IColumnDummy.h index 576b2d70459..beb0d101122 100644 --- a/dbms/src/Columns/IColumnDummy.h +++ b/dbms/src/Columns/IColumnDummy.h @@ -107,7 +107,7 @@ public: if (s != offsets.size()) throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); - return cloneDummy(s == 0 ? 0 : offsets.back()); + return cloneDummy(offsets.back()); } MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override diff --git a/dbms/src/Common/PODArray.cpp b/dbms/src/Common/PODArray.cpp new file mode 100644 index 00000000000..95810551ac4 --- /dev/null +++ b/dbms/src/Common/PODArray.cpp @@ -0,0 +1,8 @@ +#include + +namespace DB +{ +/// Used for left padding of PODArray when empty +const char EmptyPODArray[EmptyPODArraySize]{}; + +} diff --git a/dbms/src/Common/PODArray.h b/dbms/src/Common/PODArray.h index a80dc74de76..280e0f9811b 100644 --- a/dbms/src/Common/PODArray.h +++ b/dbms/src/Common/PODArray.h @@ -20,6 +20,11 @@ namespace DB { +inline constexpr size_t integerRoundUp(size_t value, size_t dividend) +{ + return ((value + dividend - 1) / dividend) * dividend; +} + /** A dynamic array for POD types. * Designed for a small number of large arrays (rather than a lot of small ones). * To be more precise - for use in ColumnVector. @@ -37,6 +42,10 @@ namespace DB * The template parameter `pad_right` - always allocate at the end of the array as many unused bytes. * Can be used to make optimistic reading, writing, copying with unaligned SIMD instructions. * + * The template parameter `pad_left` - always allocate memory before 0th element of the array (rounded up to the whole number of elements) + * and zero initialize -1th element. It allows to use -1th element that will have value 0. + * This gives performance benefits when converting an array of offsets to array of sizes. + * * Some methods using allocator have TAllocatorParams variadic arguments. * These arguments will be passed to corresponding methods of TAllocator. * Example: pointer to Arena, that is used for allocations. @@ -49,16 +58,25 @@ namespace DB * TODO Pass alignment to Allocator. * TODO Allow greater alignment than alignof(T). Example: array of char aligned to page size. */ -template , size_t pad_right_ = 0> +static constexpr size_t EmptyPODArraySize = 1024; +extern const char EmptyPODArray[EmptyPODArraySize]; + +template , size_t pad_right_ = 0, size_t pad_left_ = 0> class PODArray : private boost::noncopyable, private TAllocator /// empty base optimization { protected: /// Round padding up to an whole number of elements to simplify arithmetic. - static constexpr size_t pad_right = (pad_right_ + sizeof(T) - 1) / sizeof(T) * sizeof(T); + static constexpr size_t pad_right = integerRoundUp(pad_right_, sizeof(T)); + /// pad_left is also rounded up to 16 bytes to maintain alignment of allocated memory. + static constexpr size_t pad_left = integerRoundUp(integerRoundUp(pad_left_, sizeof(T)), 16); + /// Empty array will point to this static memory as padding. + static constexpr char * null = pad_left ? const_cast(EmptyPODArray) + EmptyPODArraySize : nullptr; - char * c_start = nullptr; - char * c_end = nullptr; - char * c_end_of_storage = nullptr; /// Does not include pad_right. + static_assert(pad_left <= EmptyPODArraySize && "Left Padding exceeds EmptyPODArraySize. Is the element size too large?"); + + char * c_start = null; /// Does not include pad_left. + char * c_end = null; + char * c_end_of_storage = null; /// Does not include pad_right. T * t_start() { return reinterpret_cast(c_start); } T * t_end() { return reinterpret_cast(c_end); } @@ -72,7 +90,7 @@ protected: static size_t byte_size(size_t num_elements) { return num_elements * sizeof(T); } /// Minimum amount of memory to allocate for num_elements, including padding. - static size_t minimum_memory_for_elements(size_t num_elements) { return byte_size(num_elements) + pad_right; } + static size_t minimum_memory_for_elements(size_t num_elements) { return byte_size(num_elements) + pad_right + pad_left; } void alloc_for_num_elements(size_t num_elements) { @@ -82,22 +100,24 @@ protected: template void alloc(size_t bytes, TAllocatorParams &&... allocator_params) { - c_start = c_end = reinterpret_cast(TAllocator::alloc(bytes, std::forward(allocator_params)...)); - c_end_of_storage = c_start + bytes - pad_right; + c_start = c_end = reinterpret_cast(TAllocator::alloc(bytes, std::forward(allocator_params)...)) + pad_left; + c_end_of_storage = c_start + bytes - pad_right - pad_left; + if (pad_left) + t_start()[-1] = {}; } void dealloc() { - if (c_start == nullptr) + if (c_start == null) return; - TAllocator::free(c_start, allocated_bytes()); + TAllocator::free(c_start - pad_left, allocated_bytes()); } template void realloc(size_t bytes, TAllocatorParams &&... allocator_params) { - if (c_start == nullptr) + if (c_start == null) { alloc(bytes, std::forward(allocator_params)...); return; @@ -105,15 +125,18 @@ protected: ptrdiff_t end_diff = c_end - c_start; - c_start = reinterpret_cast(TAllocator::realloc(c_start, allocated_bytes(), bytes, std::forward(allocator_params)...)); - + c_start = reinterpret_cast( + TAllocator::realloc(c_start - pad_left, allocated_bytes(), bytes, std::forward(allocator_params)...)) + + pad_left; c_end = c_start + end_diff; - c_end_of_storage = c_start + bytes - pad_right; + c_end_of_storage = c_start + bytes - pad_right - pad_left; + if (pad_left) + t_start()[-1] = {}; } bool isInitialized() const { - return (c_start != nullptr) && (c_end != nullptr) && (c_end_of_storage != nullptr); + return (c_start != null) && (c_end != null) && (c_end_of_storage != null); } bool isAllocatedFromStack() const @@ -139,7 +162,7 @@ protected: public: using value_type = T; - size_t allocated_bytes() const { return c_end_of_storage - c_start + pad_right; } + size_t allocated_bytes() const { return c_end_of_storage - c_start + pad_right + pad_left; } /// You can not just use `typedef`, because there is ambiguity for the constructors and `assign` functions. struct iterator : public boost::iterator_adaptor @@ -378,9 +401,9 @@ public: memcpy(dest.c_start, src.c_start, byte_size(src.size())); dest.c_end = dest.c_start + (src.c_end - src.c_start); - src.c_start = nullptr; - src.c_end = nullptr; - src.c_end_of_storage = nullptr; + src.c_start = null; + src.c_end = null; + src.c_end_of_storage = null; } else { @@ -504,15 +527,9 @@ void swap(PODArray & lhs, PODArray> -using PaddedPODArray = PODArray; - - -inline constexpr size_t integerRound(size_t value, size_t dividend) -{ - return ((value + dividend - 1) / dividend) * dividend; -} +using PaddedPODArray = PODArray; template -using PODArrayWithStackMemory = PODArray, integerRound(stack_size_in_bytes, sizeof(T))>>; +using PODArrayWithStackMemory = PODArray, integerRoundUp(stack_size_in_bytes, sizeof(T))>>; } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp index ff1e4fb5e53..71d45ca3cb7 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp @@ -20,23 +20,21 @@ ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, Even ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, Coordination::WatchCallback caller_watch_callback) { - zkutil::ZooKeeperPtr zookeeper; std::unordered_set invalidated_paths; { std::lock_guard lock(context->mutex); - if (!context->zookeeper) + if (context->all_paths_invalidated) { /// Possibly, there was a previous session and it has expired. Clear the cache. path_to_cached_znode.clear(); - - context->zookeeper = get_zookeeper(); + context->all_paths_invalidated = false; } - zookeeper = context->zookeeper; invalidated_paths.swap(context->invalidated_paths); } + zkutil::ZooKeeperPtr zookeeper = get_zookeeper(); if (!zookeeper) throw DB::Exception("Could not get znode: `" + path + "'. ZooKeeper not configured.", DB::ErrorCodes::NO_ZOOKEEPER); @@ -65,8 +63,8 @@ ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, Coor changed = owned_context->invalidated_paths.emplace(response.path).second; else if (response.state == Coordination::EXPIRED_SESSION) { - owned_context->zookeeper = nullptr; owned_context->invalidated_paths.clear(); + owned_context->all_paths_invalidated = true; changed = true; } } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h index d47cbcb36fe..73e7753cdcf 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h @@ -53,8 +53,8 @@ private: struct Context { std::mutex mutex; - zkutil::ZooKeeperPtr zookeeper; std::unordered_set invalidated_paths; + bool all_paths_invalidated = false; }; std::shared_ptr context; diff --git a/dbms/src/DataTypes/DataTypeArray.cpp b/dbms/src/DataTypes/DataTypeArray.cpp index 1b739a040ae..3dd4db815f2 100644 --- a/dbms/src/DataTypes/DataTypeArray.cpp +++ b/dbms/src/DataTypes/DataTypeArray.cpp @@ -59,7 +59,7 @@ void DataTypeArray::serializeBinary(const IColumn & column, size_t row_num, Writ const ColumnArray & column_array = static_cast(column); const ColumnArray::Offsets & offsets = column_array.getOffsets(); - size_t offset = row_num == 0 ? 0 : offsets[row_num - 1]; + size_t offset = offsets[row_num - 1]; size_t next_offset = offsets[row_num]; size_t size = next_offset - offset; @@ -113,7 +113,7 @@ namespace ? offset + limit : size; - ColumnArray::Offset prev_offset = offset == 0 ? 0 : offset_values[offset - 1]; + ColumnArray::Offset prev_offset = offset_values[offset - 1]; for (size_t i = offset; i < end; ++i) { ColumnArray::Offset current_offset = offset_values[i]; @@ -280,7 +280,7 @@ static void serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffe const ColumnArray & column_array = static_cast(column); const ColumnArray::Offsets & offsets = column_array.getOffsets(); - size_t offset = row_num == 0 ? 0 : offsets[row_num - 1]; + size_t offset = offsets[row_num - 1]; size_t next_offset = offsets[row_num]; const IColumn & nested_column = column_array.getData(); @@ -369,7 +369,7 @@ void DataTypeArray::serializeTextJSON(const IColumn & column, size_t row_num, Wr const ColumnArray & column_array = static_cast(column); const ColumnArray::Offsets & offsets = column_array.getOffsets(); - size_t offset = row_num == 0 ? 0 : offsets[row_num - 1]; + size_t offset = offsets[row_num - 1]; size_t next_offset = offsets[row_num]; const IColumn & nested_column = column_array.getData(); @@ -396,7 +396,7 @@ void DataTypeArray::serializeTextXML(const IColumn & column, size_t row_num, Wri const ColumnArray & column_array = static_cast(column); const ColumnArray::Offsets & offsets = column_array.getOffsets(); - size_t offset = row_num == 0 ? 0 : offsets[row_num - 1]; + size_t offset = offsets[row_num - 1]; size_t next_offset = offsets[row_num]; const IColumn & nested_column = column_array.getData(); diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 5ca141a64a0..5df8547bf4a 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -1142,32 +1142,24 @@ struct ToIntMonotonicity if (!type.isValueRepresentedByNumber()) return {}; - size_t size_of_type = type.getSizeOfValueInMemory(); - - /// If type is expanding - if (sizeof(T) > size_of_type) - { - /// If convert signed -> signed or unsigned -> signed, then function is monotonic. - if (std::is_signed_v || type.isValueRepresentedByUnsignedInteger()) - return {true, true, true}; - - /// If arguments from the same half, then function is monotonic. - if ((left.get() >= 0) == (right.get() >= 0)) - return {true, true, true}; - } - - /// If type is same, too. (Enum has separate case, because it is different data type) + /// If type is same, the conversion is always monotonic. + /// (Enum has separate case, because it is different data type) if (checkAndGetDataType>(&type) || checkAndGetDataType>(&type)) return { true, true, true }; - /// In other cases, if range is unbounded, we don't know, whether function is monotonic or not. - if (left.isNull() || right.isNull()) - return {}; + /// Float cases. - /// If converting from float, for monotonicity, arguments must fit in range of result type. + /// When converting to Float, the conversion is always monotonic. + if (std::is_floating_point_v) + return {true, true, true}; + + /// If converting from Float, for monotonicity, arguments must fit in range of result type. if (WhichDataType(type).isFloat()) { + if (left.isNull() || right.isNull()) + return {}; + Float64 left_float = left.get(); Float64 right_float = right.get(); @@ -1178,18 +1170,79 @@ struct ToIntMonotonicity return {}; } - /// If signedness of type is changing, or converting from Date, DateTime, then arguments must be from same half, - /// and after conversion, resulting values must be from same half. - /// Just in case, it is required in rest of cases too. - if ((left.get() >= 0) != (right.get() >= 0) - || (T(left.get()) >= 0) != (T(right.get()) >= 0)) - return {}; + /// Integer cases. - /// If type is shrinked, then for monotonicity, all bits other than that fits, must be same. - if (divideByRangeOfType(left.get()) != divideByRangeOfType(right.get())) - return {}; + const bool from_is_unsigned = type.isValueRepresentedByUnsignedInteger(); + const bool to_is_unsigned = std::is_unsigned_v; - return { true }; + const size_t size_of_from = type.getSizeOfValueInMemory(); + const size_t size_of_to = sizeof(T); + + const bool left_in_first_half = left.isNull() + ? from_is_unsigned + : (left.get() >= 0); + + const bool right_in_first_half = right.isNull() + ? !from_is_unsigned + : (right.get() >= 0); + + /// Size of type is the same. + if (size_of_from == size_of_to) + { + if (from_is_unsigned == to_is_unsigned) + return {true, true, true}; + + if (left_in_first_half == right_in_first_half) + return {true}; + + return {}; + } + + /// Size of type is expanded. + if (size_of_from < size_of_to) + { + if (from_is_unsigned == to_is_unsigned) + return {true, true, true}; + + if (!to_is_unsigned) + return {true, true, true}; + + /// signed -> unsigned. If arguments from the same half, then function is monotonic. + if (left_in_first_half == right_in_first_half) + return {true}; + + return {}; + } + + /// Size of type is shrinked. + if (size_of_from > size_of_to) + { + /// Function cannot be monotonic on unbounded ranges. + if (left.isNull() || right.isNull()) + return {}; + + if (from_is_unsigned == to_is_unsigned) + { + /// all bits other than that fits, must be same. + if (divideByRangeOfType(left.get()) == divideByRangeOfType(right.get())) + return {true}; + + return {}; + } + else + { + /// When signedness is changed, it's also required for arguments to be from the same half. + /// And they must be in the same half after converting to the result type. + if (left_in_first_half == right_in_first_half + && (T(left.get()) >= 0) == (T(right.get()) >= 0) + && divideByRangeOfType(left.get()) == divideByRangeOfType(right.get())) + return {true}; + + return {}; + } + } + + __builtin_unreachable(); } }; diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 4d8f5edccc3..862e383c51c 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -139,7 +139,7 @@ struct IntegerRoundingComputation static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) { - if (scale > size_t(std::numeric_limits::max())) + if (sizeof(T) <= sizeof(scale) && scale > size_t(std::numeric_limits::max())) *out = 0; else *out = compute(*in, scale); @@ -331,10 +331,10 @@ public: template static NO_INLINE void applyImpl(const PaddedPODArray & in, typename ColumnVector::Container & out) { - const T* end_in = in.data() + in.size(); + const T * end_in = in.data() + in.size(); - const T* __restrict p_in = in.data(); - T* __restrict p_out = out.data(); + const T * __restrict p_in = in.data(); + T * __restrict p_out = out.data(); while (p_in < end_in) { diff --git a/dbms/src/Functions/arrayReverse.cpp b/dbms/src/Functions/arrayReverse.cpp index 19d294b97cd..c6a9248c2e4 100644 --- a/dbms/src/Functions/arrayReverse.cpp +++ b/dbms/src/Functions/arrayReverse.cpp @@ -225,7 +225,7 @@ bool FunctionArrayReverse::executeString(const IColumn & src_data, const ColumnA { size_t j_reversed = array_size - j - 1; - auto src_pos = src_array_prev_offset + j_reversed == 0 ? 0 : src_string_offsets[src_array_prev_offset + j_reversed - 1]; + auto src_pos = src_string_offsets[src_array_prev_offset + j_reversed - 1]; size_t string_size = src_string_offsets[src_array_prev_offset + j_reversed] - src_pos; memcpySmallAllowReadWriteOverflow15(&res_chars[res_string_prev_offset], &src_data[src_pos], string_size); diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index a52fe389549..f3abe782134 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -271,7 +271,7 @@ struct AggregationMethodString Arena & /*pool*/) const { return StringRef( - &(*chars)[i == 0 ? 0 : (*offsets)[i - 1]], + &(*chars)[(*offsets)[i - 1]], (i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1); } }; diff --git a/dbms/src/Interpreters/DNSCacheUpdater.cpp b/dbms/src/Interpreters/DNSCacheUpdater.cpp index a4cc8a19dad..2a2d772ffb3 100644 --- a/dbms/src/Interpreters/DNSCacheUpdater.cpp +++ b/dbms/src/Interpreters/DNSCacheUpdater.cpp @@ -61,7 +61,7 @@ DNSCacheUpdater::DNSCacheUpdater(Context & context_) task_handle = pool.addTask([this] () { return run(); }); } -bool DNSCacheUpdater::run() +BackgroundProcessingPoolTaskResult DNSCacheUpdater::run() { /// TODO: Ensusre that we get global counter (not thread local) auto num_current_network_exceptions = ProfileEvents::global_counters[ProfileEvents::NetworkErrors].load(std::memory_order_relaxed); @@ -79,20 +79,20 @@ bool DNSCacheUpdater::run() last_num_network_erros = num_current_network_exceptions; last_update_time = time(nullptr); - return true; + return BackgroundProcessingPoolTaskResult::SUCCESS; } catch (...) { /// Do not increment ProfileEvents::NetworkErrors twice if (isNetworkError()) - return false; + return BackgroundProcessingPoolTaskResult::ERROR; throw; } } /// According to BackgroundProcessingPool logic, if task has done work, it could be executed again immediately. - return false; + return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO; } DNSCacheUpdater::~DNSCacheUpdater() diff --git a/dbms/src/Interpreters/DNSCacheUpdater.h b/dbms/src/Interpreters/DNSCacheUpdater.h index ad57f37b5f6..885bcc143e3 100644 --- a/dbms/src/Interpreters/DNSCacheUpdater.h +++ b/dbms/src/Interpreters/DNSCacheUpdater.h @@ -11,6 +11,7 @@ namespace DB class Context; class BackgroundProcessingPool; class BackgroundProcessingPoolTaskInfo; +enum class BackgroundProcessingPoolTaskResult; /// Add a task to BackgroundProcessingPool that watch for ProfileEvents::NetworkErrors and updates DNS cache if it has increased @@ -25,7 +26,7 @@ public: static bool incrementNetworkErrorEventsIfNeeded(); private: - bool run(); + BackgroundProcessingPoolTaskResult run(); Context & context; BackgroundProcessingPool & pool; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index e488aa5d81b..aa19a314634 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -337,56 +337,6 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & } } -bool ExpressionAnalyzer::isThereArrayJoin(const ASTPtr & ast) -{ - if (typeid_cast(ast.get())) - { - return false; - } - else if (ASTFunction * node = typeid_cast(ast.get())) - { - if (node->name == "arrayJoin") - { - return true; - } - if (functionIsInOrGlobalInOperator(node->name)) - { - return isThereArrayJoin(node->arguments->children.at(0)); - } - if (node->name == "indexHint") - { - return false; - } - if (AggregateFunctionFactory::instance().isAggregateFunctionName(node->name)) - { - return false; - } - for (auto & child : node->arguments->children) - { - if (isThereArrayJoin(child)) - { - return true; - } - } - return false; - } - else if (typeid_cast(ast.get())) - { - return false; - } - else - { - for (auto & child : ast->children) - { - if (isThereArrayJoin(child)) - { - return true; - } - } - return false; - } -} - void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts) { @@ -1124,9 +1074,4 @@ void ExpressionAnalyzer::collectUsedColumns() } -Names ExpressionAnalyzer::getRequiredSourceColumns() const -{ - return source_columns.getNames(); -} - } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index ba1e3a252d0..45944c48c85 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -150,7 +150,7 @@ public: /** Get a set of columns that are enough to read from the table to evaluate the expression. * Columns added from another table by JOIN are not counted. */ - Names getRequiredSourceColumns() const; + Names getRequiredSourceColumns() const { return source_columns.getNames(); } /** These methods allow you to build a chain of transformations over a block, that receives values in the desired sections of the query. * @@ -244,8 +244,6 @@ private: void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const; - bool isThereArrayJoin(const ASTPtr & ast); - /// If ast is ASTSelectQuery with JOIN, add actions for JOIN key columns. void getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, ExpressionActionsPtr & actions); diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 059eb3de2f1..e7bc3b1dfef 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -75,7 +75,7 @@ struct JoinKeyGetterString const Sizes &) const { return StringRef( - &(*chars)[i == 0 ? 0 : (*offsets)[i - 1]], + &(*chars)[(*offsets)[i - 1]], (i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1); } diff --git a/dbms/src/Interpreters/SetVariants.h b/dbms/src/Interpreters/SetVariants.h index 0bb5cd6a32b..8023282a331 100644 --- a/dbms/src/Interpreters/SetVariants.h +++ b/dbms/src/Interpreters/SetVariants.h @@ -86,7 +86,7 @@ struct SetMethodString const Sizes &) const { return StringRef( - &(*chars)[i == 0 ? 0 : (*offsets)[i - 1]], + &(*chars)[(*offsets)[i - 1]], (i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1); } }; diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp index 6d81d714358..c0911ac4d5e 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -25,6 +25,7 @@ namespace DB static constexpr double thread_sleep_seconds = 10; static constexpr double thread_sleep_seconds_random_part = 1.0; +static constexpr double thread_sleep_seconds_if_nothing_to_do = 0.1; /// For exponential backoff. static constexpr double task_sleep_seconds_when_no_work_min = 10; @@ -146,7 +147,7 @@ void BackgroundProcessingPool::threadFunction() while (!shutdown) { - bool done_work = false; + TaskResult task_result = TaskResult::ERROR; TaskHandle task; try @@ -198,7 +199,7 @@ void BackgroundProcessingPool::threadFunction() { CurrentMetrics::Increment metric_increment{CurrentMetrics::BackgroundPoolTask}; - done_work = task->function(); + task_result = task->function(); } } catch (...) @@ -216,7 +217,7 @@ void BackgroundProcessingPool::threadFunction() if (task->removed) continue; - if (done_work) + if (task_result == TaskResult::SUCCESS) task->count_no_work_done = 0; else ++task->count_no_work_done; @@ -225,11 +226,13 @@ void BackgroundProcessingPool::threadFunction() /// If not, add delay before next run. Poco::Timestamp next_time_to_execute; /// current time - if (!done_work) + if (task_result == TaskResult::ERROR) next_time_to_execute += 1000000 * (std::min( task_sleep_seconds_when_no_work_max, task_sleep_seconds_when_no_work_min * std::pow(task_sleep_seconds_when_no_work_multiplier, task->count_no_work_done)) + std::uniform_real_distribution(0, task_sleep_seconds_when_no_work_random_part)(rng)); + else if (task_result == TaskResult::NOTHING_TO_DO) + next_time_to_execute += 1000000 * thread_sleep_seconds_if_nothing_to_do; tasks.erase(task->iterator); task->iterator = tasks.emplace(next_time_to_execute, task); diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h index 5e3c29e1af4..4eb5d4cce56 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h @@ -21,6 +21,12 @@ namespace DB class BackgroundProcessingPool; class BackgroundProcessingPoolTaskInfo; +enum class BackgroundProcessingPoolTaskResult +{ + SUCCESS, + ERROR, + NOTHING_TO_DO, +}; /** Using a fixed number of threads, perform an arbitrary number of tasks in an infinite loop. * In this case, one task can run simultaneously from different threads. * Designed for tasks that perform continuous background work (for example, merge). @@ -31,11 +37,13 @@ class BackgroundProcessingPool { public: /// Returns true, if some useful work was done. In that case, thread will not sleep before next run of this task. - using Task = std::function; + using TaskResult = BackgroundProcessingPoolTaskResult; + using Task = std::function; using TaskInfo = BackgroundProcessingPoolTaskInfo; using TaskHandle = std::shared_ptr; + BackgroundProcessingPool(int size_); size_t getNumberOfThreads() const diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 6ee1e7ca9c9..59e7d7b7d2c 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -581,13 +581,13 @@ bool StorageMergeTree::tryMutatePart() } -bool StorageMergeTree::backgroundTask() +BackgroundProcessingPoolTaskResult StorageMergeTree::backgroundTask() { if (shutdown_called) - return false; + return BackgroundProcessingPoolTaskResult::ERROR; if (merger_mutator.actions_blocker.isCancelled()) - return false; + return BackgroundProcessingPoolTaskResult::ERROR; try { @@ -601,16 +601,19 @@ bool StorageMergeTree::backgroundTask() ///TODO: read deduplicate option from table config if (merge(false /*aggressive*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/)) - return true; + return BackgroundProcessingPoolTaskResult::SUCCESS; - return tryMutatePart(); + if (tryMutatePart()) + return BackgroundProcessingPoolTaskResult::SUCCESS; + else + return BackgroundProcessingPoolTaskResult::ERROR; } catch (Exception & e) { if (e.code() == ErrorCodes::ABORTED) { LOG_INFO(log, e.message()); - return false; + return BackgroundProcessingPoolTaskResult::ERROR; } throw; diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index c80c06a9758..c0a2b8b5463 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -137,7 +137,7 @@ private: /// Try and find a single part to mutate and mutate it. If some part was successfully mutated, return true. bool tryMutatePart(); - bool backgroundTask(); + BackgroundProcessingPoolTaskResult backgroundTask(); Int64 getCurrentMutationVersion( const MergeTreeData::DataPartPtr & part, diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index afe8cbc02ab..a31a4340205 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2053,13 +2053,13 @@ void StorageReplicatedMergeTree::mutationsUpdatingTask() } -bool StorageReplicatedMergeTree::queueTask() +BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask() { /// If replication queue is stopped exit immediately as we successfully executed the task if (queue.actions_blocker.isCancelled()) { std::this_thread::sleep_for(std::chrono::milliseconds(5)); - return true; + return BackgroundProcessingPoolTaskResult::SUCCESS; } /// This object will mark the element of the queue as running. @@ -2077,7 +2077,7 @@ bool StorageReplicatedMergeTree::queueTask() LogEntryPtr & entry = selected.first; if (!entry) - return false; + return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO; time_t prev_attempt_time = entry->last_attempt_time; @@ -2125,7 +2125,7 @@ bool StorageReplicatedMergeTree::queueTask() bool need_sleep = !res && (entry->last_attempt_time - prev_attempt_time < 10); /// If there was no exception, you do not need to sleep. - return !need_sleep; + return need_sleep ? BackgroundProcessingPoolTaskResult::ERROR : BackgroundProcessingPoolTaskResult::SUCCESS; } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 753be7f088b..37566ddd1cf 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -427,7 +427,7 @@ private: /** Performs actions from the queue. */ - bool queueTask(); + BackgroundProcessingPoolTaskResult queueTask(); /// Postcondition: /// either leader_election is fully initialized (node in ZK is created and the watching thread is launched) diff --git a/dbms/tests/performance/consistent_hashes/consistent_hashes.xml b/dbms/tests/performance/functions_hashing/consistent_hashes.xml similarity index 100% rename from dbms/tests/performance/consistent_hashes/consistent_hashes.xml rename to dbms/tests/performance/functions_hashing/consistent_hashes.xml diff --git a/dbms/tests/performance/functions_hashing/cryptographic_hashes.xml b/dbms/tests/performance/functions_hashing/cryptographic_hashes.xml new file mode 100644 index 00000000000..5dffe4e0cec --- /dev/null +++ b/dbms/tests/performance/functions_hashing/cryptographic_hashes.xml @@ -0,0 +1,50 @@ + + cryptographic_hashes + once + + + + 10000 + + + 5000 + 20000 + + + + + + + + + + crypto_hash_func + + MD5 + SHA1 + SHA224 + SHA256 + halfMD5 + sipHash64 + sipHash128 + + + + string + + materialize('') + toString(1000000000+number) + materialize('Lorem ipsum dolor sit amet, consectetur adipiscing elit. Mauris sollicitudin nisi ac erat mollis dapibus. Maecenas leo purus, bibendum eu erat eget, iaculis molestie tortor. Phasellus maximus odio nec mauris ultrices dictum. Morbi efficitur nisl eget congue mollis. Vestibulum pharetra diam vitae urna interdum, eget ultricies justo sollicitudin. Nunc sit amet purus id leo tempus dignissim. Donec ac lacus ut orci tempus scelerisque quis ultricies nibh. Nullam lobortis, erat ac ullamcorper interdum, odio nisl elementum quam, ut malesuada massa nunc eget quam. Nam suscipit neque quis sapien ultricies imperdiet. Maecenas augue libero, finibus tristique sagittis et, semper nec arcu. Morbi non tortor ultrices, sollicitudin justo sed, accumsan ligula. Nullam at ipsum in nibh auctor ullamcorper. Nullam laoreet neque id lorem condimentum tincidunt. Nullam vel orci nibh. Ut sit amet sem faucibus, fringilla orci at, lacinia enim. Mauris imperdiet ex id scelerisque eleifend. Ut tincidunt massa nibh, viverra pharetra metus') + + + + table + + numbers + numbers_mt + + + + + SELECT ignore({crypto_hash_func}({string})) FROM system.{table} LIMIT 10000000 + diff --git a/dbms/tests/performance/functions_hashing/general_purpose_hashes.xml b/dbms/tests/performance/functions_hashing/general_purpose_hashes.xml new file mode 100644 index 00000000000..3469fcc4969 --- /dev/null +++ b/dbms/tests/performance/functions_hashing/general_purpose_hashes.xml @@ -0,0 +1,55 @@ + + general_purpose_hashes + once + + + + 10000 + + + 3000 + 20000 + + + + + + + + + + gp_hash_func + + cityHash64 + farmHash64 + metroHash64 + murmurHash2_32 + murmurHash2_64 + murmurHash3_32 + murmurHash3_64 + murmurHash3_128 + javaHash + hiveHash + xxHash32 + xxHash64 + + + + string + + materialize('') + toString(1000000000+number) + materialize('Lorem ipsum dolor sit amet, consectetur adipiscing elit. Mauris sollicitudin nisi ac erat mollis dapibus. Maecenas leo purus, bibendum eu erat eget, iaculis molestie tortor. Phasellus maximus odio nec mauris ultrices dictum. Morbi efficitur nisl eget congue mollis. Vestibulum pharetra diam vitae urna interdum, eget ultricies justo sollicitudin. Nunc sit amet purus id leo tempus dignissim. Donec ac lacus ut orci tempus scelerisque quis ultricies nibh. Nullam lobortis, erat ac ullamcorper interdum, odio nisl elementum quam, ut malesuada massa nunc eget quam. Nam suscipit neque quis sapien ultricies imperdiet. Maecenas augue libero, finibus tristique sagittis et, semper nec arcu. Morbi non tortor ultrices, sollicitudin justo sed, accumsan ligula. Nullam at ipsum in nibh auctor ullamcorper. Nullam laoreet neque id lorem condimentum tincidunt. Nullam vel orci nibh. Ut sit amet sem faucibus, fringilla orci at, lacinia enim. Mauris imperdiet ex id scelerisque eleifend. Ut tincidunt massa nibh, viverra pharetra metus') + + + + table + + numbers + numbers_mt + + + + + SELECT ignore({gp_hash_func}({string})) FROM system.{table} LIMIT 10000000 + diff --git a/dbms/tests/performance/leftpad/leftpad.xml b/dbms/tests/performance/leftpad/leftpad.xml new file mode 100644 index 00000000000..0dcb09cbbb8 --- /dev/null +++ b/dbms/tests/performance/leftpad/leftpad.xml @@ -0,0 +1,37 @@ + + left pad test + + + string + + + + hashfile + + + loop + + + + 5 + 10000 + + + 50 + 60000 + + + + + + + + + + + + + + + + diff --git a/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh b/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh index 325f19dc9ec..db533844fe4 100755 --- a/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh +++ b/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh @@ -38,7 +38,7 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO test.unsigned_integer_test_table VALUE ${CLICKHOUSE_CLIENT} --query="INSERT INTO test.enum_test_table VALUES ('hello'), ('world'), ('world'), ('yandex'), ('clickhouse'), ('clickhouse');" ${CLICKHOUSE_CLIENT} --query="INSERT INTO test.date_test_table VALUES (1), (2), (2), (256), (257), (257);" -export CLICKHOUSE_CLIENT=`echo ${CLICKHOUSE_CLIENT} |sed 's/'"${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/debug/g'` +export CLICKHOUSE_CLIENT=`echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g'` ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: String -> UInt64" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.fixed_string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: FixedString -> UInt64" @@ -52,7 +52,7 @@ ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.signed_integer_test_table ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> Int64" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> UInt64" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> Int32" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: UInt32 -> Int32" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> UInt32" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "monotonic int case: UInt32 -> Int16" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "monotonic int case: UInt32 -> UInt16" @@ -68,7 +68,7 @@ ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.enum_test_table WHERE toU ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> Int32" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> UInt32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> Int16" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Date -> Int16" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> UInt16" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Date -> Int8" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Date -> UInt8" diff --git a/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.reference b/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.reference deleted file mode 100644 index a0f1f03d797..00000000000 --- a/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.reference +++ /dev/null @@ -1 +0,0 @@ -5483925 diff --git a/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.sql b/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.sql deleted file mode 100644 index 3ed06b0facb..00000000000 --- a/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT count() FROM test.mouse_clicks diff --git a/docs/en/data_types/array.md b/docs/en/data_types/array.md index 5e2b2f55127..552c549b622 100644 --- a/docs/en/data_types/array.md +++ b/docs/en/data_types/array.md @@ -48,7 +48,7 @@ SELECT ## Working with data types -When creating an array on the fly, ClickHouse automatically defines the argument type as the narrowest data type that can store all the listed arguments. If there are any [NULL](../query_language/syntax.md#null-literal) or [Nullable](nullable.md#data_type-nullable) type arguments, the type of array elements is [Nullable](nullable.md). +When creating an array on the fly, ClickHouse automatically defines the argument type as the narrowest data type that can store all the listed arguments. If there are any [NULL](../query_language/syntax.md#null-literal) or [Nullable](nullable.md#data_type-nullable) type arguments, the type of array elements is [Nullable](nullable.md). If ClickHouse couldn't determine the data type, it will generate an exception. For instance, this will happen when trying to create an array with strings and numbers simultaneously (`SELECT array(1, 'a')`). diff --git a/docs/en/data_types/enum.md b/docs/en/data_types/enum.md index 5ba82e71ec5..754b6651f56 100644 --- a/docs/en/data_types/enum.md +++ b/docs/en/data_types/enum.md @@ -8,7 +8,7 @@ Includes the `Enum8` and `Enum16` types. `Enum` saves the finite set of pairs of ## Usage examples -Here we create a table with an `Enum8('hello' = 1, 'world' = 2)` type column: +Here we create a table with an `Enum8('hello' = 1, 'world' = 2)` type column: ``` CREATE TABLE t_enum diff --git a/docs/en/data_types/float.md b/docs/en/data_types/float.md index 894b50d8237..1c908819c1f 100644 --- a/docs/en/data_types/float.md +++ b/docs/en/data_types/float.md @@ -5,7 +5,7 @@ Types are equivalent to types of C: - `Float32` - `float` -- `Float64` - `double` +- `Float64` - `double` We recommend that you store data in integer form whenever possible. For example, convert fixed precision numbers to integer values, such as monetary amounts or page load times in milliseconds. diff --git a/docs/en/data_types/nested_data_structures/aggregatefunction.md b/docs/en/data_types/nested_data_structures/aggregatefunction.md index 4a02c5c7ad3..95ed4d751e6 100644 --- a/docs/en/data_types/nested_data_structures/aggregatefunction.md +++ b/docs/en/data_types/nested_data_structures/aggregatefunction.md @@ -40,7 +40,7 @@ quantilesState(0.5, 0.9)(SendTiming) In contrast to the corresponding functions `uniq` and `quantiles`, `-State`- functions return the state, instead the final value. In other words, they return a value of `AggregateFunction` type. -In the results of `SELECT` query the values of `AggregateFunction` type have implementation-specific binary representation for all of the ClickHouse output formats. If dump data into, for example, `TabSeparated` format with `SELECT` query then this dump can be loaded back using `INSERT` query. +In the results of `SELECT` query the values of `AggregateFunction` type have implementation-specific binary representation for all of the ClickHouse output formats. If dump data into, for example, `TabSeparated` format with `SELECT` query then this dump can be loaded back using `INSERT` query. ### Data Selection diff --git a/docs/en/development/style.md b/docs/en/development/style.md index edfda73a7eb..1af7814a97f 100644 --- a/docs/en/development/style.md +++ b/docs/en/development/style.md @@ -44,7 +44,7 @@ void reinsert(const Value & x) memcpy(&buf[place_value], &x, sizeof(x)); ``` -**6.** In `if`, `for`, `while` and other expressions, a space is inserted in front of the opening bracket (as opposed to function calls). +**6.** In `if`, `for`, `while` and other expressions, a space is inserted in front of the opening bracket (as opposed to function calls). ```cpp for (size_t i = 0; i < rows; i += storage.index_granularity) @@ -93,7 +93,7 @@ struct AggregatedStatElement {} ``` -**15.** In classes and structures, write `public`, `private`, and `protected` on the same level as `class/struct`, and indent the rest of the code. +**15.** In classes and structures, write `public`, `private`, and `protected` on the same level as `class/struct`, and indent the rest of the code. ```cpp template @@ -261,7 +261,7 @@ The example is borrowed from the resource [http://home.tamk.fi/~jaalto/course/co **7.** Do not write garbage comments (author, creation date ..) at the beginning of each file. -**8.** Single-line comments begin with three slashes: `///` and multi-line comments begin with `/**`. These comments are considered "documentation". +**8.** Single-line comments begin with three slashes: `///` and multi-line comments begin with `/**`. These comments are considered "documentation". Note: You can use Doxygen to generate documentation from these comments. But Doxygen is not generally used because it is more convenient to navigate the code in the IDE. @@ -299,7 +299,7 @@ Note: You can use Doxygen to generate documentation from these comments. But Dox ## Names -**1.** Use lowercase letters with underscores in the names of variables and class members. +**1.** Use lowercase letters with underscores in the names of variables and class members. ```cpp size_t max_block_size; @@ -349,7 +349,7 @@ In all other cases, use a name that describes the meaning. bool info_successfully_loaded = false; ``` -**9.** Names of `define`s and global constants use ALL_CAPS with underscores. +**9.** Names of `define`s and global constants use ALL_CAPS with underscores. ```cpp #define MAX_SRC_TABLE_NAMES_TO_STORE 1000 diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index d9a44f78ea3..2e2e1660270 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -15,7 +15,7 @@ To run all tests, use `dbms/tests/clickhouse-test` tool. Look `--help` for the l The most simple way to invoke functional tests is to copy `clickhouse-client` to `/usr/bin/`, run `clickhouse-server` and then run `./clickhouse-test` from its own directory. -To add new test, create a `.sql` or `.sh` file in `dbms/src/tests/queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client -n --testmode < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. +To add new test, create a `.sql` or `.sh` file in `dbms/src/tests/queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client -n --testmode < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. Tests should use (create, drop, etc) only tables in `test` database that is assumed to be created beforehand; also tests can use temporary tables. diff --git a/docs/en/getting_started/example_datasets/ontime.md b/docs/en/getting_started/example_datasets/ontime.md index bd77e838e90..c4dcaf9575e 100644 --- a/docs/en/getting_started/example_datasets/ontime.md +++ b/docs/en/getting_started/example_datasets/ontime.md @@ -166,7 +166,7 @@ SELECT Origin, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Q4. The number of delays by carrier for 2007 ``` sql -SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC +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 diff --git a/docs/en/getting_started/example_datasets/star_schema.md b/docs/en/getting_started/example_datasets/star_schema.md index 5606cf71dd1..a6281b88b92 100644 --- a/docs/en/getting_started/example_datasets/star_schema.md +++ b/docs/en/getting_started/example_datasets/star_schema.md @@ -10,7 +10,7 @@ make There will be some warnings during the process, but this is normal. -Place `dbgen` and `dists.dss` in any location with 800 GB of free disk space. +Place `dbgen` and `dists.dss` in any location with 800 GB of free disk space. Generating data: @@ -73,7 +73,7 @@ CREATE TABLE partd AS part ENGINE = Distributed(perftest_3shards_1replicas, defa ``` For testing on a single server, just use MergeTree tables. -For distributed testing, you need to configure the `perftest_3shards_1replicas` cluster in the config file. +For distributed testing, you need to configure the `perftest_3shards_1replicas` cluster in the config file. Next, create MergeTree tables on each server and a Distributed above them. Downloading data (change 'customer' to 'customerd' in the distributed version): diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index caffafe61ca..d665fbd3e17 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -79,7 +79,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va ### Command Line Options -- `--host, -h` -– The server name, 'localhost' by default. You can use either the name or the IPv4 or IPv6 address. +- `--host, -h` -– The server name, 'localhost' by default. You can use either the name or the IPv4 or IPv6 address. - `--port` – The port to connect to. Default value: 9000. Note that the HTTP interface and the native interface use different ports. - `--user, -u` – The username. Default value: default. - `--password` – The password. Default value: empty string. @@ -91,11 +91,11 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--vertical, -E` – If specified, use the Vertical format by default to output the result. This is the same as '--format=Vertical'. In this format, each value is printed on a separate line, which is helpful when displaying wide tables. - `--time, -t` – If specified, print the query execution time to 'stderr' in non-interactive mode. - `--stacktrace` – If specified, also print the stack trace if an exception occurs. -- `-config-file` – The name of the configuration file. +- `--config-file` – The name of the configuration file. ### Configuration Files -`clickhouse-client` uses the first existing file of the following: +`clickhouse-client` uses the first existing file of the following: - Defined in the `-config-file` parameter. - `./clickhouse-client.xml` diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 83a0a320f18..c47e75142df 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -74,7 +74,7 @@ During a read operation, incorrect dates and dates with times can be parsed with 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: +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 @@ -242,7 +242,7 @@ SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTA } ``` -The JSON is compatible with JavaScript. To ensure this, some characters are additionally escaped: the slash `/` is escaped as `\/`; alternative line breaks `U+2028` and `U+2029`, which break some browsers, are escaped as `\uXXXX`. ASCII control characters are escaped: backspace, form feed, line feed, carriage return, and horizontal tab are replaced with `\b`, `\f`, `\n`, `\r`, `\t` , as well as the remaining bytes in the 00-1F range using `\uXXXX` sequences. Invalid UTF-8 sequences are changed to the replacement character � so the output text will consist of valid UTF-8 sequences. For compatibility with JavaScript, Int64 and UInt64 integers are enclosed in double quotes by default. To remove the quotes, you can set the configuration parameter output_format_json_quote_64bit_integers to 0. +The JSON is compatible with JavaScript. To ensure this, some characters are additionally escaped: the slash `/` is escaped as `\/`; alternative line breaks `U+2028` and `U+2029`, which break some browsers, are escaped as `\uXXXX`. ASCII control characters are escaped: backspace, form feed, line feed, carriage return, and horizontal tab are replaced with `\b`, `\f`, `\n`, `\r`, `\t` , as well as the remaining bytes in the 00-1F range using `\uXXXX` sequences. Invalid UTF-8 sequences are changed to the replacement character � so the output text will consist of valid UTF-8 sequences. For compatibility with JavaScript, Int64 and UInt64 integers are enclosed in double quotes by default. To remove the quotes, you can set the configuration parameter output_format_json_quote_64bit_integers to 0. `rows` – The total number of output rows. @@ -255,7 +255,7 @@ If the query contains GROUP BY, rows_before_limit_at_least is the exact number o This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). -ClickHouse supports [NULL](../query_language/syntax.md), which is displayed as `null` in the JSON output. +ClickHouse supports [NULL](../query_language/syntax.md), which is displayed as `null` in the JSON output. See also the JSONEachRow format. diff --git a/docs/en/introduction/ya_metrika_task.md b/docs/en/introduction/ya_metrika_task.md index 9acbdc0989e..41b33eff581 100644 --- a/docs/en/introduction/ya_metrika_task.md +++ b/docs/en/introduction/ya_metrika_task.md @@ -1,6 +1,6 @@ # Yandex.Metrica Use Case -ClickHouse was originally developed to power [Yandex.Metrica](https://metrica.yandex.com/), [the second largest web analytics platform in the world](http://w3techs.com/technologies/overview/traffic_analysis/all), and continues to be the core component of this system. With more than 13 trillion records in the database and more than 20 billion events daily, ClickHouse allows generating custom reports on the fly directly from non-aggregated data. This article briefly covers the goals of ClickHouse in the early stages of its development. +ClickHouse was originally developed to power [Yandex.Metrica](https://metrica.yandex.com/), [the second largest web analytics platform in the world](http://w3techs.com/technologies/overview/traffic_analysis/all), and continues to be the core component of this system. With more than 13 trillion records in the database and more than 20 billion events daily, ClickHouse allows generating custom reports on the fly directly from non-aggregated data. This article briefly covers the goals of ClickHouse in the early stages of its development. Yandex.Metrica builds customized reports on the fly based on hits and sessions, with arbitrary segments defined by the user. This often requires building complex aggregates, such as the number of unique users. New data for building a report is received in real time. diff --git a/docs/en/operations/configuration_files.md b/docs/en/operations/configuration_files.md index 43a062e262f..2184bb95122 100644 --- a/docs/en/operations/configuration_files.md +++ b/docs/en/operations/configuration_files.md @@ -12,7 +12,7 @@ If `replace` is specified, it replaces the entire element with the specified one If `remove` is specified, it deletes the element. -The config can also define "substitutions". If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the [include_from](server_settings/settings.md#server_settings-include_from) element in the server config. The substitution values are specified in `/yandex/substitution_name` elements in this file. If a substitution specified in `incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional="true"` attribute (for example, settings for [macros](server_settings/settings.md)). +The config can also define "substitutions". If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the [include_from](server_settings/settings.md#server_settings-include_from) element in the server config. The substitution values are specified in `/yandex/substitution_name` elements in this file. If a substitution specified in `incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional="true"` attribute (for example, settings for [macros](server_settings/settings.md)). Substitutions can also be performed from ZooKeeper. To do this, specify the attribute `from_zk = "/path/to/node"`. The element value is replaced with the contents of the node at `/path/to/node` in ZooKeeper. You can also put an entire XML subtree on the ZooKeeper node and it will be fully inserted into the source element. diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index 6a7ff2c0e52..76fe4a62bb1 100644 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -42,7 +42,7 @@ Block field ``: - ``min_part_size_ratio`` – The ratio of the minimum size of a table part to the full size of the table. - ``method`` – Compression method. Acceptable values ​: ``lz4`` or ``zstd``(experimental). -ClickHouse checks `min_part_size` and `min_part_size_ratio` and processes the `case` blocks that match these conditions. If none of the `` matches, ClickHouse applies the `lz4` compression algorithm. +ClickHouse checks `min_part_size` and `min_part_size_ratio` and processes the `case` blocks that match these conditions. If none of the `` matches, ClickHouse applies the `lz4` compression algorithm. **Example** @@ -456,7 +456,7 @@ Keys for server/client settings: - requireTLSv1 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. - fips – Activates OpenSSL FIPS mode. Supported if the library's OpenSSL version supports FIPS. - privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ````, ``KeyFileHandler``, ``test``, ````. -- invalidCertificateHandler – Class (subclass of CertificateHandler) for verifying invalid certificates. For example: `` ConsoleCertificateHandler `` . +- invalidCertificateHandler – Class (subclass of CertificateHandler) for verifying invalid certificates. For example: `` ConsoleCertificateHandler `` . - disableProtocols – Protocols that are not allowed to use. - preferServerCiphers – Preferred server ciphers on the client. @@ -641,7 +641,7 @@ The uncompressed cache is advantageous for very short queries in individual case ## user_files_path {#server_settings-user_files_path} -The directory with user files. Used in the table function [file()](../../query_language/table_functions/file.md). +The directory with user files. Used in the table function [file()](../../query_language/table_functions/file.md). **Example** diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 59ba8189261..add73212c5d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -28,7 +28,7 @@ Forces a query to an out-of-date replica if updated data is not available. See " ClickHouse selects the most relevant from the outdated replicas of the table. -Used when performing `SELECT` from a distributed table that points to replicated tables. +Used when performing `SELECT` from a distributed table that points to replicated tables. By default, 1 (enabled). @@ -38,7 +38,7 @@ Disables query execution if the index can't be used by date. Works with tables in the MergeTree family. -If `force_index_by_date=1`, ClickHouse checks whether the query has a date key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For example, the condition `Date != ' 2000-01-01 '` is acceptable even when it matches all the data in the table (i.e., running the query requires a full scan). For more information about ranges of data in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". +If `force_index_by_date=1`, ClickHouse checks whether the query has a date key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For example, the condition `Date != ' 2000-01-01 '` is acceptable even when it matches all the data in the table (i.e., running the query requires a full scan). For more information about ranges of data in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". ## force_primary_key @@ -47,7 +47,7 @@ Disables query execution if indexing by the primary key is not possible. Works with tables in the MergeTree family. -If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For more information about data ranges in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". +If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For more information about data ranges in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". ## fsync_metadata @@ -141,7 +141,7 @@ Sets the time in seconds. If a replica lags more than the set value, this replic Default value: 0 (off). -Used when performing `SELECT` from a distributed table that points to replicated tables. +Used when performing `SELECT` from a distributed table that points to replicated tables. ## max_threads {#settings-max_threads} @@ -267,7 +267,7 @@ This parameter is useful when you are using formats that require a schema defini ## stream_flush_interval_ms -Works for tables with streaming in the case of a timeout, or when a thread generates[max_insert_block_size](#settings-max_insert_block_size) rows. +Works for tables with streaming in the case of a timeout, or when a thread generates [max_insert_block_size](#settings-max_insert_block_size) rows. The default value is 7500. @@ -326,7 +326,7 @@ Replica lag is not controlled. Enable compilation of queries. By default, 0 (disabled). Compilation is only used for part of the query-processing pipeline: for the first stage of aggregation (GROUP BY). -If this portion of the pipeline was compiled, the query may run faster due to deployment of short cycles and inlining aggregate function calls. The maximum performance improvement (up to four times faster in rare cases) is seen for queries with multiple simple aggregate functions. Typically, the performance gain is insignificant. In very rare cases, it may slow down query execution. +If this portion of the pipeline was compiled, the query may run faster due to deployment of short cycles and inlining aggregate function calls. The maximum performance improvement (up to four times faster in rare cases) is seen for queries with multiple simple aggregate functions. Typically, the performance gain is insignificant. In very rare cases, it may slow down query execution. ## min_count_to_compile @@ -344,7 +344,7 @@ It works for JSONEachRow and TSKV formats. ## output_format_json_quote_64bit_integers -If the value is true, integers appear in quotes when using JSON\* Int64 and UInt64 formats (for compatibility with most JavaScript implementations); otherwise, integers are output without the quotes. +If the value is true, integers appear in quotes when using JSON\* Int64 and UInt64 formats (for compatibility with most JavaScript implementations); otherwise, integers are output without the quotes. ## format_csv_delimiter {#settings-format_csv_delimiter} @@ -373,7 +373,7 @@ The default value is 0. All the replicas in the quorum are consistent, i.e., they contain data from all previous `INSERT` queries. The `INSERT` sequence is linearized. -When reading the data written from the `insert_quorum`, you can use the[select_sequential_consistency](#select-sequential-consistency) option. +When reading the data written from the `insert_quorum`, you can use the [select_sequential_consistency](#select-sequential-consistency) option. **ClickHouse generates an exception** diff --git a/docs/en/operations/settings/settings_profiles.md b/docs/en/operations/settings/settings_profiles.md index 3953ada8a83..c335e249212 100644 --- a/docs/en/operations/settings/settings_profiles.md +++ b/docs/en/operations/settings/settings_profiles.md @@ -2,7 +2,7 @@ # Settings profiles A settings profile is a collection of settings grouped under the same name. Each ClickHouse user has a profile. -To apply all the settings in a profile, set the `profile` setting. +To apply all the settings in a profile, set the `profile` setting. Example: @@ -58,9 +58,9 @@ Example: ``` -The example specifies two profiles: `default` and `web`. The `default` profile has a special purpose: it must always be present and is applied when starting the server. In other words, the `default` profile contains default settings. The `web` profile is a regular profile that can be set using the `SET` query or using a URL parameter in an HTTP query. +The example specifies two profiles: `default` and `web`. The `default` profile has a special purpose: it must always be present and is applied when starting the server. In other words, the `default` profile contains default settings. The `web` profile is a regular profile that can be set using the `SET` query or using a URL parameter in an HTTP query. -Settings profiles can inherit from each other. To use inheritance, indicate the `profile` setting before the other settings that are listed in the 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 b41f7f02b95..d15d392d5f9 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -141,7 +141,7 @@ Formats: - active (UInt8) – Indicates whether the part is active. If a part is active, it is used in a table; otherwise, it will be deleted. Inactive data parts remain after merging. -- marks (UInt64) – The number of marks. To get the approximate number of rows in a data part, multiply ``marks`` by the index granularity (usually 8192). +- marks (UInt64) – The number of marks. To get the approximate number of rows in a data part, multiply ``marks`` by the index granularity (usually 8192). - marks_size (UInt64) – The size of the file with marks. @@ -373,7 +373,7 @@ If the path specified in 'path' doesn't exist, an exception will be thrown. Columns: -- `name String` — The name of the node. +- `name String` — The name of the node. - `path String` — The path to the node. - `value String` — Node value. - `dataLength Int32` — Size of the value. diff --git a/docs/en/operations/table_engines/custom_partitioning_key.md b/docs/en/operations/table_engines/custom_partitioning_key.md index a674bc5533f..23e275b7925 100644 --- a/docs/en/operations/table_engines/custom_partitioning_key.md +++ b/docs/en/operations/table_engines/custom_partitioning_key.md @@ -36,7 +36,7 @@ In the `system.parts` table, the `partition` column specifies the value of the p Old: `20140317_20140323_2_2_0` (minimum date - maximum date - minimum block number - maximum block number - level). -Now: `201403_2_2_0` (partition ID - minimum block number - maximum block number - level). +Now: `201403_2_2_0` (partition ID - minimum block number - maximum block number - level). The partition ID is its string identifier (human-readable, if possible) that is used for the names of data parts in the file system and in ZooKeeper. You can specify it in ALTER queries in place of the partition key. Example: Partition key `toYYYYMM(EventDate)`; ALTER can specify either `PARTITION 201710` or `PARTITION ID '201710'`. diff --git a/docs/en/operations/table_engines/kafka.md b/docs/en/operations/table_engines/kafka.md index d668def503e..3927e472e50 100644 --- a/docs/en/operations/table_engines/kafka.md +++ b/docs/en/operations/table_engines/kafka.md @@ -24,7 +24,7 @@ Kafka SETTINGS kafka_topic_list = 'topic1,topic2', kafka_group_name = 'group1', kafka_format = 'JSONEachRow', - kafka_row_delimiter = '\n' + kafka_row_delimiter = '\n', kafka_schema = '', kafka_num_consumers = 2 ``` @@ -39,7 +39,7 @@ Required parameters: Optional parameters: - `kafka_row_delimiter` - Character-delimiter of records (rows), which ends the message. -- `kafka_schema` – An optional parameter that must be used if the format requires a schema definition. For example, [Cap'n Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object. +- `kafka_schema` – An optional parameter that must be used if the format requires a schema definition. For example, [Cap'n Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object. - `kafka_num_consumers` – The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. The total number of consumers should not exceed the number of partitions in the topic, since only one consumer can be assigned per partition. Examples: @@ -122,7 +122,7 @@ If you want to change the target table by using `ALTER`, we recommend disabling Similar to GraphiteMergeTree, the Kafka engine supports extended configuration using the ClickHouse config file. There are two configuration keys that you can use: global (`kafka`) and topic-level (`kafka_*`). The global configuration is applied first, and then the topic-level configuration is applied (if it exists). ```xml - + cgrp smallest diff --git a/docs/en/operations/table_engines/merge.md b/docs/en/operations/table_engines/merge.md index e8de53bc286..366a5459bf8 100644 --- a/docs/en/operations/table_engines/merge.md +++ b/docs/en/operations/table_engines/merge.md @@ -1,8 +1,8 @@ # Merge -The `Merge` engine (not to be confused with `MergeTree`) does not store data itself, but allows reading from any number of other tables simultaneously. +The `Merge` engine (not to be confused with `MergeTree`) does not store data itself, but allows reading from any number of other tables simultaneously. Reading is automatically parallelized. Writing to a table is not supported. When reading, the indexes of tables that are actually being read are used, if they exist. -The `Merge` engine accepts parameters: the database name and a regular expression for tables. +The `Merge` engine accepts parameters: the database name and a regular expression for tables. Example: @@ -10,17 +10,17 @@ Example: Merge(hits, '^WatchLog') ``` -Data will be read from the tables in the `hits` database that have names that match the regular expression '`^WatchLog`'. +Data will be read from the tables in the `hits` database that have names that match the regular expression '`^WatchLog`'. Instead of the database name, you can use a constant expression that returns a string. For example, `currentDatabase()`. Regular expressions — [re2](https://github.com/google/re2) (supports a subset of PCRE), case-sensitive. See the notes about escaping symbols in regular expressions in the "match" section. -When selecting tables to read, the `Merge` table itself will not be selected, even if it matches the regex. This is to avoid loops. -It is possible to create two `Merge` tables that will endlessly try to read each others' data, but this is not a good idea. +When selecting tables to read, the `Merge` table itself will not be selected, even if it matches the regex. This is to avoid loops. +It is possible to create two `Merge` tables that will endlessly try to read each others' data, but this is not a good idea. -The typical way to use the `Merge` engine is for working with a large number of `TinyLog` tables as if with a single table. +The typical way to use the `Merge` engine is for working with a large number of `TinyLog` tables as if with a single table. Example 2: @@ -63,7 +63,7 @@ Virtual columns differ from normal columns in the following ways: The `Merge` type table contains a virtual `_table` column of the `String` type. (If the table already has a `_table` column, the virtual column is called `_table1`; if you already have `_table1`, it's called `_table2`, and so on.) It contains the name of the table that data was read from. -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. +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 14e5fd19fe2..abac921f9df 100644 --- a/docs/en/operations/table_engines/mergetree.md +++ b/docs/en/operations/table_engines/mergetree.md @@ -61,7 +61,7 @@ For a description of request parameters, see [request description](../../query_l By default the primary key is the same as the sorting key (which is specified by the `ORDER BY` clause). Thus in most cases it is unnecessary to specify a separate `PRIMARY KEY` clause. -- `SAMPLE BY` — An expression for sampling. +- `SAMPLE BY` — An expression for sampling. If a sampling expression is used, the primary key must contain it. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. diff --git a/docs/en/operations/table_engines/replacingmergetree.md b/docs/en/operations/table_engines/replacingmergetree.md index a147e9c8dd6..53b9a47c5c2 100644 --- a/docs/en/operations/table_engines/replacingmergetree.md +++ b/docs/en/operations/table_engines/replacingmergetree.md @@ -4,7 +4,7 @@ The engine differs from [MergeTree](mergetree.md#table_engines-mergetree) in tha Data deduplication occurs only during a merge. Merging occurs in the background at an unknown time, so you can't plan for it. Some of the data may remain unprocessed. Although you can run an unscheduled merge using the `OPTIMIZE` query, don't count on using it, because the `OPTIMIZE` query will read and write a large amount of data. -Thus, `ReplacingMergeTree` is suitable for clearing out duplicate data in the background in order to save space, but it doesn't guarantee the absence of duplicates. +Thus, `ReplacingMergeTree` is suitable for clearing out duplicate data in the background in order to save space, but it doesn't guarantee the absence of duplicates. ## Creating a Table @@ -33,7 +33,7 @@ For a description of request parameters, see [request description](../../query_l **Query clauses** -When creating a `ReplacingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table. +When creating a `ReplacingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table.
Deprecated Method for Creating a Table diff --git a/docs/en/operations/table_engines/summingmergetree.md b/docs/en/operations/table_engines/summingmergetree.md index efea8da74be..32444b61c1a 100644 --- a/docs/en/operations/table_engines/summingmergetree.md +++ b/docs/en/operations/table_engines/summingmergetree.md @@ -1,9 +1,9 @@ # SummingMergeTree -The engine inherits from [MergeTree](mergetree.md#table_engines-mergetree). The difference is that when merging data parts for `SummingMergeTree` tables ClickHouse replaces all the rows with the same primary key (or more accurately, with the same [sorting key](mergetree.md)) with one row which contains summarized values for the columns with the numeric data type. If the sorting key is composed in a way that a single key value corresponds to large number of rows, this significantly reduces storage volume and speeds up data selection. +The engine inherits from [MergeTree](mergetree.md#table_engines-mergetree). The difference is that when merging data parts for `SummingMergeTree` tables ClickHouse replaces all the rows with the same primary key (or more accurately, with the same [sorting key](mergetree.md)) with one row which contains summarized values for the columns with the numeric data type. If the sorting key is composed in a way that a single key value corresponds to large number of rows, this significantly reduces storage volume and speeds up data selection. -We recommend to use the engine together with `MergeTree`. Store complete data in `MergeTree` table, and use `SummingMergeTree` for aggregated data storing, for example, when preparing reports. Such an approach will prevent you from losing valuable data due to an incorrectly composed primary key. +We recommend to use the engine together with `MergeTree`. Store complete data in `MergeTree` table, and use `SummingMergeTree` for aggregated data storing, for example, when preparing reports. Such an approach will prevent you from losing valuable data due to an incorrectly composed primary key. ## Creating a Table @@ -31,7 +31,7 @@ The columns must be of a numeric type and must not be in the primary key. **Query clauses** -When creating a `SummingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table. +When creating a `SummingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table.
Deprecated Method for Creating a Table @@ -72,7 +72,7 @@ Insert data to it: :) INSERT INTO summtt Values(1,1),(1,2),(2,1) ``` -ClickHouse may sum all the rows not completely ([see below](#data-processing)), so we use an aggregate function `sum` and `GROUP BY` clause in the query. +ClickHouse may sum all the rows not completely ([see below](#data-processing)), so we use an aggregate function `sum` and `GROUP BY` clause in the query. ```sql SELECT key, sum(value) FROM summtt GROUP BY key diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index bd058afb6dd..e3b66a639ba 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -18,7 +18,7 @@ You can use `turbostat` to view the CPU's actual clock rate under a load. ## CPU Scaling Governor -Always use the `performance` scaling governor. The `on-demand` scaling governor works much worse with constantly high demand. +Always use the `performance` scaling governor. The `on-demand` scaling governor works much worse with constantly high demand. ```bash sudo echo 'performance' | tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor diff --git a/docs/en/query_language/agg_functions/parametric_functions.md b/docs/en/query_language/agg_functions/parametric_functions.md index 5a49b077089..1cbe784e621 100644 --- a/docs/en/query_language/agg_functions/parametric_functions.md +++ b/docs/en/query_language/agg_functions/parametric_functions.md @@ -37,7 +37,7 @@ Pattern syntax: Any quantity of any type of events is allowed over the specified time. -Instead of `>=`, the following operators can be used:`<`, `>`, `<=`. +Instead of `>=`, the following operators can be used:`<`, `>`, `<=`. Any number may be specified in place of 1800. diff --git a/docs/en/query_language/agg_functions/reference.md b/docs/en/query_language/agg_functions/reference.md index 882371dedc0..782be3374b1 100644 --- a/docs/en/query_language/agg_functions/reference.md +++ b/docs/en/query_language/agg_functions/reference.md @@ -295,7 +295,7 @@ Uses the [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) algorithm to a The result is determinate (it doesn't depend on the order of query processing). -We don't recommend using this function. In most cases, use the `uniq` or `uniqCombined` function. +We don't recommend using this function. In most cases, use the `uniq` or `uniqCombined` function. ## uniqExact(x) @@ -382,8 +382,8 @@ For its purpose (calculating quantiles of page loading times), using this functi ## quantileTimingWeighted(level)(x, weight) -Differs from the `quantileTiming` function in that it has a second argument, "weights". Weight is a non-negative integer. -The result is calculated as if the `x` value were passed `weight` number of times to the `quantileTiming` function. +Differs from the `quantileTiming` function in that it has a second argument, "weights". Weight is a non-negative integer. +The result is calculated as if the `x` value were passed `weight` number of times to the `quantileTiming` function. ## quantileExact(level)(x) @@ -437,7 +437,7 @@ The result is equal to the square root of `varPop(x)`. Returns an array of the most frequent values in the specified column. The resulting array is sorted in descending order of frequency of values (not by the values themselves). -Implements the [ Filtered Space-Saving](http://www.l2f.inesc-id.pt/~fmmb/wiki/uploads/Work/misnis.ref0a.pdf) algorithm for analyzing TopK, based on the reduce-and-combine algorithm from [Parallel Space Saving](https://arxiv.org/pdf/1401.0702.pdf). +Implements the [ Filtered Space-Saving](http://www.l2f.inesc-id.pt/~fmmb/wiki/uploads/Work/misnis.ref0a.pdf) algorithm for analyzing TopK, based on the reduce-and-combine algorithm from [Parallel Space Saving](https://arxiv.org/pdf/1401.0702.pdf). ``` topK(N)(column) diff --git a/docs/en/query_language/alter.md b/docs/en/query_language/alter.md index 4d146f55cbf..c3d504f07bb 100644 --- a/docs/en/query_language/alter.md +++ b/docs/en/query_language/alter.md @@ -54,7 +54,7 @@ If there is a failure during one of the successive stages, data can be restored The `ALTER` query lets you create and delete separate elements (columns) in nested data structures, but not whole nested data structures. To add a nested data structure, you can add columns with a name like `name.nested_name` and the type `Array(T)`. A nested data structure is equivalent to multiple array columns with a name that has the same prefix before the dot. -There is no support for deleting columns in the primary key or the sampling key (columns that are in the `ENGINE` expression). Changing the type for columns that are included in the primary key is only possible if this change does not cause the data to be modified (for example, it is allowed to add values to an Enum or change a type with `DateTime` to `UInt32`). +There is no support for deleting columns in the primary key or the sampling key (columns that are in the `ENGINE` expression). Changing the type for columns that are included in the primary key is only possible if this change does not cause the data to be modified (for example, it is allowed to add values to an Enum or change a type with `DateTime` to `UInt32`). If the `ALTER` query is not sufficient for making the table changes you need, you can create a new table, copy the data to it using the `INSERT SELECT` query, then switch the tables using the `RENAME` query and delete the old table. @@ -115,10 +115,10 @@ Data directory: `/var/lib/clickhouse/data/database/table/`,where `/var/lib/click ```bash $ ls -l /var/lib/clickhouse/data/test/visits/ total 48 -drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_2_2_0 -drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_4_4_0 -drwxrwxrwx 2 clickhouse clickhouse 4096 May 5 02:55 detached --rw-rw-rw- 1 clickhouse clickhouse 2 May 5 02:58 increment.txt +drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_2_2_0 +drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_4_4_0 +drwxrwxrwx 2 clickhouse clickhouse 4096 May 5 02:55 detached +-rw-rw-rw- 1 clickhouse clickhouse 2 May 5 02:58 increment.txt ``` Here, `20140317_20140323_2_2_0` and ` 20140317_20140323_4_4_0` are the directories of data parts. @@ -193,7 +193,7 @@ The `ALTER ... FREEZE PARTITION` query is not replicated. A local backup is only As an alternative, you can manually copy data from the `/var/lib/clickhouse/data/database/table` directory. But if you do this while the server is running, race conditions are possible when copying directories with files being added or changed, and the backup may be inconsistent. You can do this if the server isn't running – then the resulting data will be the same as after the `ALTER TABLE t FREEZE PARTITION` query. -`ALTER TABLE ... FREEZE PARTITION` only copies data, not table metadata. To make a backup of table metadata, copy the file `/var/lib/clickhouse/metadata/database/table.sql` +`ALTER TABLE ... FREEZE PARTITION` only copies data, not table metadata. To make a backup of table metadata, copy the file `/var/lib/clickhouse/metadata/database/table.sql` To restore from a backup: @@ -216,7 +216,7 @@ Although the query is called `ALTER TABLE`, it does not change the table structu Data is placed in the `detached` directory. You can use the `ALTER TABLE ... ATTACH` query to attach the data. -The ` FROM` clause specifies the path in ` ZooKeeper`. For example, `/clickhouse/tables/01-01/visits`. +The ` FROM` clause specifies the path in ` ZooKeeper`. For example, `/clickhouse/tables/01-01/visits`. Before downloading, the system checks that the partition exists and the table structure matches. The most appropriate replica is selected automatically from the healthy replicas. The `ALTER ... FETCH PARTITION` query is not replicated. The partition will be downloaded to the 'detached' directory only on the local server. Note that if after this you use the `ALTER TABLE ... ATTACH` query to add data to the table, the data will be added on all replicas (on one of the replicas it will be added from the 'detached' directory, and on the rest it will be loaded from neighboring replicas). 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 e3b6498c9a9..03279688d6c 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_layout.md +++ b/docs/en/query_language/dicts/external_dicts_dict_layout.md @@ -95,7 +95,7 @@ Configuration example: The dictionary is stored in memory in the form of a hash table with an ordered array of ranges and their corresponding values. -This storage method works the same way as hashed and allows using date/time ranges in addition to the key, if they appear in the dictionary. +This storage method works the same way as hashed and allows using date/time ranges in addition to the key, if they appear in the dictionary. Example: The table contains discounts for each advertiser in the format: 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 4ab8ba4ea20..cbd78da16ad 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_lifetime.md +++ b/docs/en/query_language/dicts/external_dicts_dict_lifetime.md @@ -15,7 +15,7 @@ Example of settings: ``` -Setting ` 0 ` prevents updating dictionaries. +Setting ` 0 ` prevents updating dictionaries. You can set a time interval for upgrades, and ClickHouse will choose a uniformly random time within this range. This is necessary in order to distribute the load on the dictionary source when upgrading on a large number of servers. diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index 656abc1ea01..3a16db67e8c 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -243,7 +243,7 @@ This function can also be used in higher-order functions. For example, you can u ## arrayEnumerateUniq(arr, ...) Returns an array the same size as the source array, indicating for each element what its position is among elements with the same value. -For example: arrayEnumerateUniq(\[10, 20, 10, 30\]) = \[1, 1, 2, 1\]. +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: @@ -378,7 +378,7 @@ arrayPushFront(array, single_value) **Parameters** - `array` – Array. -- `single_value` – A single value. Only numbers can be added to an array with numbers, and only strings can be added to an array of strings. When adding numbers, ClickHouse automatically sets the `single_value` type for the data type of the array. For more information about the types of data in ClickHouse, see "[Data types](../../data_types/index.md#data_types)". Can be `NULL`. The function adds a `NULL` element to an array, and the type of array elements converts to `Nullable`. +- `single_value` – A single value. Only numbers can be added to an array with numbers, and only strings can be added to an array of strings. When adding numbers, ClickHouse automatically sets the `single_value` type for the data type of the array. For more information about the types of data in ClickHouse, see "[Data types](../../data_types/index.md#data_types)". Can be `NULL`. The function adds a `NULL` element to an array, and the type of array elements converts to `Nullable`. **Example** diff --git a/docs/en/query_language/functions/ext_dict_functions.md b/docs/en/query_language/functions/ext_dict_functions.md index 6d0a8bb2615..99059d79016 100644 --- a/docs/en/query_language/functions/ext_dict_functions.md +++ b/docs/en/query_language/functions/ext_dict_functions.md @@ -16,7 +16,7 @@ For information on connecting and configuring external dictionaries, see "[Exter `dictGetT('dict_name', 'attr_name', id)` -- Get the value of the attr_name attribute from the dict_name dictionary using the 'id' key.`dict_name` and `attr_name` are constant strings.`id`must be UInt64. +- Get the value of the attr_name attribute from the dict_name dictionary using the 'id' key.`dict_name` and `attr_name` are constant strings.`id`must be UInt64. If there is no `id` key in the dictionary, it returns the default value specified in the dictionary description. ## dictGetTOrDefault diff --git a/docs/en/query_language/functions/functions_for_nulls.md b/docs/en/query_language/functions/functions_for_nulls.md index dcbc745066a..dd0594b8185 100644 --- a/docs/en/query_language/functions/functions_for_nulls.md +++ b/docs/en/query_language/functions/functions_for_nulls.md @@ -102,7 +102,7 @@ coalesce(x,...) **Returned values** -- The first non-`NULL` argument. +- The first non-`NULL` argument. - `NULL`, if all arguments are `NULL`. **Example** @@ -207,7 +207,7 @@ SELECT nullIf(1, 2) ## assumeNotNull -Results in a value of type [Nullable](../../data_types/nullable.md)i for a non- `Nullable`, if the value is not `NULL`. +Results in a value of type [Nullable](../../data_types/nullable.md) for a non- `Nullable`, if the value is not `NULL`. ``` assumeNotNull(x) diff --git a/docs/en/query_language/functions/geo.md b/docs/en/query_language/functions/geo.md index 13fc8673746..2c9a3aac38f 100644 --- a/docs/en/query_language/functions/geo.md +++ b/docs/en/query_language/functions/geo.md @@ -77,7 +77,7 @@ pointInPolygon((x, y), [(a, b), (c, d) ...], ...) **Input values** -- `(x, y)` — Coordinates of a point on the plane. Data type — [Tuple](../../data_types/tuple.md) — A tuple of two numbers. +- `(x, y)` — Coordinates of a point on the plane. Data type — [Tuple](../../data_types/tuple.md) — A tuple of two numbers. - `[(a, b), (c, d) ...]` — Polygon vertices. Data type — [Array](../../data_types/array.md). Each vertex is represented by a pair of coordinates `(a, b)`. Vertices should be specified in a clockwise or counterclockwise order. The minimum number of vertices is 3. The polygon must be constant. - The function also supports polygons with holes (cut out sections). In this case, add polygons that define the cut out sections using additional arguments of the function. The function does not support non-simply-connected polygons. diff --git a/docs/en/query_language/functions/other_functions.md b/docs/en/query_language/functions/other_functions.md index 85cd1dfe5c5..e49bedd8199 100644 --- a/docs/en/query_language/functions/other_functions.md +++ b/docs/en/query_language/functions/other_functions.md @@ -336,7 +336,7 @@ toColumnTypeName(value) **Returned values** -- A string with the name of the class that is used for representing the `value` data type in RAM. +- A string with the name of the class that is used for representing the `value` data type in RAM. **Example of the difference between` toTypeName ' and ' toColumnTypeName`** @@ -376,7 +376,7 @@ dumpColumnStructure(value) **Returned values** -- A string describing the structure that is used for representing the `value` data type in RAM. +- A string describing the structure that is used for representing the `value` data type in RAM. **Example** diff --git a/docs/en/query_language/functions/string_search_functions.md b/docs/en/query_language/functions/string_search_functions.md index 22af3c8550d..ea5b1ecf8ff 100644 --- a/docs/en/query_language/functions/string_search_functions.md +++ b/docs/en/query_language/functions/string_search_functions.md @@ -18,7 +18,7 @@ For a case-insensitive search, use the function `positionCaseInsensitiveUTF8`. ## match(haystack, pattern) -Checks whether the string matches the `pattern` regular expression. A `re2` regular expression. The [syntax](https://github.com/google/re2/wiki/Syntax) of the `re2` regular expressions is more limited than the syntax of the Perl regular expressions. +Checks whether the string matches the `pattern` regular expression. A `re2` regular expression. The [syntax](https://github.com/google/re2/wiki/Syntax) of the `re2` regular expressions is more limited than the syntax of the Perl regular expressions. Returns 0 if it doesn't match, or 1 if it matches. diff --git a/docs/en/query_language/functions/url_functions.md b/docs/en/query_language/functions/url_functions.md index efe48970338..d32c05aff57 100644 --- a/docs/en/query_language/functions/url_functions.md +++ b/docs/en/query_language/functions/url_functions.md @@ -34,7 +34,7 @@ For example, `cutToFirstSignificantSubdomain('https://news.yandex.com.tr/') = 'y ### path -Returns the path. Example: `/top/news.html` The path does not include the query string. +Returns the path. Example: `/top/news.html` The path does not include the query string. ### pathFull @@ -42,7 +42,7 @@ The same as above, but including query string and fragment. Example: /top/news.h ### queryString -Returns the query string. Example: page=1&lr=213. query-string does not include the initial question mark, as well as \# and everything after \#. +Returns the query string. Example: page=1&lr=213. query-string does not include the initial question mark, as well as \# and everything after \#. ### fragment diff --git a/docs/en/query_language/insert_into.md b/docs/en/query_language/insert_into.md index fc9421d3479..1d6d268e631 100644 --- a/docs/en/query_language/insert_into.md +++ b/docs/en/query_language/insert_into.md @@ -49,7 +49,7 @@ INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... Columns are mapped according to their position in the SELECT clause. However, their names in the SELECT expression and the table for INSERT may differ. If necessary, type casting is performed. -None of the data formats except Values allow setting values to expressions such as `now()`, `1 + 2`, and so on. The Values format allows limited use of expressions, but this is not recommended, because in this case inefficient code is used for their execution. +None of the data formats except Values allow setting values to expressions such as `now()`, `1 + 2`, and so on. The Values format allows limited use of expressions, but this is not recommended, because in this case inefficient code is used for their execution. Other queries for modifying data parts are not supported: `UPDATE`, `DELETE`, `REPLACE`, `MERGE`, `UPSERT`, `INSERT UPDATE`. However, you can delete old data using `ALTER TABLE ... DROP PARTITION`. diff --git a/docs/en/query_language/misc.md b/docs/en/query_language/misc.md index 01cf2c934c2..89ad8f3bca8 100644 --- a/docs/en/query_language/misc.md +++ b/docs/en/query_language/misc.md @@ -76,7 +76,7 @@ There is no `DETACH DATABASE` query. ## DROP -This query has two types: `DROP DATABASE` and `DROP TABLE`. +This query has two types: `DROP DATABASE` and `DROP TABLE`. ``` sql DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster] diff --git a/docs/en/query_language/operators.md b/docs/en/query_language/operators.md index fc4d152bf40..61e008c2ba8 100644 --- a/docs/en/query_language/operators.md +++ b/docs/en/query_language/operators.md @@ -5,7 +5,7 @@ Groups of operators are listed in order of priority (the higher it is in the lis ## Access Operators -`a[N]` Access to an element of an array; ` arrayElement(a, N) function`. +`a[N]` Access to an element of an array; ` arrayElement(a, N) function`. `a.N` – Access to a tuble element; `tupleElement(a, N)` function. @@ -120,7 +120,7 @@ The following operators do not have a priority, since they are brackets: ## Associativity All binary operators have left associativity. For example, `1 + 2 + 3` is transformed to `plus(plus(1, 2), 3)`. -Sometimes this doesn't work the way you expect. For example, ` SELECT 4 > 2 > 3` will result in 0. +Sometimes this doesn't work the way you expect. For example, ` SELECT 4 > 2 > 3` will result in 0. For efficiency, the `and` and `or` functions accept any number of arguments. The corresponding chains of `AND` and `OR` operators are transformed to a single call of these functions. diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index 02a5cc3578d..a4aeec35ec9 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -575,11 +575,11 @@ When using `max_bytes_before_external_group_by`, we recommend that you set max_m For example, if `max_memory_usage` was set to 10000000000 and you want to use external aggregation, it makes sense to set `max_bytes_before_external_group_by` to 10000000000, and max_memory_usage to 20000000000. When external aggregation is triggered (if there was at least one dump of temporary data), maximum consumption of RAM is only slightly more than ` max_bytes_before_external_group_by`. -With distributed query processing, external aggregation is performed on remote servers. In order for the requestor server to use only a small amount of RAM, set ` distributed_aggregation_memory_efficient` to 1. +With distributed query processing, external aggregation is performed on remote servers. In order for the requestor server to use only a small amount of RAM, set ` distributed_aggregation_memory_efficient` to 1. When merging data flushed to the disk, as well as when merging results from remote servers when the ` distributed_aggregation_memory_efficient` setting is enabled, consumes up to 1/256 \* the number of threads from the total amount of RAM. -When external aggregation is enabled, if there was less than ` max_bytes_before_external_group_by` of data (i.e. data was not flushed), the query runs just as fast as without external aggregation. If any temporary data was flushed, the run time will be several times longer (approximately three times). +When external aggregation is enabled, if there was less than ` max_bytes_before_external_group_by` of data (i.e. data was not flushed), the query runs just as fast as without external aggregation. If any temporary data was flushed, the run time will be several times longer (approximately three times). If you have an ORDER BY with a small LIMIT after GROUP BY, then the ORDER BY CLAUSE will not use significant amounts of RAM. But if the ORDER BY doesn't have LIMIT, don't forget to enable external sorting (`max_bytes_before_external_sort`). @@ -693,7 +693,7 @@ The result will be the same as if GROUP BY were specified across all the fields DISTINCT is not supported if SELECT has at least one array column. -`DISTINCT` works with [NULL](syntax.md) as if `NULL` were a specific value, and `NULL=NULL`. In other words, in the `DISTINCT` results, different combinations with `NULL` only occur once. +`DISTINCT` works with [NULL](syntax.md) as if `NULL` were a specific value, and `NULL=NULL`. In other words, in the `DISTINCT` results, different combinations with `NULL` only occur once. ### LIMIT Clause @@ -813,7 +813,7 @@ A subquery in the IN clause is always run just one time on a single server. Ther #### NULL processing -During request processing, the IN operator assumes that the result of an operation with [NULL](syntax.md) is always equal to `0`, regardless of whether `NULL` is on the right or left side of the operator. `NULL` values are not included in any dataset, do not correspond to each other and cannot be compared. +During request processing, the IN operator assumes that the result of an operation with [NULL](syntax.md) is always equal to `0`, regardless of whether `NULL` is on the right or left side of the operator. `NULL` values are not included in any dataset, do not correspond to each other and cannot be compared. Here is an example with the `t_null` table: @@ -847,18 +847,18 @@ FROM t_null #### Distributed Subqueries {#select-distributed-subqueries} -There are two options for IN-s with subqueries (similar to JOINs): normal `IN` / `JOIN` and `GLOBAL IN` / `GLOBAL JOIN`. They differ in how they are run for distributed query processing. +There are two options for IN-s with subqueries (similar to JOINs): normal `IN` / `JOIN` and `GLOBAL IN` / `GLOBAL JOIN`. They differ in how they are run for distributed query processing. !!! attention Remember that the algorithms described below may work differently depending on the [settings](../operations/settings/settings.md) `distributed_product_mode` setting. When using the regular IN, the query is sent to remote servers, and each of them runs the subqueries in the `IN` or `JOIN` clause. -When using `GLOBAL IN` / `GLOBAL JOINs`, first all the subqueries are run for `GLOBAL IN` / `GLOBAL JOINs`, and the results are collected in temporary tables. Then the temporary tables are sent to each remote server, where the queries are run using this temporary data. +When using `GLOBAL IN` / `GLOBAL JOINs`, first all the subqueries are run for `GLOBAL IN` / `GLOBAL JOINs`, and the results are collected in temporary tables. Then the temporary tables are sent to each remote server, where the queries are run using this temporary data. For a non-distributed query, use the regular `IN` / `JOIN`. -Be careful when using subqueries in the `IN` / `JOIN` clauses for distributed query processing. +Be careful when using subqueries in the `IN` / `JOIN` clauses for distributed query processing. Let's look at some examples. Assume that each server in the cluster has a normal **local_table**. Each server also has a **distributed_table** table with the **Distributed** type, which looks at all the servers in the cluster. diff --git a/docs/en/query_language/syntax.md b/docs/en/query_language/syntax.md index d7307a64376..606df81a087 100644 --- a/docs/en/query_language/syntax.md +++ b/docs/en/query_language/syntax.md @@ -24,7 +24,7 @@ There may be any number of space symbols between syntactical constructions (incl SQL-style and C-style comments are supported. SQL-style comments: from `--` to the end of the line. The space after `--` can be omitted. -Comments in C-style: from `/*` to `*/`. These comments can be multiline. Spaces are not required here, either. +Comments in C-style: from `/*` to `*/`. These comments can be multiline. Spaces are not required here, either. ## Keywords diff --git a/docs/en/query_language/table_functions/url.md b/docs/en/query_language/table_functions/url.md index 662049aa0f9..edeabdc6902 100644 --- a/docs/en/query_language/table_functions/url.md +++ b/docs/en/query_language/table_functions/url.md @@ -4,7 +4,7 @@ `url(URL, format, structure)` - returns a table created from the `URL` with given `format` and `structure`. -URL - HTTP or HTTPS server address, which can accept `GET` and/or `POST` requests. +URL - HTTP or HTTPS server address, which can accept `GET` and/or `POST` requests. format - [format](../../interfaces/formats.md#formats) of the data. diff --git a/docs/fa/data_types/float.md b/docs/fa/data_types/float.md index 6c48b71d9a4..c83a4831ff3 100644 --- a/docs/fa/data_types/float.md +++ b/docs/fa/data_types/float.md @@ -7,7 +7,7 @@ Type های float در ClickHouse مشابه C می باشد: - `Float32` - `float` -- `Float64` - `double` +- `Float64` - `double` توصیه می کنیم که داده ها را هرزمان که امکان پذیره است به جای float به صورت int ذخیره کنید. برای مثال: تبدیل دقت اعداد به یک مقدار int، مثل سرعت page load در قالب میلی ثانیه. diff --git a/docs/fa/data_types/nested_data_structures/aggregatefunction.md b/docs/fa/data_types/nested_data_structures/aggregatefunction.md index d57a3eddba0..3ebcde1b6ae 100644 --- a/docs/fa/data_types/nested_data_structures/aggregatefunction.md +++ b/docs/fa/data_types/nested_data_structures/aggregatefunction.md @@ -2,7 +2,7 @@ # AggregateFunction(name, types_of_arguments...) -حالت متوسط از توابع aggregate. برای دریافت آن، از توابع aggregate به همراه پسوند '-State' استفاده کنید. برای اطلاعات بیشتر قسمت "AggregatingMergeTree" را ببینید. +حالت متوسط از توابع aggregate. برای دریافت آن، از توابع aggregate به همراه پسوند '-State' استفاده کنید. برای اطلاعات بیشتر قسمت "AggregatingMergeTree" را ببینید. [مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/nested_data_structures/aggregatefunction/) diff --git a/docs/fa/data_types/nested_data_structures/nested.md b/docs/fa/data_types/nested_data_structures/nested.md index 1f3b675eef1..a7a4be5b1ea 100644 --- a/docs/fa/data_types/nested_data_structures/nested.md +++ b/docs/fa/data_types/nested_data_structures/nested.md @@ -69,7 +69,7 @@ LIMIT 10
-ساده ترین راه برای فکر کردن به یک ساختار داده nestet این است که، یک nestet مجموعه ای از آرایه های چند ستونی با طول ثابت است. +ساده ترین راه برای فکر کردن به یک ساختار داده nestet این است که، یک nestet مجموعه ای از آرایه های چند ستونی با طول ثابت است. تنها جایی که یک دستور SELECT می تواند کل ساختار داده ی nested را به جای مشخص کردن ستون های آن قرار دهد، عبارت ARRAY JOIN است. برای اطلاعات بیشتر "ARRAY JOIN clouse" را ببینید. مثال: diff --git a/docs/fa/getting_started/example_datasets/amplab_benchmark.md b/docs/fa/getting_started/example_datasets/amplab_benchmark.md index 6ed6f97a344..6718b358f49 100644 --- a/docs/fa/getting_started/example_datasets/amplab_benchmark.md +++ b/docs/fa/getting_started/example_datasets/amplab_benchmark.md @@ -1,6 +1,6 @@
-# بنچمارک AMPLab Big Data +# بنچمارک AMPLab Big Data ببینید diff --git a/docs/fa/getting_started/example_datasets/nyc_taxi.md b/docs/fa/getting_started/example_datasets/nyc_taxi.md index 892b706e616..1f7c5c1a573 100644 --- a/docs/fa/getting_started/example_datasets/nyc_taxi.md +++ b/docs/fa/getting_started/example_datasets/nyc_taxi.md @@ -8,7 +8,7 @@ برای توضیحات بیشتر در ارتباط با دیتاست و موارد مربوط به دانلود به دو لینک و مراجعه کنید. -دانلود فایل ها حدود 277 گیگابایت داده ی غیرفشرده در قالب فایل های CSV می باشد. دانلود با استفاده ازبیش از یک کانکشن 1 Gbit نزدیک 1 ساعت طول می کشد (دانلود موازی از s3.amazonaws.com حداقل نصف کانال 1 Gbit رو جبران می کند). بعضی از فایل ها ممکن است به طول کامل دانلود نشوند. اندازه فایل ها را بررسی کنید و اگر فایلی مشکوک بود، مجددا دانلود کنید. +دانلود فایل ها حدود 277 گیگابایت داده ی غیرفشرده در قالب فایل های CSV می باشد. دانلود با استفاده ازبیش از یک کانکشن 1 Gbit نزدیک 1 ساعت طول می کشد (دانلود موازی از s3.amazonaws.com حداقل نصف کانال 1 Gbit رو جبران می کند). بعضی از فایل ها ممکن است به طول کامل دانلود نشوند. اندازه فایل ها را بررسی کنید و اگر فایلی مشکوک بود، مجددا دانلود کنید. بعضی از فایل ها ممکن است دارای سطرهای نامعتبر باشه. با اجرای دستورات زیر این موارد برطرف می شود: @@ -23,7 +23,7 @@ mv data/yellow_tripdata_2010-03.csv_ data/yellow_tripdata_2010-03.csv
-سپس داده ها باید در PostgreSQL پیش پردازش شوند. این کار نقاط انتخابی چند ضلعی را ایجاد می کند (برای مطابقت با نقاط بر روی نقشه با مناطق شهر نیویورک) و تمام داده ها را با استفاده از JOIN در یک جدول flat و denormal ترکیب می کند. برای این کار شما نیاز به نصب PostgreSQL با پشتیبانی از PostGIS دارید. +سپس داده ها باید در PostgreSQL پیش پردازش شوند. این کار نقاط انتخابی چند ضلعی را ایجاد می کند (برای مطابقت با نقاط بر روی نقشه با مناطق شهر نیویورک) و تمام داده ها را با استفاده از JOIN در یک جدول flat و denormal ترکیب می کند. برای این کار شما نیاز به نصب PostgreSQL با پشتیبانی از PostGIS دارید. در هنگام اجرای `initialize_database.sh` مراقب باشید و به صورت دستی مجددا تمام جداول را چک کنید. @@ -277,7 +277,7 @@ FROM trips
-این کار با سرعت 428 هزار رکورد در ثانیه و 3030 ثانیه طول خواهد کشید. برای load سریعتر، شما می توانید یک جدول با موتور `Log` به جای `MergeTree` بسازید. در این مورد، دانلود سریعتر از 200 ثانیه کار می کند. +این کار با سرعت 428 هزار رکورد در ثانیه و 3030 ثانیه طول خواهد کشید. برای load سریعتر، شما می توانید یک جدول با موتور `Log` به جای `MergeTree` بسازید. در این مورد، دانلود سریعتر از 200 ثانیه کار می کند. این جدول 126 گیابایت فضا بر روی دیسک اشغال می کند. @@ -355,7 +355,7 @@ Two Intel(R) Xeon(R) CPU E5-2650 v2 @ 2.60GHz, 16 physical kernels total,
``` -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) +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) ```
diff --git a/docs/fa/getting_started/example_datasets/ontime.md b/docs/fa/getting_started/example_datasets/ontime.md index 71c09531a8d..12f844f1181 100644 --- a/docs/fa/getting_started/example_datasets/ontime.md +++ b/docs/fa/getting_started/example_datasets/ontime.md @@ -197,7 +197,7 @@ Q4. تعداد تاخیرها براساس carrier در سال 78
``` sql -SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC +SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC ```
diff --git a/docs/fa/getting_started/example_datasets/star_schema.md b/docs/fa/getting_started/example_datasets/star_schema.md index 90bf5246e5c..fe7f861dd15 100644 --- a/docs/fa/getting_started/example_datasets/star_schema.md +++ b/docs/fa/getting_started/example_datasets/star_schema.md @@ -14,7 +14,7 @@ make
-در هنگام پردازش چند warnings نمایش داده می شود که مشکلی نیست و طبیعی است. +در هنگام پردازش چند warnings نمایش داده می شود که مشکلی نیست و طبیعی است. `dbgen` و ` dists.dss` را در یک جا با 800 گیگابایت فضای حالی دیسک قرار دهید. diff --git a/docs/fa/getting_started/index.md b/docs/fa/getting_started/index.md index 3fbe345b888..9189e0cabae 100644 --- a/docs/fa/getting_started/index.md +++ b/docs/fa/getting_started/index.md @@ -46,7 +46,7 @@ sudo apt-get install clickhouse-client clickhouse-server شما همچنین می توانید از طریق لینک زیر پکیج ClickHouse را به صورت دستی دانلود و نصب کنید: . -ClickHouse دارای تنظیمات محدودیت دسترسی می باشد. این تنظیمات در فایل 'users.xml' (کنار 'config.xml') می باشد. به صورت پیش فرض دسترسی برای کاربر 'default' از همه جا بدون نیاز به پسورد وجود دارد. 'user/default/networks' را مشاهده کنید. برای اطلاعات بیشتر قسمت "تنظیمات فایل ها" را مشاهده کنید. +ClickHouse دارای تنظیمات محدودیت دسترسی می باشد. این تنظیمات در فایل 'users.xml' (کنار 'config.xml') می باشد. به صورت پیش فرض دسترسی برای کاربر 'default' از همه جا بدون نیاز به پسورد وجود دارد. 'user/default/networks' را مشاهده کنید. برای اطلاعات بیشتر قسمت "تنظیمات فایل ها" را مشاهده کنید. ### نصب از طریق Source diff --git a/docs/fa/interfaces/cli.md b/docs/fa/interfaces/cli.md index 4b30ecbfb1b..8501f46ecd7 100644 --- a/docs/fa/interfaces/cli.md +++ b/docs/fa/interfaces/cli.md @@ -45,7 +45,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA در حالت intercative، شما یک command line برای درج query های خود دریافت می کنید. -اگر 'multiline' مشخص نشده باشد (به صورت پیش فرض): برای اجرای یک query، دکمه Enter را بزنید. سیمی کالن در انتهای query اجباری نیست. برای درج یک query چند خطی (multiline)، دکمه ی بک اسلش `\` را قبل از line feed فشار دهید. بعد از فشردن Enter، از شما برای درج خط بعدی query درخواست خواهد شد. +اگر 'multiline' مشخص نشده باشد (به صورت پیش فرض): برای اجرای یک query، دکمه Enter را بزنید. سیمی کالن در انتهای query اجباری نیست. برای درج یک query چند خطی (multiline)، دکمه ی بک اسلش `\` را قبل از line feed فشار دهید. بعد از فشردن Enter، از شما برای درج خط بعدی query درخواست خواهد شد. اگر چند خطی (multiline) مشخص شده باشد: برای اجرای query، در انتها سیمی کالن را وارد کنید و سپس Enter بزنید. اگر سیمی کالن از انتهای خط حذف می شد، از شما برای درج خط جدید query درخواست می شد. @@ -55,7 +55,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA command line برا پایه 'readline' (و 'history' یا 'libedit'، یه بدون کتابخانه بسته به build) می باشد. به عبارت دیگر، این محیط از shortcut های آشنا استفاده می کند و history دستورات را نگه می دار. history ها در فایل ~/.clickhouse-client-history نوشته می شوند. -به صورت پیش فرض فرمت خروجی PrettyCompact می باشد. شما میتوانید از طریق دستور FORMAT در یک query، یا با مشخص کردن `\G` در انتهای query، استفاده از آرگومان های `--format` یا `--vertical` یا از کانفیگ فایل کلاینت، فرمت خروجی را مشخص کنید. +به صورت پیش فرض فرمت خروجی PrettyCompact می باشد. شما میتوانید از طریق دستور FORMAT در یک query، یا با مشخص کردن `\G` در انتهای query، استفاده از آرگومان های `--format` یا `--vertical` یا از کانفیگ فایل کلاینت، فرمت خروجی را مشخص کنید. برای خروج از کلاینت، Ctrl-D (یا Ctrl+C) را فشار دهید؛ و یا یکی از دستورات زیر را به جای اجرای query اجرا کنید: "exit", "quit", "logout", "exit;", "quit;", "logout;", "q", "Q", ":q" @@ -68,7 +68,7 @@ command line برا پایه 'readline' (و 'history' یا 'libedit'، یه بد شما میتوانید query های طولانی را با فشردن Ctrl-C کنسل کنید. هر چند، بعد از این کار همچنان نیاز به انتظار چند ثانیه ای برای قطع کردن درخواست توسط سرور می باشید. امکان کنسل کردن یک query در مراحل خاص وجود ندارد. اگر شما صبر نکنید و برای بار دوم Ctrl+C را وارد کنید از client خارج می شوید. -کلاینت commant-line اجازه ی پاس دادن داده های external (جداول موقت external) را برای query ها می دهد. برای اطلاعات بیشتر به بخش "داده های External برای پردازش query" مراجعه کنید. +کلاینت commant-line اجازه ی پاس دادن داده های external (جداول موقت external) را برای query ها می دهد. برای اطلاعات بیشتر به بخش "داده های External برای پردازش query" مراجعه کنید. ## پیکربندی {#interfaces_cli_configuration} diff --git a/docs/fa/interfaces/formats.md b/docs/fa/interfaces/formats.md index 04e4a4ea3bf..8391352ea93 100644 --- a/docs/fa/interfaces/formats.md +++ b/docs/fa/interfaces/formats.md @@ -43,9 +43,9 @@ Format | INSERT | SELECT 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 به صورت خودکار تمایز می یابند. +به عنوان یک استثنا، پارس کردن تاریخ به همراه ساعت، اگر مقدار دقیقا شامل 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 می تواند به هر یک از حالات زیر پارس شود:: +رشته های دارای کاراکتر های ویژه backslash-escaped چاپ می شوند. escape های در ادامه برای خروجی استفاده می شوند: `\b`، `\f`، `\r`، `\n`، `\t`، `\0`, `\'`، `\\`. پارسر همچنین از `\a`، `\v`، و `\xHH` (hex escape) و هر `\c` پشتیبانی می کند. بدین ترتیب خواندن داده ها از فرمت line feed که می تواند به صورت `\n` یا `\` نوشته شود پشتیبانی می کند. برای مثال، رشته ی `Hello world` به همراه line feed بین کلمات به جای space می تواند به هر یک از حالات زیر پارس شود::
@@ -146,7 +146,7 @@ SearchPhrase=baku count()=1000 Comma Separated Values format ([RFC](https://tools.ietf.org/html/rfc4180)). -زمانی که از این روش برای فرمت استفاده می شود، سطر ها با دابل کوتیشن enclosed می شوند. دابل کوتیشن داخل یک رشته خروجی آن به صورت دو دابل کوتیشن در یک سطر است. قانون دیگری برای escape کردن کاراکترها وجود ندارد. تاریخ و تاریخ-ساعت در دابل کوتیشن ها enclosed می شوند. اعداد بدون دابل کوتیشن در خروجی می آیند. مقادیر با جدا کننده * مشخص می شوند. سطر ها با استفاده از line feed (LF) جدا می شوند. آرایه ها در csv به این صورت serialize می شوند: ابتدا آرایه به یک رشته با فرمت TabSeparate سریالایز می شوند، و سپس رشته ی حاصل در دابل کوتیشن برای csv ارسال می شود. Tuple ها در فرمت CSV در ستون های جدا سریالایز می شوند (به این ترتیب، nest ها در tuble از دست میروند) +زمانی که از این روش برای فرمت استفاده می شود، سطر ها با دابل کوتیشن enclosed می شوند. دابل کوتیشن داخل یک رشته خروجی آن به صورت دو دابل کوتیشن در یک سطر است. قانون دیگری برای escape کردن کاراکترها وجود ندارد. تاریخ و تاریخ-ساعت در دابل کوتیشن ها enclosed می شوند. اعداد بدون دابل کوتیشن در خروجی می آیند. مقادیر با جدا کننده * مشخص می شوند. سطر ها با استفاده از line feed (LF) جدا می شوند. آرایه ها در csv به این صورت serialize می شوند: ابتدا آرایه به یک رشته با فرمت TabSeparate سریالایز می شوند، و سپس رشته ی حاصل در دابل کوتیشن برای csv ارسال می شود. Tuple ها در فرمت CSV در ستون های جدا سریالایز می شوند (به این ترتیب، nest ها در tuble از دست میروند)
@@ -340,7 +340,7 @@ JSON با جاوااسکریپت سازگار است. برای اطمینان ا ## Pretty -خروجی داده ها به صورت جداول Unicode-art، همچنین استفاده از ANSI-escape برای تنظیم رنگ های ترمینال. یک جدول کامل کشیده می شود، و هر سطر دو خط از ترمینال را اشغال می کند. هر بلاکِ نتیجه، به عنوان یک جدول جدا چاپ می شود.پس بلاک ها می توانند بدون بافر کردن نتایج چاپ شوند (بافرینگ برای pre-calculate تمام مقادیر قابل مشاهده ضروری است). برای جلوگیری از دامپ زیاد داده ها در ترمینال، 10 هزار سطر اول چاپ می شوند. اگر تعداد سطر های بزرگتر مساوی 10 هزار باشد، پیغام " 10 هزار اول نمایش داده شد" چاپ می شود. این فرمت فقط مناسب خروجی نتایج query ها می باشد، نه برای پارس کردن (دریافت داده ها و درج آن در جدول). +خروجی داده ها به صورت جداول Unicode-art، همچنین استفاده از ANSI-escape برای تنظیم رنگ های ترمینال. یک جدول کامل کشیده می شود، و هر سطر دو خط از ترمینال را اشغال می کند. هر بلاکِ نتیجه، به عنوان یک جدول جدا چاپ می شود.پس بلاک ها می توانند بدون بافر کردن نتایج چاپ شوند (بافرینگ برای pre-calculate تمام مقادیر قابل مشاهده ضروری است). برای جلوگیری از دامپ زیاد داده ها در ترمینال، 10 هزار سطر اول چاپ می شوند. اگر تعداد سطر های بزرگتر مساوی 10 هزار باشد، پیغام " 10 هزار اول نمایش داده شد" چاپ می شود. این فرمت فقط مناسب خروجی نتایج query ها می باشد، نه برای پارس کردن (دریافت داده ها و درج آن در جدول). فرمت Pretty از total values (هنگام استفاده از WITH TOTALS) و extreme (هنگام که 'extremes' برابر با 1 است) برای خروجی پشتیبانی می کند. در این موارد، total values و extreme values بعد از نمایش داده های اصلی در جداول جدا، چاپ می شوند. مثال (برای فرمت PrettyCompact نمایش داده شده است): diff --git a/docs/fa/interfaces/http.md b/docs/fa/interfaces/http.md index 418a92f4daf..6462c68b224 100644 --- a/docs/fa/interfaces/http.md +++ b/docs/fa/interfaces/http.md @@ -87,7 +87,7 @@ $ echo 'SELECT 1 FORMAT Pretty' | curl 'http://localhost:8123/?' --data-binary @
-برای query های INSERT متد POST ضروری است. در این مورد، شما می توانید ابتدای query خود را در URL parameter بنویسید، و از POST برای پاس داده داده ها برای درج استفاده کنید. داده ی برای درج می تواند، برای مثال یک دامپ tab-separated شده از MySQL باشد. به این ترتیب، query INSERT جایگزین LOAD DATA LOCAL INFILE از MySQL می شود. +برای query های INSERT متد POST ضروری است. در این مورد، شما می توانید ابتدای query خود را در URL parameter بنویسید، و از POST برای پاس داده داده ها برای درج استفاده کنید. داده ی برای درج می تواند، برای مثال یک دامپ tab-separated شده از MySQL باشد. به این ترتیب، query INSERT جایگزین LOAD DATA LOCAL INFILE از MySQL می شود. مثال: ساخت جدول @@ -175,7 +175,7 @@ echo 'DROP TABLE t' | curl 'http://localhost:8123/' --data-binary @- شما می توانید از فرمت فشرده سازی داخلی ClickHouse در هنگان انتقال داده ها استفاده کنید. این فشرده سازی داده، یک فرمت غیراستاندارد است، و شما باید از برنامه مخصوص فشرده سازی ClickHouse برای استفاده از آن استفاده کنید. (این برنامه در هنگام نصب پکیج clickhouse-client نصب شده است) -اگر شما در URL پارامتر 'compress=1' را قرار دهید، سرور داده های ارسالی به شما را فشرده سازی می کند. اگر شما پارامتر 'decompress=1' را در URL ست کنید، سرور داده های ارسالی توسط متد POST را decompress می کند. +اگر شما در URL پارامتر 'compress=1' را قرار دهید، سرور داده های ارسالی به شما را فشرده سازی می کند. اگر شما پارامتر 'decompress=1' را در URL ست کنید، سرور داده های ارسالی توسط متد POST را decompress می کند. همچنین استفاده از فشرده سازی استاندارد gzip در HTTP ممکن است. برای ارسال درخواست POST و فشرده سازی آن به صورت gzip، هدر `Content-Encoding: gzip` را به request خود اضافه کنید. برای اینکه ClickHouse، response فشرده شده به صورت gzip برای شما ارسال کند، ابتدا باید `enable_http_compression` را در تنظیمات ClickHouse فعال کنید و در ادامه هدر `Accept-Encoding: gzip` را به درخواست خود اضافه کنید. diff --git a/docs/fa/introduction/distinctive_features.md b/docs/fa/introduction/distinctive_features.md index 1eafdbd6863..0dbe60db7f3 100644 --- a/docs/fa/introduction/distinctive_features.md +++ b/docs/fa/introduction/distinctive_features.md @@ -4,7 +4,7 @@ ## مدیریت دیتابیس ستون گرای واقعی -در یک مدیریت دیتابیس ستون گرای واقعی، هیچ مقداری فضای اضافی برای ذخیره سازی ندارد. برای مثال، این به این معنیست که برای مقادیر، constant-length باید پشتیبانی شوند تا از ذخیره سازی طول مقدار به عنوان یه عدد integer کنار مقدار جلوگیری شود. در این مورد، یک میلیارد مقدار Uint8 باید در واقع در حالت غیرفشرده 1 گیگابایت فضا اشغال کند، در غیراین صورت به شدت بر عملکرد CPU تاثیر میگذارد. این خیلی مهم هست که داده ها به صورت compact ذخیره سازی شوند حتی زمانی که uncompressed هستند، از آنجا که سرعت سرعت decompress (CPU Usage) عمدتا به حجم داده های uncompress بستگی دارد. +در یک مدیریت دیتابیس ستون گرای واقعی، هیچ مقداری فضای اضافی برای ذخیره سازی ندارد. برای مثال، این به این معنیست که برای مقادیر، constant-length باید پشتیبانی شوند تا از ذخیره سازی طول مقدار به عنوان یه عدد integer کنار مقدار جلوگیری شود. در این مورد، یک میلیارد مقدار Uint8 باید در واقع در حالت غیرفشرده 1 گیگابایت فضا اشغال کند، در غیراین صورت به شدت بر عملکرد CPU تاثیر میگذارد. این خیلی مهم هست که داده ها به صورت compact ذخیره سازی شوند حتی زمانی که uncompressed هستند، از آنجا که سرعت سرعت decompress (CPU Usage) عمدتا به حجم داده های uncompress بستگی دارد. این بسیار قابل توجه است چون سیستم هایی وجود دارند که توانایی ذخیره سازی مقادیر ستون ها را به صورت جداگانه دارند، اما به دلیل بهینه سازی آنها برای دیگر سناریو ها، نمیتوانند به طور موثر پردازش های تحیلی انجام دهند. برای مثال HBase، BigTable، Cassandra و HyperTable. در این سیستم ها، شما توان عملیاتی حدود صدها هزار سطر در ثانیه را دارید، اما نه صدها میلیون سطر در ثانیه. diff --git a/docs/ru/data_types/tuple.md b/docs/ru/data_types/tuple.md index 1763967c052..56a195d6398 100644 --- a/docs/ru/data_types/tuple.md +++ b/docs/ru/data_types/tuple.md @@ -33,7 +33,7 @@ SELECT ## Особенности работы с типами данных -При создании кортежа "на лету" ClickHouse автоматически определяет тип каждого аргументов как минимальный из типов, который может сохранить значение аргумента. Если аргумент — [NULL](../query_language/syntax.md#null-literal), то тип элемента кортежа — [Nullable](nullable.md). +При создании кортежа "на лету" ClickHouse автоматически определяет тип каждого аргументов как минимальный из типов, который может сохранить значение аргумента. Если аргумент — [NULL](../query_language/syntax.md#null-literal), то тип элемента кортежа — [Nullable](nullable.md). Пример автоматического определения типа данных: diff --git a/docs/ru/development/style.md b/docs/ru/development/style.md index 4bcff77ca3a..047291d3341 100644 --- a/docs/ru/development/style.md +++ b/docs/ru/development/style.md @@ -44,13 +44,13 @@ void reinsert(const Value & x) memcpy(&buf[place_value], &x, sizeof(x)); ``` -**6.** В выражениях `if`, `for`, `while` и т.д. перед открывающей скобкой ставится пробел (в отличие от вызовов функций). +**6.** В выражениях `if`, `for`, `while` и т.д. перед открывающей скобкой ставится пробел (в отличие от вызовов функций). ```cpp for (size_t i = 0; i < rows; i += storage.index_granularity) ``` -**7.** Вокруг бинарных операторов (`+`, `-`, `*`, `/`, `%`, ...), а также тернарного оператора `?:` ставятся пробелы. +**7.** Вокруг бинарных операторов (`+`, `-`, `*`, `/`, `%`, ...), а также тернарного оператора `?:` ставятся пробелы. ```cpp UInt16 year = (s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0'); @@ -58,7 +58,7 @@ UInt8 month = (s[5] - '0') * 10 + (s[6] - '0'); UInt8 day = (s[8] - '0') * 10 + (s[9] - '0'); ``` -**8.** Если ставится перенос строки, то оператор пишется на новой строке, и перед ним увеличивается отступ. +**8.** Если ставится перенос строки, то оператор пишется на новой строке, и перед ним увеличивается отступ. ```cpp if (elapsed_ns) @@ -67,7 +67,7 @@ if (elapsed_ns) << bytes_read_on_server * 1000.0 / elapsed_ns << " MB/s.) "; ``` -**9.** Внутри строки можно, выполнять выравнивание с помощью пробелов. +**9.** Внутри строки можно, выполнять выравнивание с помощью пробелов. ```cpp dst.ClickLogID = click.LogID; @@ -85,7 +85,7 @@ dst.ClickGoodEvent = click.GoodEvent; **13.** Оператор `[]` не отделяется пробелами. -**14.** В выражении `template <...>`, между `template` и `<` ставится пробел, а после `<` и до `>` не ставится. +**14.** В выражении `template <...>`, между `template` и `<` ставится пробел, а после `<` и до `>` не ставится. ```cpp template @@ -257,7 +257,7 @@ void executeQuery( */ ``` -Пример взят с ресурса [http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/](http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/). +Пример взят с ресурса [http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/](http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/). **7.** Нельзя писать мусорные комментарии (автор, дата создания...) в начале каждого файла. @@ -305,7 +305,7 @@ void executeQuery( size_t max_block_size; ``` -**2.** Имена функций (методов) camelCase с маленькой буквы. +**2.** Имена функций (методов) camelCase с маленькой буквы. ```cpp std::string getName() const override { return "Memory"; } @@ -348,7 +348,7 @@ class IBlockInputStream bool info_successfully_loaded = false; ``` -**9.** В именах `define` и глобальных констант используется ALL_CAPS с подчёркиванием. +**9.** В именах `define` и глобальных констант используется ALL_CAPS с подчёркиванием. ```cpp #define MAX_SRC_TABLE_NAMES_TO_STORE 1000 diff --git a/docs/ru/getting_started/example_datasets/nyc_taxi.md b/docs/ru/getting_started/example_datasets/nyc_taxi.md index 3cfe7e439c5..cd86ff05b76 100644 --- a/docs/ru/getting_started/example_datasets/nyc_taxi.md +++ b/docs/ru/getting_started/example_datasets/nyc_taxi.md @@ -323,7 +323,7 @@ ORDER BY year, count(*) DESC На каждом сервере: ``` -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) +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) ``` На исходном сервере: diff --git a/docs/ru/getting_started/example_datasets/ontime.md b/docs/ru/getting_started/example_datasets/ontime.md index d08d40d6692..f138817b400 100644 --- a/docs/ru/getting_started/example_datasets/ontime.md +++ b/docs/ru/getting_started/example_datasets/ontime.md @@ -166,7 +166,7 @@ SELECT Origin, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Q4. Количество задержек по перевозчикам за 2007 год ``` sql -SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC +SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC ``` Q5. Процент задержек по перевозчикам за 2007 год diff --git a/docs/ru/index.md b/docs/ru/index.md index 24e97aaff83..2db293a36ff 100644 --- a/docs/ru/index.md +++ b/docs/ru/index.md @@ -72,9 +72,9 @@ ClickHouse - столбцовая система управления базам ### По вводу-выводу -1. Для выполнения аналитического запроса, требуется прочитать небольшое количество столбцов таблицы. В столбцовой БД для этого можно читать только нужные данные. Например, если вам требуется только 5 столбцов из 100, то следует рассчитывать на 20-кратное уменьшение ввода-вывода. -2. Так как данные читаются пачками, то их проще сжимать. Данные, лежащие по столбцам также лучше сжимаются. За счёт этого, дополнительно уменьшается объём ввода-вывода. -3. За счёт уменьшения ввода-вывода, больше данных влезает в системный кэш. +1. Для выполнения аналитического запроса, требуется прочитать небольшое количество столбцов таблицы. В столбцовой БД для этого можно читать только нужные данные. Например, если вам требуется только 5 столбцов из 100, то следует рассчитывать на 20-кратное уменьшение ввода-вывода. +2. Так как данные читаются пачками, то их проще сжимать. Данные, лежащие по столбцам также лучше сжимаются. За счёт этого, дополнительно уменьшается объём ввода-вывода. +3. За счёт уменьшения ввода-вывода, больше данных влезает в системный кэш. Например, для запроса "посчитать количество записей для каждой рекламной системы", требуется прочитать один столбец "идентификатор рекламной системы", который занимает 1 байт в несжатом виде. Если большинство переходов было не с рекламных систем, то можно рассчитывать хотя бы на десятикратное сжатие этого столбца. При использовании быстрого алгоритма сжатия, возможно разжатие данных со скоростью более нескольких гигабайт несжатых данных в секунду. То есть, такой запрос может выполняться со скоростью около нескольких миллиардов строк в секунду на одном сервере. На практике, такая скорость действительно достигается. diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index 8a0d954445a..eaf2cf132af 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -59,10 +59,10 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA При выполнении запроса, клиент показывает: -1. Прогресс выполнение запроса, который обновляется не чаще, чем 10 раз в секунду (по умолчанию). При быстрых запросах, прогресс может не успеть отобразиться. -2. Отформатированный запрос после его парсинга - для отладки. -3. Результат в заданном формате. -4. Количество строк результата, прошедшее время, а также среднюю скорость выполнения запроса. +1. Прогресс выполнение запроса, который обновляется не чаще, чем 10 раз в секунду (по умолчанию). При быстрых запросах, прогресс может не успеть отобразиться. +2. Отформатированный запрос после его парсинга - для отладки. +3. Результат в заданном формате. +4. Количество строк результата, прошедшее время, а также среднюю скорость выполнения запроса. Вы можете прервать длинный запрос, нажав Ctrl+C. При этом вам всё равно придётся чуть-чуть подождать, пока сервер остановит запрос. На некоторых стадиях выполнения, запрос невозможно прервать. Если вы не дождётесь и нажмёте Ctrl+C второй раз, то клиент будет завершён. @@ -82,7 +82,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA ### Параметры командной строки -- `--host, -h` - имя сервера, по умолчанию - localhost. Вы можете использовать как имя, так и IPv4 или IPv6 адрес. +- `--host, -h` - имя сервера, по умолчанию - localhost. Вы можете использовать как имя, так и IPv4 или IPv6 адрес. - `--port` - порт, к которому соединяться, по умолчанию - 9000. Замечу, что для HTTP и родного интерфейса используются разные порты. - `--user, -u` - имя пользователя, по умолчанию - default. - `--password` - пароль, по умолчанию - пустая строка. diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index 162eb494166..effd512ea3f 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -135,7 +135,7 @@ echo 'DROP TABLE t' | curl 'http://localhost:8123/' --data-binary @- Если вы указали в URL decompress=1, то сервер будет разжимать те данные, которые вы передаёте ему POST-ом. Также имеется возможность использования стандартного сжатия HTTP, на основе gzip. Чтобы отправить POST-запрос, сжатый с помощью gzip, добавьте к запросу заголовок `Content-Encoding: gzip`. -Чтобы ClickHouse сжимал ответ на запрос с помощью gzip, необходимо добавить `Accept-Encoding: gzip` к заголовкам запроса, и включить настройку ClickHouse `enable_http_compression`. +Чтобы ClickHouse сжимал ответ на запрос с помощью gzip, необходимо добавить `Accept-Encoding: gzip` к заголовкам запроса, и включить настройку ClickHouse `enable_http_compression`. Это может быть использовано для уменьшения трафика по сети при передаче большого количества данных, а также для создания сразу сжатых дампов. @@ -159,13 +159,13 @@ $ echo 'SELECT number FROM numbers LIMIT 10' | curl 'http://localhost:8123/?data Имя пользователя и пароль могут быть указаны в одном из двух вариантов: -1. С использованием HTTP Basic Authentification. Пример: +1. С использованием HTTP Basic Authentification. Пример: ```bash echo 'SELECT 1' | curl 'http://user:password@localhost:8123/' -d @- ``` -2. В параметрах URL user и password. Пример: +2. В параметрах URL user и password. Пример: ```bash echo 'SELECT 1' | curl 'http://localhost:8123/?user=user&password=password' -d @- diff --git a/docs/ru/introduction/distinctive_features.md b/docs/ru/introduction/distinctive_features.md index 73d2826d16c..9a01388f630 100644 --- a/docs/ru/introduction/distinctive_features.md +++ b/docs/ru/introduction/distinctive_features.md @@ -51,9 +51,9 @@ ClickHouse поддерживает таблицы с первичным клю ClickHouse предоставляет различные способы разменять точность вычислений на производительность: -1. Система содержит агрегатные функции для приближённого вычисления количества различных значений, медианы и квантилей. -2. Поддерживается возможность выполнить запрос на основе части (выборки) данных и получить приближённый результат. При этом, с диска будет считано пропорционально меньше данных. -3. Поддерживается возможность выполнить агрегацию не для всех ключей, а для ограниченного количества первых попавшихся ключей. При выполнении некоторых условий на распределение ключей в данных, это позволяет получить достаточно точный результат с использованием меньшего количества ресурсов. +1. Система содержит агрегатные функции для приближённого вычисления количества различных значений, медианы и квантилей. +2. Поддерживается возможность выполнить запрос на основе части (выборки) данных и получить приближённый результат. При этом, с диска будет считано пропорционально меньше данных. +3. Поддерживается возможность выполнить агрегацию не для всех ключей, а для ограниченного количества первых попавшихся ключей. При выполнении некоторых условий на распределение ключей в данных, это позволяет получить достаточно точный результат с использованием меньшего количества ресурсов. ## Репликация данных и поддержка целостности diff --git a/docs/ru/operations/server_settings/settings.md b/docs/ru/operations/server_settings/settings.md index 9c855fa6ff0..68b7c4becc0 100644 --- a/docs/ru/operations/server_settings/settings.md +++ b/docs/ru/operations/server_settings/settings.md @@ -456,7 +456,7 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat - requireTLSv1_2 - Требование соединения TLSv1.2. Допустимые значения: ``true``, ``false``. - fips - Активация режима OpenSSL FIPS. Поддерживается, если версия OpenSSL, с которой собрана библиотека поддерживает fips. - privateKeyPassphraseHandler - Класс (подкласс PrivateKeyPassphraseHandler)запрашивающий кодовую фразу доступа к секретному ключу. Например, ````, ``KeyFileHandler``, ``test``, ````. -- invalidCertificateHandler - Класс (подкласс CertificateHandler) для подтвеждения невалидных сертификатов. Например, `` ConsoleCertificateHandler ``. +- invalidCertificateHandler - Класс (подкласс CertificateHandler) для подтвеждения невалидных сертификатов. Например, `` ConsoleCertificateHandler ``. - disableProtocols - Запрещенные к искользованию протоколы. - preferServerCiphers - Предпочтение серверных шифров на клиенте. diff --git a/docs/ru/operations/settings/permissions_for_queries.md b/docs/ru/operations/settings/permissions_for_queries.md index 69a00ab4f8e..6eebea9fdbe 100644 --- a/docs/ru/operations/settings/permissions_for_queries.md +++ b/docs/ru/operations/settings/permissions_for_queries.md @@ -29,7 +29,7 @@ После установки `readonly = 1`пользователь не может изменить настройки `readonly` и `allow_ddl` в текущей сессии. -При использовании метода `GET` в [HTTP интерфейсе](../../interfaces/http.md#http_interface), `readonly = 1` устанавливается автоматически. Для изменения данных используйте метод `POST`. +При использовании метода `GET` в [HTTP интерфейсе](../../interfaces/http.md#http_interface), `readonly = 1` устанавливается автоматически. Для изменения данных используйте метод `POST`. **Значение по умолчанию** @@ -46,7 +46,7 @@ - 0 — DDL запросы не разрешены. - 1 — DDL запросы разрешены. -Если `allow_ddl = 0`, то невозможно выполнить `SET allow_ddl = 1` для текущей сессии. +Если `allow_ddl = 0`, то невозможно выполнить `SET allow_ddl = 1` для текущей сессии. **Значение по умолчанию** diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index a06290f51f1..ed1395a1aaf 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -16,7 +16,7 @@ ClickHouse применяет настройку в тех случаях, ко Возможные значения: -- `deny` - (по умолчанию) запрещает использование таких подзапросов (При попытке использование вернет исключение "Double-distributed IN/JOIN subqueries is denied"); +- `deny` - (по умолчанию) запрещает использование таких подзапросов (При попытке использование вернет исключение "Double-distributed IN/JOIN subqueries is denied"); - `local` - заменит базу данных и таблицу в подзапросе на локальные для конечного сервера (шарда), оставив обычный `IN` / `JOIN`; - `global` - заменит запрос `IN` / `JOIN` на `GLOBAL IN` / `GLOBAL JOIN`; - `allow` - разрешает использование таких подзапросов. diff --git a/docs/ru/operations/table_engines/collapsingmergetree.md b/docs/ru/operations/table_engines/collapsingmergetree.md index 2a1d7f0d111..9b28deb880a 100644 --- a/docs/ru/operations/table_engines/collapsingmergetree.md +++ b/docs/ru/operations/table_engines/collapsingmergetree.md @@ -31,7 +31,7 @@ CollapsingMergeTree(EventDate, (CounterID, EventDate, intHash32(UniqID), VisitID Существует несколько способов получения полностью "схлопнутых" данных из таблицы типа `CollapsingMergeTree`: -1. Написать запрос с GROUP BY и агрегатными функциями, учитывающими знак. Например, чтобы посчитать количество, надо вместо count() написать sum(Sign); чтобы посчитать сумму чего-либо, надо вместо sum(x) написать sum(Sign \* x) и т. п., а также добавить HAVING sum(Sign) `>` 0. Не все величины можно посчитать подобным образом. Например, агрегатные функции min, max не могут быть переписаны. -2. Если необходимо вынимать данные без агрегации (например, проверить наличие строк, самые новые значения которых удовлетворяют некоторым условиям), можно использовать модификатор FINAL для секции FROM. Это вариант существенно менее эффективен. +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/graphitemergetree.md b/docs/ru/operations/table_engines/graphitemergetree.md index 9142a4023a6..52e0d98df81 100644 --- a/docs/ru/operations/table_engines/graphitemergetree.md +++ b/docs/ru/operations/table_engines/graphitemergetree.md @@ -40,7 +40,7 @@ default ... ``` -При обработке записи ClickHouse проверит правила в секции `pattern`. Если имя метрики соответствует шаблону `regexp`, то применяются правила из `pattern`, в противном случае из `default`. +При обработке записи ClickHouse проверит правила в секции `pattern`. Если имя метрики соответствует шаблону `regexp`, то применяются правила из `pattern`, в противном случае из `default`. Поля шаблона правил. diff --git a/docs/ru/operations/table_engines/kafka.md b/docs/ru/operations/table_engines/kafka.md index 73b4fb395ee..4c426932596 100644 --- a/docs/ru/operations/table_engines/kafka.md +++ b/docs/ru/operations/table_engines/kafka.md @@ -23,7 +23,7 @@ Kafka SETTINGS kafka_topic_list = 'topic1,topic2', kafka_group_name = 'group1', kafka_format = 'JSONEachRow', - kafka_row_delimiter = '\n' + kafka_row_delimiter = '\n', kafka_schema = '', kafka_num_consumers = 2 ``` @@ -122,7 +122,7 @@ Kafka SETTINGS Аналогично GraphiteMergeTree, движок Kafka поддерживает расширенную конфигурацию с помощью конфигурационного файла ClickHouse. Существует два конфигурационных ключа, которые можно использовать - глобальный (`kafka`) и по топикам (`kafka_*`). Сначала применяется глобальная конфигурация, затем конфигурация по топикам (если она существует). ```xml - + cgrp smallest diff --git a/docs/ru/operations/table_engines/replication.md b/docs/ru/operations/table_engines/replication.md index 7eb9e1216c3..d80ef509986 100644 --- a/docs/ru/operations/table_engines/replication.md +++ b/docs/ru/operations/table_engines/replication.md @@ -159,10 +159,10 @@ sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data Если на одном из серверов исчезли все данные и метаданные, восстановление делается следующим образом: -1. Установите на сервер ClickHouse. Корректно пропишите подстановки в конфигурационном файле, отвечающие за идентификатор шарда и реплики, если вы их используете. -2. Если у вас были нереплицируемые таблицы, которые должны быть вручную продублированы на серверах, скопируйте их данные (в директории `/var/lib/clickhouse/data/db_name/table_name/`) с реплики. -3. Скопируйте с реплики определения таблиц, находящиеся в `/var/lib/clickhouse/metadata/`. Если в определениях таблиц, идентификатор шарда или реплики, прописаны в явном виде - исправьте их, чтобы они соответствовали данной реплике. (Альтернативный вариант - запустить сервер и сделать самостоятельно все запросы `ATTACH TABLE`, которые должны были бы быть в соответствующих .sql файлах в `/var/lib/clickhouse/metadata/`.) -4. Создайте в ZooKeeper узел `/path_to_table/replica_name/flags/force_restore_data` с любым содержимым или выполните команду для восстановления всех реплицируемых таблиц: `sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data` +1. Установите на сервер ClickHouse. Корректно пропишите подстановки в конфигурационном файле, отвечающие за идентификатор шарда и реплики, если вы их используете. +2. Если у вас были нереплицируемые таблицы, которые должны быть вручную продублированы на серверах, скопируйте их данные (в директории `/var/lib/clickhouse/data/db_name/table_name/`) с реплики. +3. Скопируйте с реплики определения таблиц, находящиеся в `/var/lib/clickhouse/metadata/`. Если в определениях таблиц, идентификатор шарда или реплики, прописаны в явном виде - исправьте их, чтобы они соответствовали данной реплике. (Альтернативный вариант - запустить сервер и сделать самостоятельно все запросы `ATTACH TABLE`, которые должны были бы быть в соответствующих .sql файлах в `/var/lib/clickhouse/metadata/`.) +4. Создайте в ZooKeeper узел `/path_to_table/replica_name/flags/force_restore_data` с любым содержимым или выполните команду для восстановления всех реплицируемых таблиц: `sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data` Затем запустите сервер (перезапустите, если уже запущен). Данные будут скачаны с реплик. diff --git a/docs/ru/query_language/agg_functions/index.md b/docs/ru/query_language/agg_functions/index.md index 5bcd7fa0301..138bae48fd3 100644 --- a/docs/ru/query_language/agg_functions/index.md +++ b/docs/ru/query_language/agg_functions/index.md @@ -4,8 +4,8 @@ ClickHouse поддерживает также: -- [Параметрические агрегатные функции](parametric_functions.md#aggregate_functions_parametric), которые помимо столбцов принимаю и другие параметры. -- [Комбинаторы](combinators.md#aggregate_functions_combinators), которые изменяют поведение агрегатных фунций. +- [Параметрические агрегатные функции](parametric_functions.md#aggregate_functions_parametric), которые помимо столбцов принимаю и другие параметры. +- [Комбинаторы](combinators.md#aggregate_functions_combinators), которые изменяют поведение агрегатных фунций. ## Обработка NULL diff --git a/docs/ru/query_language/agg_functions/reference.md b/docs/ru/query_language/agg_functions/reference.md index dafbcc07a5c..1a8aefc7e21 100644 --- a/docs/ru/query_language/agg_functions/reference.md +++ b/docs/ru/query_language/agg_functions/reference.md @@ -6,7 +6,7 @@ Считает количество строк. Принимает ноль аргументов, возвращает UInt64. Не поддерживается синтаксис `COUNT (DISTINCT x)`. Для этого существует агрегатная функция`uniq`. -Запрос вида `SELECT count() FROM table` не оптимизируется, так как количество записей в таблице нигде не хранится отдельно. Из таблицы будет выбран какой-нибудь достаточно маленький столбец, и будет посчитано количество значений в нём. +Запрос вида `SELECT count() FROM table` не оптимизируется, так как количество записей в таблице нигде не хранится отдельно. Из таблицы будет выбран какой-нибудь достаточно маленький столбец, и будет посчитано количество значений в нём. ## any(x) {#agg_function-any} diff --git a/docs/ru/query_language/alter.md b/docs/ru/query_language/alter.md index 945338d3126..f8dd65e8c45 100644 --- a/docs/ru/query_language/alter.md +++ b/docs/ru/query_language/alter.md @@ -112,10 +112,10 @@ SELECT * FROM system.parts WHERE active ```bash $ ls -l /var/lib/clickhouse/data/test/visits/ total 48 -drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_2_2_0 -drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_4_4_0 -drwxrwxrwx 2 clickhouse clickhouse 4096 May 5 02:55 detached --rw-rw-rw- 1 clickhouse clickhouse 2 May 5 02:58 increment.txt +drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_2_2_0 +drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_4_4_0 +drwxrwxrwx 2 clickhouse clickhouse 4096 May 5 02:55 detached +-rw-rw-rw- 1 clickhouse clickhouse 2 May 5 02:58 increment.txt ``` Здесь `20140317_20140323_2_2_0`, `20140317_20140323_4_4_0` - директории кусков. diff --git a/docs/ru/query_language/create.md b/docs/ru/query_language/create.md index a0027d3abc1..9f4cd95863c 100644 --- a/docs/ru/query_language/create.md +++ b/docs/ru/query_language/create.md @@ -116,7 +116,7 @@ CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [TO[db.]name] [ENGINE = eng При создании материализованного представления, нужно обязательно указать ENGINE - движок таблицы для хранения данных. -Материализованное представление работает следующим образом: при вставлении данных в таблицу, указанную в SELECT, часть вставленных данных конвертируется запросом, а результат вставляется в представление. +Материализованное представление работает следующим образом: при вставлении данных в таблицу, указанную в SELECT, часть вставленных данных конвертируется запросом, а результат вставляется в представление. Обычные представления не хранят никаких данных, а всего лишь производят чтение из другой таблицы. То есть, обычное представление - не более чем сохранённый запрос. При чтении из представления, этот сохранённый запрос, используется в качестве подзапроса в секции FROM. diff --git a/docs/ru/query_language/dicts/external_dicts_dict.md b/docs/ru/query_language/dicts/external_dicts_dict.md index 776ffb24756..94d923a6a75 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict.md +++ b/docs/ru/query_language/dicts/external_dicts_dict.md @@ -24,10 +24,10 @@ ``` -- name - Идентификатор, под которым словарь будет доступен для использования. Используйте символы `[a-zA-Z0-9_\-]`. -- [source](external_dicts_dict_sources.md) - Источник словаря. -- [layout](external_dicts_dict_layout.md) - Размещение словаря в памяти. -- [structure](external_dicts_dict_structure.md) - Структура словаря. Ключ и атрибуты, которые можно получить по ключу. -- [lifetime](external_dicts_dict_lifetime.md) - Периодичность обновления словарей. +- name - Идентификатор, под которым словарь будет доступен для использования. Используйте символы `[a-zA-Z0-9_\-]`. +- [source](external_dicts_dict_sources.md) - Источник словаря. +- [layout](external_dicts_dict_layout.md) - Размещение словаря в памяти. +- [structure](external_dicts_dict_structure.md) - Структура словаря. Ключ и атрибуты, которые можно получить по ключу. +- [lifetime](external_dicts_dict_lifetime.md) - Периодичность обновления словарей. [Оригинальная статья](https://clickhouse.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 fbac880c403..aafcf531860 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict_layout.md +++ b/docs/ru/query_language/dicts/external_dicts_dict_layout.md @@ -202,10 +202,10 @@ Укажите достаточно большой размер кэша. Количество ячеек следует подобрать экспериментальным путём: -1. Выставить некоторое значение. -2. Запросами добиться полной заполненности кэша. -3. Оценить потребление оперативной памяти с помощью таблицы `system.dictionaries`. -4. Увеличивать/уменьшать количество ячеек до получения требуемого расхода оперативной памяти. +1. Выставить некоторое значение. +2. Запросами добиться полной заполненности кэша. +3. Оценить потребление оперативной памяти с помощью таблицы `system.dictionaries`. +4. Увеличивать/уменьшать количество ячеек до получения требуемого расхода оперативной памяти. !!! warning Не используйте в качестве источника ClickHouse, поскольку он медленно обрабатывает запросы со случайным чтением. diff --git a/docs/ru/query_language/functions/array_functions.md b/docs/ru/query_language/functions/array_functions.md index af470725f3e..7cc4cb41b8e 100644 --- a/docs/ru/query_language/functions/array_functions.md +++ b/docs/ru/query_language/functions/array_functions.md @@ -144,7 +144,7 @@ hasAny(array1, array2) **Возвращаемые значения** -- `1`, если `array1` и `array2` имеют хотя бы один одинаковый элемент. +- `1`, если `array1` и `array2` имеют хотя бы один одинаковый элемент. - `0`, в противном случае. **Особенности** @@ -378,7 +378,7 @@ arrayPushFront(array, single_value) **Параметры** - `array` - Массив. -- `single_value` - Одиночное значение. В массив с числам можно добавить только числа, в массив со строками только строки. При добавлении чисел ClickHouse автоматически приводит тип `single_value` к типу данных массива. Подробнее о типах данных в ClickHouse читайте в разделе "[Типы данных](../../data_types/index.md#data_types)". Может быть равно `NULL`. Функция добавит элемент `NULL` в массив, а тип элементов массива преобразует в `Nullable`. +- `single_value` - Одиночное значение. В массив с числам можно добавить только числа, в массив со строками только строки. При добавлении чисел ClickHouse автоматически приводит тип `single_value` к типу данных массива. Подробнее о типах данных в ClickHouse читайте в разделе "[Типы данных](../../data_types/index.md#data_types)". Может быть равно `NULL`. Функция добавит элемент `NULL` в массив, а тип элементов массива преобразует в `Nullable`. **Пример** @@ -442,7 +442,7 @@ arraySlice(array, offset[, length]) - `array` - Массив данных. - `offset` - Отступ от края массива. Положительное значение - отступ слева, отрицательное значение - отступ справа. Отсчет элементов массива начинается с 1. -- `length` - Длина необходимого среза. Если указать отрицательное значение, то функция вернёт открытый срез `[offset, array_length - length)`. Если не указать значение, то функция вернёт срез `[offset, the_end_of_array]`. +- `length` - Длина необходимого среза. Если указать отрицательное значение, то функция вернёт открытый срез `[offset, array_length - length)`. Если не указать значение, то функция вернёт срез `[offset, the_end_of_array]`. **Пример** diff --git a/docs/ru/query_language/functions/json_functions.md b/docs/ru/query_language/functions/json_functions.md index 6958f33e2ba..464b037ad3a 100644 --- a/docs/ru/query_language/functions/json_functions.md +++ b/docs/ru/query_language/functions/json_functions.md @@ -4,10 +4,10 @@ Делаются следующие допущения: -1. Имя поля (аргумент функции) должно быть константой; -2. Считается, что имя поля в JSON-е закодировано некоторым каноническим образом. Например, `visitParamHas('{"abc":"def"}', 'abc') = 1`, но `visitParamHas('{"\\u0061\\u0062\\u0063":"def"}', 'abc') = 0` -3. Поля ищутся на любом уровне вложенности, без разбора. Если есть несколько подходящих полей - берётся первое. -4. В JSON-е нет пробельных символов вне строковых литералов. +1. Имя поля (аргумент функции) должно быть константой; +2. Считается, что имя поля в JSON-е закодировано некоторым каноническим образом. Например, `visitParamHas('{"abc":"def"}', 'abc') = 1`, но `visitParamHas('{"\\u0061\\u0062\\u0063":"def"}', 'abc') = 0` +3. Поля ищутся на любом уровне вложенности, без разбора. Если есть несколько подходящих полей - берётся первое. +4. В JSON-е нет пробельных символов вне строковых литералов. ## visitParamHas(params, name) Проверить наличие поля с именем name. diff --git a/docs/ru/query_language/functions/other_functions.md b/docs/ru/query_language/functions/other_functions.md index eede1a97264..453147dd4af 100644 --- a/docs/ru/query_language/functions/other_functions.md +++ b/docs/ru/query_language/functions/other_functions.md @@ -116,7 +116,7 @@ ORDER BY h ASC Преобразовать значение согласно явно указанному отображению одних элементов на другие. Имеется два варианта функции: -1. `transform(x, array_from, array_to, default)` +1. `transform(x, array_from, array_to, default)` `x` - что преобразовывать. @@ -158,7 +158,7 @@ ORDER BY c DESC └───────────┴────────┘ ``` -2. `transform(x, array_from, array_to)` +2. `transform(x, array_from, array_to)` Отличается от первого варианта отсутствующим аргументом default. Если значение x равно одному из элементов массива array_from, то возвращает соответствующий (такой же по номеру) элемент массива array_to; иначе возвращает x. diff --git a/docs/ru/query_language/misc.md b/docs/ru/query_language/misc.md index 000daf746dc..050d092befa 100644 --- a/docs/ru/query_language/misc.md +++ b/docs/ru/query_language/misc.md @@ -201,9 +201,9 @@ Readonly-пользователи могут останавливать толь Синхронный вариант (`SYNC`) ожидает остановки всех запросов и построчно выводит информацию о процессах по ходу их остановки. Ответ содержит колонку `kill_status`, которая может принимать следующие значения: -1. 'finished' - запрос был успешно остановлен; -2. 'waiting' - запросу отправлен сигнал завершения, ожидается его остановка; -3. остальные значения описывают причину невозможности остановки запроса. +1. 'finished' - запрос был успешно остановлен; +2. 'waiting' - запросу отправлен сигнал завершения, ожидается его остановка; +3. остальные значения описывают причину невозможности остановки запроса. Тестовый вариант запроса (`TEST`) только проверяет права пользователя и выводит список запросов для остановки. diff --git a/docs/ru/query_language/select.md b/docs/ru/query_language/select.md index ce1fa6de9a0..2709b24f28b 100644 --- a/docs/ru/query_language/select.md +++ b/docs/ru/query_language/select.md @@ -434,7 +434,7 @@ LIMIT 10 #### Обработка NULL -На поведение JOIN влияет настройка [join_use_nulls](../operations/settings/settings.md). При `join_use_nulls=1` `JOIN` работает как в стандартном SQL. +На поведение JOIN влияет настройка [join_use_nulls](../operations/settings/settings.md). При `join_use_nulls=1` `JOIN` работает как в стандартном SQL. Если ключами JOIN выступают поля типа [Nullable](../data_types/nullable.md), то строки, где хотя бы один из ключей имеет значение [NULL](syntax.md), не соединяются. @@ -706,9 +706,9 @@ WHERE и HAVING отличаются тем, что WHERE выполняется `DISTINCT` не поддерживается, если в `SELECT` присутствует хотя бы один столбец типа массив. -`DISTINCT` работает с [NULL](syntax.md) как если бы `NULL` был конкретным значением, причём `NULL=NULL`. Т.е. в результате `DISTINCT` разные комбинации с `NULL` встретятся только по одному разу. +`DISTINCT` работает с [NULL](syntax.md) как если бы `NULL` был конкретным значением, причём `NULL=NULL`. Т.е. в результате `DISTINCT` разные комбинации с `NULL` встретятся только по одному разу. -`DISTINCT` работает с [NULL](syntax.md) как если бы `NULL` был конкретным значением, причём `NULL=NULL`. Т.е. в результате `DISTINCT` разные комбинации с `NULL` встретятся только по одному разу. +`DISTINCT` работает с [NULL](syntax.md) как если бы `NULL` был конкретным значением, причём `NULL=NULL`. Т.е. в результате `DISTINCT` разные комбинации с `NULL` встретятся только по одному разу. ### Секция LIMIT @@ -830,7 +830,7 @@ ORDER BY EventDate ASC #### Обработка NULL -При обработке запроса оператор IN будет считать, что результат операции с [NULL](syntax.md) всегда равен `0`, независимо от того, находится `NULL` в правой или левой части оператора. Значения `NULL` не входят ни в какое множество, не соответствуют друг другу и не могут сравниваться. +При обработке запроса оператор IN будет считать, что результат операции с [NULL](syntax.md) всегда равен `0`, независимо от того, находится `NULL` в правой или левой части оператора. Значения `NULL` не входят ни в какое множество, не соответствуют друг другу и не могут сравниваться. Рассмотрим для примера таблицу `t_null`: @@ -955,11 +955,11 @@ SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID GLOBAL Это гораздо более оптимально, чем при использовании обычного IN. Но при этом, следует помнить о нескольких вещах: -1. При создании временной таблицы данные не уникализируются. Чтобы уменьшить объём передаваемых по сети данных, укажите в подзапросе DISTINCT (для обычного IN-а этого делать не нужно). -2. Временная таблица будет передана на все удалённые серверы. Передача не учитывает топологию сети. Например, если 10 удалённых серверов расположены в удалённом относительно сервера-инициатора запроса датацентре, то по каналу в удалённый датацентр данные будет переданы 10 раз. Старайтесь не использовать большие множества при использовании GLOBAL IN. -3. При передаче данных на удалённые серверы не настраивается ограничение использования сетевой полосы. Вы можете перегрузить сеть. -4. Старайтесь распределять данные по серверам так, чтобы в GLOBAL IN-ах не было частой необходимости. -5. Если в GLOBAL IN есть частая необходимость, то спланируйте размещение кластера ClickHouse таким образом, чтобы в каждом датацентре была хотя бы одна реплика каждого шарда, и среди них была быстрая сеть - чтобы запрос целиком можно было бы выполнить, передавая данные в пределах одного датацентра. +1. При создании временной таблицы данные не уникализируются. Чтобы уменьшить объём передаваемых по сети данных, укажите в подзапросе DISTINCT (для обычного IN-а этого делать не нужно). +2. Временная таблица будет передана на все удалённые серверы. Передача не учитывает топологию сети. Например, если 10 удалённых серверов расположены в удалённом относительно сервера-инициатора запроса датацентре, то по каналу в удалённый датацентр данные будет переданы 10 раз. Старайтесь не использовать большие множества при использовании GLOBAL IN. +3. При передаче данных на удалённые серверы не настраивается ограничение использования сетевой полосы. Вы можете перегрузить сеть. +4. Старайтесь распределять данные по серверам так, чтобы в GLOBAL IN-ах не было частой необходимости. +5. Если в GLOBAL IN есть частая необходимость, то спланируйте размещение кластера ClickHouse таким образом, чтобы в каждом датацентре была хотя бы одна реплика каждого шарда, и среди них была быстрая сеть - чтобы запрос целиком можно было бы выполнить, передавая данные в пределах одного датацентра. В секции `GLOBAL IN` также имеет смысл указывать локальную таблицу - в случае, если эта локальная таблица есть только на сервере-инициаторе запроса, и вы хотите воспользоваться данными из неё на удалённых серверах. diff --git a/docs/tools/mdx_clickhouse.py b/docs/tools/mdx_clickhouse.py index ae57d1309e6..a2d71b0ea58 100755 --- a/docs/tools/mdx_clickhouse.py +++ b/docs/tools/mdx_clickhouse.py @@ -47,6 +47,9 @@ class ClickHousePreprocessor(markdown.util.Processor): for line in lines: if '' not in line: yield line + else: + for line in lines: + yield line class ClickHouseMarkdown(markdown.extensions.Extension): diff --git a/docs/zh/data_types/array.md b/docs/zh/data_types/array.md index b6764498183..e9e59f2ed6f 100644 --- a/docs/zh/data_types/array.md +++ b/docs/zh/data_types/array.md @@ -48,7 +48,7 @@ SELECT ## 使用数据类型 -ClickHouse会自动检测数组元素,并根据元素计算出存储这些元素最小的数据类型。如果在元素中存在[NULL](../query_language/syntax.md#null-literal)或存在[Nullable](nullable.md#data_type-nullable)类型元素,那么数组的元素类型将会变成[Nullable](nullable.md)。 +ClickHouse会自动检测数组元素,并根据元素计算出存储这些元素最小的数据类型。如果在元素中存在 [NULL](../query_language/syntax.md#null-literal) 或存在 [Nullable](nullable.md#data_type-nullable) 类型元素,那么数组的元素类型将会变成 [Nullable](nullable.md)。 如果 ClickHouse 无法确定数据类型,它将产生异常。当尝试同时创建一个包含字符串和数字的数组时会发生这种情况 (`SELECT array(1, 'a')`)。 diff --git a/docs/zh/data_types/float.md b/docs/zh/data_types/float.md index 7c71fc5e260..113e640e611 100644 --- a/docs/zh/data_types/float.md +++ b/docs/zh/data_types/float.md @@ -5,7 +5,7 @@ 类型与以下 C 语言中类型是相同的: - `Float32` - `float` -- `Float64` - `double` +- `Float64` - `double` 我们建议您尽可能以整数形式存储数据。例如,将固定精度的数字转换为整数值,例如货币数量或页面加载时间用毫秒为单位表示 @@ -69,4 +69,4 @@ SELECT 0 / 0 └──────────────┘ ``` -可以在[ORDER BY 子句](../query_language/select.md) 查看更多关于 ` NaN` 排序的规则。 +可以在 [ORDER BY 子句](../query_language/select.md) 查看更多关于 ` NaN` 排序的规则。 diff --git a/docs/zh/development/style.md b/docs/zh/development/style.md index d31cd450b6e..828ef44bfe9 100644 --- a/docs/zh/development/style.md +++ b/docs/zh/development/style.md @@ -261,7 +261,7 @@ void executeQuery( **7.** 不要在每个文件的开头写入垃圾注释(作者,创建日期...)。 -**8.** 单行注释用三个斜杆: `///` ,多行注释以 `/**`开始。 这些注释会当做文档。 +**8.** 单行注释用三个斜杆: `///` ,多行注释以 `/**`开始。 这些注释会当做文档。 注意:您可以使用 Doxygen 从这些注释中生成文档。 但是通常不使用 Doxygen,因为在 IDE 中导航代码更方便。 @@ -349,7 +349,7 @@ class IBlockInputStream bool info_successfully_loaded = false; ``` -**9.** `define` 和全局常量的名称使用带下划线的 `ALL_CAPS`。 +**9.** `define` 和全局常量的名称使用带下划线的 `ALL_CAPS`。 ```cpp #define MAX_SRC_TABLE_NAMES_TO_STORE 1000 @@ -564,7 +564,7 @@ Fork不用于并行化。 **12.** 数值类型。 -使用 `UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, 以及 `Int64`, `size_t`, `ssize_t` 还有 `ptrdiff_t`。 +使用 `UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, 以及 `Int64`, `size_t`, `ssize_t` 还有 `ptrdiff_t`。 不要使用这些类型:`signed / unsigned long`,`long long`,`short`,`signed / unsigned char`,`char`。 diff --git a/docs/zh/faq/general.md b/docs/zh/faq/general.md index adc0b5791b4..9a1aef5dde6 100644 --- a/docs/zh/faq/general.md +++ b/docs/zh/faq/general.md @@ -2,7 +2,7 @@ ## 为什么不使用MapReduce之类的产品呢? -我们可以将MapReduce这类的系统称为分布式计算系统,其reduce操作基于分布式排序。其中最常见的开源解决方案是[Apache Hadoop](http://hadoop.apache.org)。 Yandex使用他们的内部解决方案YT。 +我们可以将MapReduce这类的系统称为分布式计算系统,其reduce操作基于分布式排序。其中最常见的开源解决方案是 [Apache Hadoop](http://hadoop.apache.org)。 Yandex使用他们的内部解决方案YT。 这些系统不适合在线查询,因为它们的延迟高。换句话说,它们不能用作Web接口的后端服务。这些系统对于实时数据更新是没有用的。如果操作的结果和所有中间结果(如果有的话)位于单个服务器的内存中,则分布式排序不是执行reduce操作的最佳方式,但这通常是在线查询的情况。在这种情况下,哈希表是执行reduce操作的最佳方式。优化map-reduce任务的常用方法是使用内存中的哈希表进行预聚合(部分reduce),用户手动执行此优化操作。分布式排序是运行简单map-reduce任务时性能降低的主要原因之一。 diff --git a/docs/zh/getting_started/example_datasets/nyc_taxi.md b/docs/zh/getting_started/example_datasets/nyc_taxi.md index a5b2bd82dea..16adae18120 100644 --- a/docs/zh/getting_started/example_datasets/nyc_taxi.md +++ b/docs/zh/getting_started/example_datasets/nyc_taxi.md @@ -320,7 +320,7 @@ Two Intel(R) Xeon(R) CPU E5-2650 v2 @ 2.60GHz, 16 physical kernels total,128 GiB 在每台服务器中运行: ``` -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) +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) ``` 在之前的服务器中运行: diff --git a/docs/zh/getting_started/example_datasets/ontime.md b/docs/zh/getting_started/example_datasets/ontime.md index 49d48fb1660..b5f78352f4c 100644 --- a/docs/zh/getting_started/example_datasets/ontime.md +++ b/docs/zh/getting_started/example_datasets/ontime.md @@ -166,7 +166,7 @@ SELECT Origin, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Q4. 查询2007年各航空公司延误超过10分钟以上的次数 ```sql -SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC +SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC ``` Q5. 查询2007年各航空公司延误超过10分钟以上的百分比 diff --git a/docs/zh/interfaces/cli.md b/docs/zh/interfaces/cli.md index 7a61a315b2b..933b43c9469 100644 --- a/docs/zh/interfaces/cli.md +++ b/docs/zh/interfaces/cli.md @@ -77,7 +77,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA ### 命令行参数 -- `--host, -h` -– 服务端的 host 名称, 默认是 'localhost'。 您可以选择使用 host 名称或者 IPv4 或 IPv6 地址。 +- `--host, -h` -– 服务端的 host 名称, 默认是 'localhost'。 您可以选择使用 host 名称或者 IPv4 或 IPv6 地址。 - `--port` – 连接的端口,默认值: 9000。注意 HTTP 接口以及 TCP 原生接口是使用不同端口的。 - `--user, -u` – 用户名。 默认值: default。 - `--password` – 密码。 默认值: 空字符串。 @@ -93,7 +93,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA ### 配置文件 -`clickhouse-client` 使用一下第一个存在的文件: +`clickhouse-client` 使用一下第一个存在的文件: - 通过 `-config-file` 参数指定的文件. - `./clickhouse-client.xml` diff --git a/docs/zh/interfaces/formats.md b/docs/zh/interfaces/formats.md index e10532174ff..80985542fac 100644 --- a/docs/zh/interfaces/formats.md +++ b/docs/zh/interfaces/formats.md @@ -159,7 +159,7 @@ x=1 y=\N clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv ``` -*默认情况下间隔符是 `,` ,在[format_csv_delimiter](../operations/settings/settings.md#format_csv_delimiter)中可以了解更多间隔符配置。 +*默认情况下间隔符是 `,` ,在 [format_csv_delimiter](../operations/settings/settings.md#format_csv_delimiter) 中可以了解更多间隔符配置。 解析的时候,可以使用或不使用引号来解析所有值。支持双引号和单引号。行也可以不用引号排列。 在这种情况下,它们被解析为逗号或换行符(CR 或 LF)。在解析不带引号的行时,若违反 RFC 规则,会忽略前导和尾随的空格和制表符。 对于换行,全部支持 Unix(LF),Windows(CR LF)和 Mac OS Classic(CR LF)。 @@ -256,7 +256,7 @@ JSON 与 JavaScript 兼容。为了确保这一点,一些字符被另外转义 该格式仅适用于输出查询结果,但不适用于解析输入(将数据插入到表中)。 -ClickHouse 支持 [NULL](../query_language/syntax.md), 在 JSON 格式中以 `null` 输出来表示. +ClickHouse 支持 [NULL](../query_language/syntax.md), 在 JSON 格式中以 `null` 输出来表示. 参考 JSONEachRow 格式。 @@ -428,10 +428,10 @@ watch -n1 "clickhouse-client --query='SELECT event, value FROM system.events FOR 整数使用固定长度的小端表示法。 例如,UInt64 使用8个字节。 DateTime 被表示为 UInt32 类型的Unix 时间戳值。 Date 被表示为 UInt16 对象,它的值为 1970-01-01以来的天数。 -字符串表示为 varint 长度(无符号[LEB128](https://en.wikipedia.org/wiki/LEB128)),后跟字符串的字节数。 +字符串表示为 varint 长度(无符号 [LEB128](https://en.wikipedia.org/wiki/LEB128)),后跟字符串的字节数。 FixedString 被简单地表示为一个字节序列。 -数组表示为 varint 长度(无符号[LEB128](https://en.wikipedia.org/wiki/LEB128)),后跟有序的数组元素。 +数组表示为 varint 长度(无符号 [LEB128](https://en.wikipedia.org/wiki/LEB128)),后跟有序的数组元素。 对于 [NULL](../query_language/syntax.md#null-literal) 的支持, 一个为 1 或 0 的字节会加在每个 [Nullable](../data_types/nullable.md) 值前面。如果为 1, 那么该值就是 `NULL`。 如果为 0,则不为 `NULL`。 @@ -587,7 +587,7 @@ struct Message { } ``` -格式文件存储的目录可以在服务配置中的[ format_schema_path ](../operations/server_settings/settings.md) 指定。 +格式文件存储的目录可以在服务配置中的 [ format_schema_path ](../operations/server_settings/settings.md) 指定。 Cap'n Proto 反序列化是很高效的,通常不会增加系统的负载。 diff --git a/docs/zh/interfaces/http.md b/docs/zh/interfaces/http.md index cbf2f868ce1..8700f0be3c9 100644 --- a/docs/zh/interfaces/http.md +++ b/docs/zh/interfaces/http.md @@ -10,7 +10,7 @@ $ curl 'http://localhost:8123/' Ok. ``` -通过 URL 中的 `query` 参数来发送请求,或者发送 POST 请求,或者将查询的开头部分放在 URL 的 `query` 参数中,其他部分放在 POST 中(我们会在后面解释为什么这样做是有必要的)。URL 的大小会限制在 16 KB,所以发送大型查询时要时刻记住这点。 +通过 URL 中的 `query` 参数来发送请求,或者发送 POST 请求,或者将查询的开头部分放在 URL 的 `query` 参数中,其他部分放在 POST 中(我们会在后面解释为什么这样做是有必要的)。URL 的大小会限制在 16 KB,所以发送大型查询时要时刻记住这点。 如果请求成功,将会收到 200 的响应状态码和响应主体中的结果。 如果发生了某个异常,将会收到 500 的响应状态码和响应主体中的异常描述信息。 diff --git a/docs/zh/interfaces/jdbc.md b/docs/zh/interfaces/jdbc.md index 41bf39ab2f7..61656c35217 100644 --- a/docs/zh/interfaces/jdbc.md +++ b/docs/zh/interfaces/jdbc.md @@ -1,6 +1,6 @@ # JDBC 驱动 -- ClickHouse官方有 JDBC 的驱动。 见[这里](https://github.com/yandex/clickhouse-jdbc)。 +- ClickHouse官方有 JDBC 的驱动。 见 [这里](https://github.com/yandex/clickhouse-jdbc)。 - 三方提供的 JDBC 驱动 [ClickHouse-Native-JDBC](https://github.com/housepower/ClickHouse-Native-JDBC). diff --git a/docs/zh/interfaces/odbc.md b/docs/zh/interfaces/odbc.md index e24e557f090..2de4bee64fc 100644 --- a/docs/zh/interfaces/odbc.md +++ b/docs/zh/interfaces/odbc.md @@ -1,5 +1,5 @@ # ODBC 驱动 -- ClickHouse官方有 ODBC 的驱动。 见[这里](https://github.com/yandex/clickhouse-odbc)。 +- ClickHouse官方有 ODBC 的驱动。 见 [这里](https://github.com/yandex/clickhouse-odbc)。 [来源文章](https://clickhouse.yandex/docs/zh/interfaces/odbc/) diff --git a/docs/zh/interfaces/tcp.md b/docs/zh/interfaces/tcp.md index c60fc2e09aa..7eab7dea1b1 100644 --- a/docs/zh/interfaces/tcp.md +++ b/docs/zh/interfaces/tcp.md @@ -1,5 +1,5 @@ # 原生客户端接口(TCP) -本机协议用于[命令行客户端](cli.md),用于分布式查询处理期间的服务器间通信,以及其他C ++程序。 不幸的是,本机ClickHouse协议还没有正式的规范,但它可以从ClickHouse源代码进行逆向工程[从这里开始](https://github.com/yandex/ClickHouse/tree/master/dbms/src/Client))和/或拦截和分析TCP流量。 +本机协议用于 [命令行客户端](cli.md),用于分布式查询处理期间的服务器间通信,以及其他C ++程序。 不幸的是,本机ClickHouse协议还没有正式的规范,但它可以从ClickHouse源代码进行逆向工程 [从这里开始](https://github.com/yandex/ClickHouse/tree/master/dbms/src/Client))和/或拦截和分析TCP流量。 [来源文章](https://clickhouse.yandex/docs/zh/interfaces/tcp/) diff --git a/docs/zh/interfaces/third-party/gui.md b/docs/zh/interfaces/third-party/gui.md index e2f4a50d336..35e4717a497 100644 --- a/docs/zh/interfaces/third-party/gui.md +++ b/docs/zh/interfaces/third-party/gui.md @@ -40,7 +40,7 @@ ClickHouse Web 界面 [Tabix](https://github.com/tabixio/tabix). ### LightHouse -[LightHouse](https://github.com/VKCOM/lighthouse)是ClickHouse的轻量级Web界面。 +[LightHouse](https://github.com/VKCOM/lighthouse) 是ClickHouse的轻量级Web界面。 特征: @@ -50,7 +50,7 @@ ClickHouse Web 界面 [Tabix](https://github.com/tabixio/tabix). ### clickhouse-cli -[clickhouse-cli](https://github.com/hatarist/clickhouse-cli)是ClickHouse的替代命令行客户端,用Python 3编写。 +[clickhouse-cli](https://github.com/hatarist/clickhouse-cli) 是ClickHouse的替代命令行客户端,用Python 3编写。 特征: - 自动完成。 @@ -62,7 +62,7 @@ ClickHouse Web 界面 [Tabix](https://github.com/tabixio/tabix). ### DBeaver -[DBeaver](https://dbeaver.io/)具有ClickHouse支持的通用桌面数据库客户端。 +[DBeaver](https://dbeaver.io/) 具有ClickHouse支持的通用桌面数据库客户端。 特征: @@ -72,7 +72,7 @@ ClickHouse Web 界面 [Tabix](https://github.com/tabixio/tabix). ### DataGrip -[DataGrip](https://www.jetbrains.com/datagrip/)是JetBrains的数据库IDE,专门支持ClickHouse。 它还嵌入到其他基于IntelliJ的工具中:PyCharm,IntelliJ IDEA,GoLand,PhpStorm等。 +[DataGrip](https://www.jetbrains.com/datagrip/) 是JetBrains的数据库IDE,专门支持ClickHouse。 它还嵌入到其他基于IntelliJ的工具中:PyCharm,IntelliJ IDEA,GoLand,PhpStorm等。 特征: diff --git a/docs/zh/interfaces/third-party/proxy.md b/docs/zh/interfaces/third-party/proxy.md index f4f7c528440..ee5ff42bbb9 100644 --- a/docs/zh/interfaces/third-party/proxy.md +++ b/docs/zh/interfaces/third-party/proxy.md @@ -1,6 +1,6 @@ # 来自第三方开发人员的代理服务器 -[chproxy](https://github.com/Vertamedia/chproxy)是ClickHouse数据库的http代理和负载均衡器。 +[chproxy](https://github.com/Vertamedia/chproxy) 是ClickHouse数据库的http代理和负载均衡器。 特征 @@ -12,7 +12,7 @@ ## KittenHouse -[KittenHouse](https://github.com/VKCOM/kittenhouse)设计为ClickHouse和应用程序服务器之间的本地代理,以防在应用程序端缓冲INSERT数据是不可能或不方便的。 +[KittenHouse](https://github.com/VKCOM/kittenhouse) 设计为ClickHouse和应用程序服务器之间的本地代理,以防在应用程序端缓冲INSERT数据是不可能或不方便的。 特征: @@ -24,7 +24,7 @@ ## ClickHouse-Bulk -[ClickHouse-Bulk](https://github.com/nikepan/clickhouse-bulk)是一个简单的ClickHouse插入收集器。 +[ClickHouse-Bulk](https://github.com/nikepan/clickhouse-bulk) 是一个简单的ClickHouse插入收集器。 特征: diff --git a/docs/zh/introduction/distinctive_features.md b/docs/zh/introduction/distinctive_features.md index 6a9557a301c..0308db9e765 100644 --- a/docs/zh/introduction/distinctive_features.md +++ b/docs/zh/introduction/distinctive_features.md @@ -59,6 +59,6 @@ ClickHouse提供各种各样在允许牺牲数据精度的情况下对查询进 ClickHouse使用异步的多主复制技术。当数据被写入任何一个可用副本后,系统会在后台将数据分发给其他副本,以保证系统在不同副本上保持相同的数据。在大多数情况下ClickHouse能在故障后自动恢复,在一些复杂的情况下需要少量的手动恢复。 -更多信息,参见[数据复制](../operations/table_engines/replication.md)。 +更多信息,参见 [数据复制](../operations/table_engines/replication.md)。 [来源文章](https://clickhouse.yandex/docs/en/introduction/distinctive_features/) diff --git a/docs/zh/introduction/features_considered_disadvantages.md b/docs/zh/introduction/features_considered_disadvantages.md index 7a1c2721584..4deea397c2f 100644 --- a/docs/zh/introduction/features_considered_disadvantages.md +++ b/docs/zh/introduction/features_considered_disadvantages.md @@ -1,7 +1,7 @@ # ClickHouse可以考虑缺点的功能 1. 没有完整的事物支持。 -2. 缺少高频率,低延迟的修改或删除已存在数据的能力。仅能用于批量删除或修改数据,但这符合[GDPR](https://gdpr-info.eu)。 +2. 缺少高频率,低延迟的修改或删除已存在数据的能力。仅能用于批量删除或修改数据,但这符合 [GDPR](https://gdpr-info.eu)。 3. 稀疏索引使得ClickHouse不适合通过其键检索单行的点查询。 [来源文章](https://clickhouse.yandex/docs/zh/introduction/features_considered_disadvantages/) diff --git a/docs/zh/introduction/performance.md b/docs/zh/introduction/performance.md index f06e6892187..a740db13995 100644 --- a/docs/zh/introduction/performance.md +++ b/docs/zh/introduction/performance.md @@ -1,8 +1,8 @@ # Performance -根据Yandex的内部测试结果,ClickHouse表现出了比同类可比较产品更优的性能。你可以在[这里](https://clickhouse.yandex/benchmark.html)查看具体的测试结果。 +根据Yandex的内部测试结果,ClickHouse表现出了比同类可比较产品更优的性能。你可以在 [这里](https://clickhouse.yandex/benchmark.html) 查看具体的测试结果。 -许多其他的测试也证实这一点。你可以使用互联网搜索到它们,或者你也可以从[我们收集的部分相关连接](https://clickhouse.yandex/#independent-benchmarks)中查看。 +许多其他的测试也证实这一点。你可以使用互联网搜索到它们,或者你也可以从 [我们收集的部分相关连接](https://clickhouse.yandex/#independent-benchmarks) 中查看。 ## 单个大查询的吞吐量 diff --git a/docs/zh/introduction/ya_metrika_task.md b/docs/zh/introduction/ya_metrika_task.md index 72d8f1bddd5..f4758ae4e99 100644 --- a/docs/zh/introduction/ya_metrika_task.md +++ b/docs/zh/introduction/ya_metrika_task.md @@ -1,6 +1,6 @@ # Yandex.Metrica的使用案例 -ClickHouse最初是为[Yandex.Metrica](https://metrica.yandex.com/)[世界第二大Web分析平台](http://w3techs.com/technologies/overview/traffic_analysis/all)而开发的。多年来一直作为该系统的核心组件被该系统持续使用着。目前为止,该系统在ClickHouse中有超过13万亿条记录,并且每天超过200多亿个事件被处理。它允许直接从原始数据中动态查询并生成报告。本文简要介绍了ClickHouse在其早期发展阶段的目标。 +ClickHouse最初是为 [Yandex.Metrica](https://metrica.yandex.com/) [世界第二大Web分析平台](http://w3techs.com/technologies/overview/traffic_analysis/all) 而开发的。多年来一直作为该系统的核心组件被该系统持续使用着。目前为止,该系统在ClickHouse中有超过13万亿条记录,并且每天超过200多亿个事件被处理。它允许直接从原始数据中动态查询并生成报告。本文简要介绍了ClickHouse在其早期发展阶段的目标。 Yandex.Metrica基于用户定义的字段,对实时访问、连接会话,生成实时的统计报表。这种需求往往需要复杂聚合方式,比如对访问用户进行去重。构建报表的数据,是实时接收存储的新数据。 diff --git a/docs/zh/operations/configuration_files.md b/docs/zh/operations/configuration_files.md index acbc37634d8..eafa5d34235 100644 --- a/docs/zh/operations/configuration_files.md +++ b/docs/zh/operations/configuration_files.md @@ -12,7 +12,7 @@ If `replace` is specified, it replaces the entire element with the specified one If `remove` is specified, it deletes the element. -The config can also define "substitutions". If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the [include_from](server_settings/settings.md#server_settings-include_from) element in the server config. The substitution values are specified in `/yandex/substitution_name` elements in this file. If a substitution specified in `incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional="true"` attribute (for example, settings for [macros](#macros) server_settings/settings.md)). +The config can also define "substitutions". If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the [include_from](server_settings/settings.md#server_settings-include_from) element in the server config. The substitution values are specified in `/yandex/substitution_name` elements in this file. If a substitution specified in `incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional="true"` attribute (for example, settings for [macros](#macros) server_settings/settings.md)). Substitutions can also be performed from ZooKeeper. To do this, specify the attribute `from_zk = "/path/to/node"`. The element value is replaced with the contents of the node at `/path/to/node` in ZooKeeper. You can also put an entire XML subtree on the ZooKeeper node and it will be fully inserted into the source element. diff --git a/docs/zh/operations/server_settings/settings.md b/docs/zh/operations/server_settings/settings.md index 0eadd7ebcec..5b86bc068c5 100644 --- a/docs/zh/operations/server_settings/settings.md +++ b/docs/zh/operations/server_settings/settings.md @@ -42,7 +42,7 @@ Block field ``: - ``min_part_size_ratio`` – The ratio of the minimum size of a table part to the full size of the table. - ``method`` – Compression method. Acceptable values ​: ``lz4`` or ``zstd``(experimental). -ClickHouse checks `min_part_size` and `min_part_size_ratio` and processes the `case` blocks that match these conditions. If none of the `` matches, ClickHouse applies the `lz4` compression algorithm. +ClickHouse checks `min_part_size` and `min_part_size_ratio` and processes the `case` blocks that match these conditions. If none of the `` matches, ClickHouse applies the `lz4` compression algorithm. **Example** @@ -456,7 +456,7 @@ Keys for server/client settings: - requireTLSv1 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. - fips – Activates OpenSSL FIPS mode. Supported if the library's OpenSSL version supports FIPS. - privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ````, ``KeyFileHandler``, ``test``, ````. -- invalidCertificateHandler – Class (subclass of CertificateHandler) for verifying invalid certificates. For example: `` ConsoleCertificateHandler `` . +- invalidCertificateHandler – Class (subclass of CertificateHandler) for verifying invalid certificates. For example: `` ConsoleCertificateHandler `` . - disableProtocols – Protocols that are not allowed to use. - preferServerCiphers – Preferred server ciphers on the client. @@ -640,7 +640,7 @@ The uncompressed cache is advantageous for very short queries in individual case ## user_files_path {#server_settings-user_files_path} -The directory with user files. Used in the table function [file()](../../query_language/table_functions/file.md). +The directory with user files. Used in the table function [file()](../../query_language/table_functions/file.md). **Example** diff --git a/docs/zh/operations/settings/settings.md b/docs/zh/operations/settings/settings.md index 3a456749a78..4a40828babb 100644 --- a/docs/zh/operations/settings/settings.md +++ b/docs/zh/operations/settings/settings.md @@ -28,7 +28,7 @@ Forces a query to an out-of-date replica if updated data is not available. See " ClickHouse selects the most relevant from the outdated replicas of the table. -Used when performing `SELECT` from a distributed table that points to replicated tables. +Used when performing `SELECT` from a distributed table that points to replicated tables. By default, 1 (enabled). @@ -38,7 +38,7 @@ Disables query execution if the index can't be used by date. Works with tables in the MergeTree family. -If `force_index_by_date=1`, ClickHouse checks whether the query has a date key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For example, the condition `Date != ' 2000-01-01 '` is acceptable even when it matches all the data in the table (i.e., running the query requires a full scan). For more information about ranges of data in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". +If `force_index_by_date=1`, ClickHouse checks whether the query has a date key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For example, the condition `Date != ' 2000-01-01 '` is acceptable even when it matches all the data in the table (i.e., running the query requires a full scan). For more information about ranges of data in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". ## force_primary_key @@ -47,7 +47,7 @@ Disables query execution if indexing by the primary key is not possible. Works with tables in the MergeTree family. -If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For more information about data ranges in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". +If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For more information about data ranges in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". ## fsync_metadata @@ -126,7 +126,7 @@ Sets the time in seconds. If a replica lags more than the set value, this replic Default value: 0 (off). -Used when performing `SELECT` from a distributed table that points to replicated tables. +Used when performing `SELECT` from a distributed table that points to replicated tables. ## max_threads {#settings-max_threads} @@ -311,7 +311,7 @@ Replica lag is not controlled. Enable compilation of queries. By default, 0 (disabled). Compilation is only used for part of the query-processing pipeline: for the first stage of aggregation (GROUP BY). -If this portion of the pipeline was compiled, the query may run faster due to deployment of short cycles and inlining aggregate function calls. The maximum performance improvement (up to four times faster in rare cases) is seen for queries with multiple simple aggregate functions. Typically, the performance gain is insignificant. In very rare cases, it may slow down query execution. +If this portion of the pipeline was compiled, the query may run faster due to deployment of short cycles and inlining aggregate function calls. The maximum performance improvement (up to four times faster in rare cases) is seen for queries with multiple simple aggregate functions. Typically, the performance gain is insignificant. In very rare cases, it may slow down query execution. ## min_count_to_compile @@ -329,7 +329,7 @@ It works for JSONEachRow and TSKV formats. ## output_format_json_quote_64bit_integers -If the value is true, integers appear in quotes when using JSON\* Int64 and UInt64 formats (for compatibility with most JavaScript implementations); otherwise, integers are output without the quotes. +If the value is true, integers appear in quotes when using JSON\* Int64 and UInt64 formats (for compatibility with most JavaScript implementations); otherwise, integers are output without the quotes. ## format_csv_delimiter {#format_csv_delimiter} @@ -358,7 +358,7 @@ The default value is 0. All the replicas in the quorum are consistent, i.e., they contain data from all previous `INSERT` queries. The `INSERT` sequence is linearized. -When reading the data written from the `insert_quorum`, you can use the[select_sequential_consistency](#select-sequential-consistency) option. +When reading the data written from the `insert_quorum`, you can use the [select_sequential_consistency](#select-sequential-consistency) option. **ClickHouse generates an exception** diff --git a/docs/zh/operations/settings/settings_profiles.md b/docs/zh/operations/settings/settings_profiles.md index 3953ada8a83..c335e249212 100644 --- a/docs/zh/operations/settings/settings_profiles.md +++ b/docs/zh/operations/settings/settings_profiles.md @@ -2,7 +2,7 @@ # Settings profiles A settings profile is a collection of settings grouped under the same name. Each ClickHouse user has a profile. -To apply all the settings in a profile, set the `profile` setting. +To apply all the settings in a profile, set the `profile` setting. Example: @@ -58,9 +58,9 @@ Example: ``` -The example specifies two profiles: `default` and `web`. The `default` profile has a special purpose: it must always be present and is applied when starting the server. In other words, the `default` profile contains default settings. The `web` profile is a regular profile that can be set using the `SET` query or using a URL parameter in an HTTP query. +The example specifies two profiles: `default` and `web`. The `default` profile has a special purpose: it must always be present and is applied when starting the server. In other words, the `default` profile contains default settings. The `web` profile is a regular profile that can be set using the `SET` query or using a URL parameter in an HTTP query. -Settings profiles can inherit from each other. To use inheritance, indicate the `profile` setting before the other settings that are listed in the 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/zh/operations/system_tables.md b/docs/zh/operations/system_tables.md index b41f7f02b95..d15d392d5f9 100644 --- a/docs/zh/operations/system_tables.md +++ b/docs/zh/operations/system_tables.md @@ -141,7 +141,7 @@ Formats: - active (UInt8) – Indicates whether the part is active. If a part is active, it is used in a table; otherwise, it will be deleted. Inactive data parts remain after merging. -- marks (UInt64) – The number of marks. To get the approximate number of rows in a data part, multiply ``marks`` by the index granularity (usually 8192). +- marks (UInt64) – The number of marks. To get the approximate number of rows in a data part, multiply ``marks`` by the index granularity (usually 8192). - marks_size (UInt64) – The size of the file with marks. @@ -373,7 +373,7 @@ If the path specified in 'path' doesn't exist, an exception will be thrown. Columns: -- `name String` — The name of the node. +- `name String` — The name of the node. - `path String` — The path to the node. - `value String` — Node value. - `dataLength Int32` — Size of the value. diff --git a/docs/zh/operations/table_engines/custom_partitioning_key.md b/docs/zh/operations/table_engines/custom_partitioning_key.md index 98df0bfc1c8..9e723efd42e 100644 --- a/docs/zh/operations/table_engines/custom_partitioning_key.md +++ b/docs/zh/operations/table_engines/custom_partitioning_key.md @@ -38,7 +38,7 @@ In the `system.parts` table, the `partition` column specifies the value of the p Old: `20140317_20140323_2_2_0` (minimum date - maximum date - minimum block number - maximum block number - level). -Now: `201403_2_2_0` (partition ID - minimum block number - maximum block number - level). +Now: `201403_2_2_0` (partition ID - minimum block number - maximum block number - level). The partition ID is its string identifier (human-readable, if possible) that is used for the names of data parts in the file system and in ZooKeeper. You can specify it in ALTER queries in place of the partition key. Example: Partition key `toYYYYMM(EventDate)`; ALTER can specify either `PARTITION 201710` or `PARTITION ID '201710'`. diff --git a/docs/zh/operations/table_engines/kafka.md b/docs/zh/operations/table_engines/kafka.md index d592c5a958e..22e001f2c1c 100644 --- a/docs/zh/operations/table_engines/kafka.md +++ b/docs/zh/operations/table_engines/kafka.md @@ -23,7 +23,7 @@ Kafka SETTINGS kafka_topic_list = 'topic1,topic2', kafka_group_name = 'group1', kafka_format = 'JSONEachRow', - kafka_row_delimiter = '\n' + kafka_row_delimiter = '\n', kafka_schema = '', kafka_num_consumers = 2 ``` @@ -121,7 +121,7 @@ Kafka SETTINGS 与 `GraphiteMergeTree` 类似,Kafka 引擎支持使用ClickHouse配置文件进行扩展配置。可以使用两个配置键:全局 (`kafka`) 和 主题级别 (`kafka_*`)。首先应用全局配置,然后应用主题级配置(如果存在)。 ```xml - + cgrp smallest diff --git a/docs/zh/operations/table_engines/merge.md b/docs/zh/operations/table_engines/merge.md index e8de53bc286..366a5459bf8 100644 --- a/docs/zh/operations/table_engines/merge.md +++ b/docs/zh/operations/table_engines/merge.md @@ -1,8 +1,8 @@ # Merge -The `Merge` engine (not to be confused with `MergeTree`) does not store data itself, but allows reading from any number of other tables simultaneously. +The `Merge` engine (not to be confused with `MergeTree`) does not store data itself, but allows reading from any number of other tables simultaneously. Reading is automatically parallelized. Writing to a table is not supported. When reading, the indexes of tables that are actually being read are used, if they exist. -The `Merge` engine accepts parameters: the database name and a regular expression for tables. +The `Merge` engine accepts parameters: the database name and a regular expression for tables. Example: @@ -10,17 +10,17 @@ Example: Merge(hits, '^WatchLog') ``` -Data will be read from the tables in the `hits` database that have names that match the regular expression '`^WatchLog`'. +Data will be read from the tables in the `hits` database that have names that match the regular expression '`^WatchLog`'. Instead of the database name, you can use a constant expression that returns a string. For example, `currentDatabase()`. Regular expressions — [re2](https://github.com/google/re2) (supports a subset of PCRE), case-sensitive. See the notes about escaping symbols in regular expressions in the "match" section. -When selecting tables to read, the `Merge` table itself will not be selected, even if it matches the regex. This is to avoid loops. -It is possible to create two `Merge` tables that will endlessly try to read each others' data, but this is not a good idea. +When selecting tables to read, the `Merge` table itself will not be selected, even if it matches the regex. This is to avoid loops. +It is possible to create two `Merge` tables that will endlessly try to read each others' data, but this is not a good idea. -The typical way to use the `Merge` engine is for working with a large number of `TinyLog` tables as if with a single table. +The typical way to use the `Merge` engine is for working with a large number of `TinyLog` tables as if with a single table. Example 2: @@ -63,7 +63,7 @@ Virtual columns differ from normal columns in the following ways: The `Merge` type table contains a virtual `_table` column of the `String` type. (If the table already has a `_table` column, the virtual column is called `_table1`; if you already have `_table1`, it's called `_table2`, and so on.) It contains the name of the table that data was read from. -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. +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/zh/operations/table_engines/mergetree.md b/docs/zh/operations/table_engines/mergetree.md index 3481e9c54ef..0782e2b242d 100644 --- a/docs/zh/operations/table_engines/mergetree.md +++ b/docs/zh/operations/table_engines/mergetree.md @@ -61,7 +61,7 @@ For a description of request parameters, see [request description](../../query_l By default the primary key is the same as the sorting key (which is specified by the `ORDER BY` clause). Thus in most cases it is unnecessary to specify a separate `PRIMARY KEY` clause. -- `SAMPLE BY` — An expression for sampling. +- `SAMPLE BY` — An expression for sampling. If a sampling expression is used, the primary key must contain it. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. diff --git a/docs/zh/operations/table_engines/replacingmergetree.md b/docs/zh/operations/table_engines/replacingmergetree.md index 2d0e179ac30..568f94822dd 100644 --- a/docs/zh/operations/table_engines/replacingmergetree.md +++ b/docs/zh/operations/table_engines/replacingmergetree.md @@ -4,7 +4,7 @@ The engine differs from [MergeTree](mergetree.md) in that it removes duplicate e Data deduplication occurs only during a merge. Merging occurs in the background at an unknown time, so you can't plan for it. Some of the data may remain unprocessed. Although you can run an unscheduled merge using the `OPTIMIZE` query, don't count on using it, because the `OPTIMIZE` query will read and write a large amount of data. -Thus, `ReplacingMergeTree` is suitable for clearing out duplicate data in the background in order to save space, but it doesn't guarantee the absence of duplicates. +Thus, `ReplacingMergeTree` is suitable for clearing out duplicate data in the background in order to save space, but it doesn't guarantee the absence of duplicates. ## Creating a Table @@ -33,7 +33,7 @@ For a description of request parameters, see [request description](../../query_l **Query clauses** -When creating a `ReplacingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table. +When creating a `ReplacingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table.
Deprecated Method for Creating a Table diff --git a/docs/zh/operations/table_engines/summingmergetree.md b/docs/zh/operations/table_engines/summingmergetree.md index 2013c7142b9..d1e35124fe0 100644 --- a/docs/zh/operations/table_engines/summingmergetree.md +++ b/docs/zh/operations/table_engines/summingmergetree.md @@ -1,9 +1,9 @@ # SummingMergeTree -The engine inherits from [MergeTree](mergetree.md). The difference is that when merging data parts for `SummingMergeTree` tables ClickHouse replaces all the rows with the same primary key with one row which contains summarized values for the columns with the numeric data type. If the primary key is composed in a way that a single key value corresponds to large number of rows, this significantly reduces storage volume and speeds up data selection. +The engine inherits from [MergeTree](mergetree.md). The difference is that when merging data parts for `SummingMergeTree` tables ClickHouse replaces all the rows with the same primary key with one row which contains summarized values for the columns with the numeric data type. If the primary key is composed in a way that a single key value corresponds to large number of rows, this significantly reduces storage volume and speeds up data selection. -We recommend to use the engine together with `MergeTree`. Store complete data in `MergeTree` table, and use `SummingMergeTree` for aggregated data storing, for example, when preparing reports. Such an approach will prevent you from losing valuable data due to an incorrectly composed primary key. +We recommend to use the engine together with `MergeTree`. Store complete data in `MergeTree` table, and use `SummingMergeTree` for aggregated data storing, for example, when preparing reports. Such an approach will prevent you from losing valuable data due to an incorrectly composed primary key. ## Creating a Table @@ -31,7 +31,7 @@ The columns must be of a numeric type and must not be in the primary key. **Query clauses** -When creating a `SummingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table. +When creating a `SummingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table.
Deprecated Method for Creating a Table @@ -72,7 +72,7 @@ Insert data to it: :) INSERT INTO summtt Values(1,1),(1,2),(2,1) ``` -ClickHouse may sum all the rows not completely ([see below](#data-processing)), so we use an aggregate function `sum` and `GROUP BY` clause in the query. +ClickHouse may sum all the rows not completely ([see below](#data-processing)), so we use an aggregate function `sum` and `GROUP BY` clause in the query. ```sql SELECT key, sum(value) FROM summtt GROUP BY key diff --git a/docs/zh/operations/tips.md b/docs/zh/operations/tips.md index a34f79f869f..11b7a782deb 100644 --- a/docs/zh/operations/tips.md +++ b/docs/zh/operations/tips.md @@ -18,7 +18,7 @@ You can use `turbostat` to view the CPU's actual clock rate under a load. ## CPU Scaling Governor -Always use the `performance` scaling governor. The `on-demand` scaling governor works much worse with constantly high demand. +Always use the `performance` scaling governor. The `on-demand` scaling governor works much worse with constantly high demand. ```bash sudo echo 'performance' | tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor diff --git a/docs/zh/query_language/create.md b/docs/zh/query_language/create.md index 332b53f812a..78364e45afa 100644 --- a/docs/zh/query_language/create.md +++ b/docs/zh/query_language/create.md @@ -43,7 +43,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ... 以上所有情况,如果指定了`IF NOT EXISTS`,那么在该表已经存在的情况下,查询不会返回任何错误。在这种情况下,查询几乎不会做任何事情。 -在`ENGINE`子句后还可能存在一些其他的子句,更详细的信息可以参考[表引擎](../operations/table_engines/index.md)中关于建表的描述。 +在`ENGINE`子句后还可能存在一些其他的子句,更详细的信息可以参考 [表引擎](../operations/table_engines/index.md) 中关于建表的描述。 ### 默认值 diff --git a/docs/zh/query_language/insert_into.md b/docs/zh/query_language/insert_into.md index 58af549fc35..33caac11d42 100644 --- a/docs/zh/query_language/insert_into.md +++ b/docs/zh/query_language/insert_into.md @@ -14,9 +14,9 @@ INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), . - 如果存在`DEFAULT`表达式,根据`DEFAULT`表达式计算被填充的值。 - 如果没有定义`DEFAULT`表达式,则填充零或空字符串。 -如果[strict_insert_defaults=1](../operations/settings/settings.md),你必须在查询中列出所有没有定义`DEFAULT`表达式的列。 +如果 [strict_insert_defaults=1](../operations/settings/settings.md),你必须在查询中列出所有没有定义`DEFAULT`表达式的列。 -数据可以以ClickHouse支持的任何[输入输出格式](../interfaces/formats.md#formats)传递给INSERT。格式的名称必须显示的指定在查询中: +数据可以以ClickHouse支持的任何 [输入输出格式](../interfaces/formats.md#formats) 传递给INSERT。格式的名称必须显示的指定在查询中: ``` sql INSERT INTO [db.]table [(c1, c2, c3)] FORMAT format_name data_set diff --git a/docs/zh/query_language/select.md b/docs/zh/query_language/select.md index 607f26ded71..8786be6e208 100644 --- a/docs/zh/query_language/select.md +++ b/docs/zh/query_language/select.md @@ -339,7 +339,7 @@ ARRAY JOIN nest AS n, arrayEnumerate(`nest.x`) AS num JOIN子句用于连接数据,作用与[SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL))的定义相同。 !!! info "注意" - 与[ARRAY JOIN](#select-array-join-clause)没有关系. + 与 [ARRAY JOIN](#select-array-join-clause) 没有关系. ``` sql @@ -365,7 +365,7 @@ FROM 在使用`ALL`修饰符对JOIN进行修饰时,如果右表中存在多个与左表关联的数据,那么系统则将右表中所有可以与左表关联的数据全部返回在结果中。这与SQL标准的JOIN行为相同。 在使用`ANY`修饰符对JOIN进行修饰时,如果右表中存在多个与左表关联的数据,那么系统仅返回第一个与左表匹配的结果。如果左表与右表一一对应,不存在多余的行时,`ANY`与`ALL`的结果相同。 -你可以在会话中通过设置[join_default_strictness](../operations/settings/settings.md)来指定默认的JOIN修饰符。 +你可以在会话中通过设置 [join_default_strictness](../operations/settings/settings.md) 来指定默认的JOIN修饰符。 **`GLOBAL` distribution** @@ -373,7 +373,7 @@ FROM 当使用`GLOBAL ... JOIN`,首先会在请求服务器上计算右表并以临时表的方式将其发送到所有服务器。这时每台服务器将直接使用它进行计算。 -使用`GLOBAL`时需要小心。更多信息,参阅[Distributed subqueries](#select-distributed-subqueries)部分。 +使用`GLOBAL`时需要小心。更多信息,参阅 [Distributed subqueries](#select-distributed-subqueries) 部分。 **使用建议** @@ -435,13 +435,13 @@ LIMIT 10 在一些场景下,使用`IN`代替`JOIN`将会得到更高的效率。在各种类型的JOIN中,最高效的是`ANY LEFT JOIN`,然后是`ANY INNER JOIN`,效率最差的是`ALL LEFT JOIN`以及`ALL INNER JOIN`。 -如果你需要使用`JOIN`来关联一些纬度表(包含纬度属性的一些相对比较小的表,例如广告活动的名称),那么`JOIN`可能不是好的选择,因为语法负责,并且每次查询都将重新访问这些表。对于这种情况,您应该使用“外部字典”的功能来替换`JOIN`。更多信息,参见[外部字典](dicts/external_dicts.md)部分。 +如果你需要使用`JOIN`来关联一些纬度表(包含纬度属性的一些相对比较小的表,例如广告活动的名称),那么`JOIN`可能不是好的选择,因为语法负责,并且每次查询都将重新访问这些表。对于这种情况,您应该使用“外部字典”的功能来替换`JOIN`。更多信息,参见 [外部字典](dicts/external_dicts.md) 部分。 #### Null的处理 -JOIN的行为受[join_use_nulls](../operations/settings/settings.md)的影响。当`join_use_nulls=1`时,`JOIN`的工作与SQL标准相同。 +JOIN的行为受 [join_use_nulls](../operations/settings/settings.md) 的影响。当`join_use_nulls=1`时,`JOIN`的工作与SQL标准相同。 -如果JOIN的key是[Nullable](../data_types/nullable.md)类型的字段,则其中至少一个存在[NULL](syntax.md)值的key不会被关联。 +如果JOIN的key是 [Nullable](../data_types/nullable.md) 类型的字段,则其中至少一个存在 [NULL](syntax.md) 值的key不会被关联。 ### WHERE 子句 @@ -511,7 +511,7 @@ GROUP BY子句会为遇到的每一个不同的key计算一组聚合函数的值 #### NULL 处理 -对于GROUP BY子句,ClickHouse将[NULL](syntax.md)解释为一个值,并且支持`NULL=NULL`。 +对于GROUP BY子句,ClickHouse将 [NULL](syntax.md) 解释为一个值,并且支持`NULL=NULL`。 下面这个例子将说明这将意味着什么。 @@ -693,7 +693,7 @@ WHERE于HAVING不同之处在于WHERE在聚合前(GROUP BY)执行,HAVING在聚 在SELECT表达式中存在Array类型的列时,不能使用DISTINCT。 -`DISTINCT`可以与[NULL](syntax.md)一起工作,就好像`NULL`仅是一个特殊的值一样,并且`NULL=NULL`。换而言之,在`DISTINCT`的结果中,与`NULL`不同的组合仅能出现一次。 +`DISTINCT`可以与 [NULL](syntax.md)一起工作,就好像`NULL`仅是一个特殊的值一样,并且`NULL=NULL`。换而言之,在`DISTINCT`的结果中,与`NULL`不同的组合仅能出现一次。 ### LIMIT 子句 @@ -814,7 +814,7 @@ IN子句中的子查询仅在单个服务器上运行一次。不能够是相关 #### NULL 处理 -在处理中,IN操作符总是假定[NULL](syntax.md)值的操作结果总是等于`0`,而不管`NULL`位于左侧还是右侧。`NULL`值不应该包含在任何数据集中,它们彼此不能够对应,并且不能够比较。 +在处理中,IN操作符总是假定 [NULL](syntax.md) 值的操作结果总是等于`0`,而不管`NULL`位于左侧还是右侧。`NULL`值不应该包含在任何数据集中,它们彼此不能够对应,并且不能够比较。 下面的示例中有一个`t_null`表: @@ -851,11 +851,11 @@ FROM t_null 对于带有子查询的(类似与JOINs)IN中,有两种选择:普通的`IN`/`JOIN`与`GLOBAL IN` / `GLOBAL JOIN`。它们对于分布式查询的处理运行方式是不同的。 !!! 注意 - 请记住,下面描述的算法可能因为根据[settings](../operations/settings/settings.md)配置的不同而不同。 + 请记住,下面描述的算法可能因为根据 [settings](../operations/settings/settings.md) 配置的不同而不同。 当使用普通的IN时,查询总是被发送到远程的服务器,并且在每个服务器中运行“IN”或“JOIN”子句中的子查询。 -当使用`GLOBAL IN` / `GLOBAL JOIN`时,首先会为`GLOBAL IN` / `GLOBAL JOIN`运行所有子查询,并将结果收集到临时表中,并将临时表发送到每个远程服务器,并使用该临时表运行查询。 +当使用`GLOBAL IN` / `GLOBAL JOIN`时,首先会为`GLOBAL IN` / `GLOBAL JOIN`运行所有子查询,并将结果收集到临时表中,并将临时表发送到每个远程服务器,并使用该临时表运行查询。 对于非分布式查询,请使用普通的`IN` / `JOIN`。 diff --git a/libs/libmysqlxx/cmake/find_mysqlclient.cmake b/libs/libmysqlxx/cmake/find_mysqlclient.cmake index 85a6275f1ff..71cb2bfeb1d 100644 --- a/libs/libmysqlxx/cmake/find_mysqlclient.cmake +++ b/libs/libmysqlxx/cmake/find_mysqlclient.cmake @@ -5,7 +5,7 @@ else () option (USE_INTERNAL_MYSQL_LIBRARY "Set to FALSE to use system mysqlclient library instead of bundled" OFF) endif () -if (USE_INTERNAL_MYSQL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/mariadb-connector-c/README") +if (USE_INTERNAL_MYSQL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/mariadb-connector-c/README.md") message (WARNING "submodule contrib/mariadb-connector-c is missing. to fix try run: \n git submodule update --init --recursive") set (USE_INTERNAL_MYSQL_LIBRARY 0) endif ()