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

This commit is contained in:
BayoNet 2018-12-27 14:50:55 +03:00
commit 5a25d65a7c
156 changed files with 636 additions and 459 deletions

2
.gitmodules vendored
View File

@ -36,7 +36,7 @@
url = https://github.com/ClickHouse-Extras/llvm url = https://github.com/ClickHouse-Extras/llvm
[submodule "contrib/mariadb-connector-c"] [submodule "contrib/mariadb-connector-c"]
path = 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"] [submodule "contrib/jemalloc"]
path = contrib/jemalloc path = contrib/jemalloc
url = https://github.com/jemalloc/jemalloc.git url = https://github.com/jemalloc/jemalloc.git

@ -1 +1 @@
Subproject commit a0fd36cc5a5313414a5a2ebe9322577a29b4782a Subproject commit d85d0e98999cd9e28ceb66645999b4a9ce85370e

View File

@ -85,7 +85,7 @@ public:
const ColumnArray & first_array_column = static_cast<const ColumnArray &>(*columns[0]); const ColumnArray & first_array_column = static_cast<const ColumnArray &>(*columns[0]);
const IColumn::Offsets & offsets = first_array_column.getOffsets(); 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]; 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. /// Sanity check. NOTE We can implement specialization for a case with single argument, if the check will hurt performance.

View File

@ -146,7 +146,7 @@ public:
const ColumnArray & first_array_column = static_cast<const ColumnArray &>(*columns[0]); const ColumnArray & first_array_column = static_cast<const ColumnArray &>(*columns[0]);
const IColumn::Offsets & offsets = first_array_column.getOffsets(); 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]; 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. /// Sanity check. NOTE We can implement specialization for a case with single argument, if the check will hurt performance.

View File

@ -119,7 +119,7 @@ public:
ColumnArray & arr_to = static_cast<ColumnArray &>(to); ColumnArray & arr_to = static_cast<ColumnArray &>(to);
ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); 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<T>::Container & data_to = static_cast<ColumnVector<T> &>(arr_to.getData()).getData(); typename ColumnVector<T>::Container & data_to = static_cast<ColumnVector<T> &>(arr_to.getData()).getData();
data_to.insert(this->data(place).value.begin(), this->data(place).value.end()); data_to.insert(this->data(place).value.begin(), this->data(place).value.end());
@ -370,7 +370,7 @@ public:
auto & column_array = static_cast<ColumnArray &>(to); auto & column_array = static_cast<ColumnArray &>(to);
auto & offsets = column_array.getOffsets(); 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(); auto & column_data = column_array.getData();

View File

@ -83,7 +83,7 @@ public:
const typename State::Set & set = this->data(place).value; const typename State::Set & set = this->data(place).value;
size_t size = set.size(); 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<T>::Container & data_to = static_cast<ColumnVector<T> &>(arr_to.getData()).getData(); typename ColumnVector<T>::Container & data_to = static_cast<ColumnVector<T> &>(arr_to.getData()).getData();
size_t old_size = data_to.size(); size_t old_size = data_to.size();
@ -207,7 +207,7 @@ public:
IColumn & data_to = arr_to.getData(); IColumn & data_to = arr_to.getData();
auto & set = this->data(place).value; 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) for (auto & elem : set)
{ {

View File

@ -138,7 +138,7 @@ public:
ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); ColumnArray::Offsets & offsets_to = arr_to.getOffsets();
size_t size = levels.size(); 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) if (!size)
return; return;

View File

@ -83,7 +83,7 @@ public:
const ColumnArray & array_column = static_cast<const ColumnArray &>(*columns[0]); const ColumnArray & array_column = static_cast<const ColumnArray &>(*columns[0]);
const IColumn::Offsets & offsets = array_column.getOffsets(); const IColumn::Offsets & offsets = array_column.getOffsets();
const auto & keys_vec = static_cast<const ColVecType &>(array_column.getData()); const auto & keys_vec = static_cast<const ColVecType &>(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); const size_t keys_vec_size = (offsets[row_num] - keys_vec_offset);
// Columns 1..n contain arrays of numeric values to sum // Columns 1..n contain arrays of numeric values to sum
@ -93,7 +93,7 @@ public:
Field value; Field value;
const ColumnArray & array_column = static_cast<const ColumnArray &>(*columns[col + 1]); const ColumnArray & array_column = static_cast<const ColumnArray &>(*columns[col + 1]);
const IColumn::Offsets & offsets = array_column.getOffsets(); 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); const size_t values_vec_size = (offsets[row_num] - values_vec_offset);
// Expect key and value arrays to be of same length // Expect key and value arrays to be of same length

View File

@ -93,7 +93,7 @@ public:
auto result_vec = set.topK(threshold); auto result_vec = set.topK(threshold);
size_t size = result_vec.size(); 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<T>::Container & data_to = static_cast<ColumnVector<T> &>(arr_to.getData()).getData(); typename ColumnVector<T>::Container & data_to = static_cast<ColumnVector<T> &>(arr_to.getData()).getData();
size_t old_size = data_to.size(); size_t old_size = data_to.size();
@ -212,7 +212,7 @@ public:
IColumn & data_to = arr_to.getData(); IColumn & data_to = arr_to.getData();
auto result_vec = this->data(place).value.topK(threshold); 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) for (auto & elem : result_vec)
{ {

View File

@ -166,7 +166,7 @@ void ColumnArray::insertData(const char * pos, size_t length)
if (pos != end) if (pos != end)
throw Exception("Incorrect length argument for method ColumnArray::insertData", ErrorCodes::BAD_ARGUMENTS); 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) for (size_t i = 0; i < array_size; ++i)
pos = getData().deserializeAndInsertFromArena(pos); pos = getData().deserializeAndInsertFromArena(pos);
getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + array_size); getOffsets().push_back(getOffsets().back() + array_size);
return pos; return pos;
} }
@ -216,7 +216,7 @@ void ColumnArray::insert(const Field & x)
size_t size = array.size(); size_t size = array.size();
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
getData().insert(array[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); size_t offset = src.offsetAt(n);
getData().insertRangeFrom(src.getData(), offset, size); 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() void ColumnArray::insertDefault()
{ {
getOffsets().push_back(getOffsets().size() == 0 ? 0 : getOffsets().back()); getOffsets().push_back(getOffsets().back());
} }

View File

@ -124,8 +124,8 @@ private:
ColumnPtr data; ColumnPtr data;
ColumnPtr offsets; ColumnPtr offsets;
size_t ALWAYS_INLINE offsetAt(size_t i) const { return i == 0 ? 0 : 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 i == 0 ? getOffsets()[0] : (getOffsets()[i] - 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<T>. /// Multiply values if the nested column is ColumnVector<T>.

View File

@ -148,7 +148,7 @@ ColumnPtr ColumnString::permute(const Permutation & perm, size_t limit) const
for (size_t i = 0; i < limit; ++i) for (size_t i = 0; i < limit; ++i)
{ {
size_t j = perm[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; size_t string_size = offsets[j] - string_offset;
memcpySmallAllowReadWriteOverflow15(&res_chars[current_new_offset], &chars[string_offset], string_size); memcpySmallAllowReadWriteOverflow15(&res_chars[current_new_offset], &chars[string_offset], string_size);
@ -219,7 +219,7 @@ ColumnPtr ColumnString::indexImpl(const PaddedPODArray<Type> & indexes, size_t l
for (size_t i = 0; i < limit; ++i) for (size_t i = 0; i < limit; ++i)
{ {
size_t j = indexes[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; size_t string_size = offsets[j] - string_offset;
memcpySmallAllowReadWriteOverflow15(&res_chars[current_new_offset], &chars[string_offset], string_size); memcpySmallAllowReadWriteOverflow15(&res_chars[current_new_offset], &chars[string_offset], string_size);

View File

@ -31,10 +31,10 @@ private:
/// For convenience, every string ends with terminating zero byte. Note that strings could contain zero bytes in the middle. /// For convenience, every string ends with terminating zero byte. Note that strings could contain zero bytes in the middle.
Chars chars; 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 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 <bool positive> template <bool positive>
struct less; struct less;
@ -203,7 +203,7 @@ public:
void insertDefault() override void insertDefault() override
{ {
chars.push_back(0); 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 int compareAt(size_t n, size_t m, const IColumn & rhs_, int /*nan_direction_hint*/) const override

View File

@ -107,7 +107,7 @@ public:
if (s != offsets.size()) if (s != offsets.size())
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); 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 MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override

View File

@ -0,0 +1,8 @@
#include <Common/PODArray.h>
namespace DB
{
/// Used for left padding of PODArray when empty
const char EmptyPODArray[EmptyPODArraySize]{};
}

View File

@ -20,6 +20,11 @@
namespace DB 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. /** A dynamic array for POD types.
* Designed for a small number of large arrays (rather than a lot of small ones). * Designed for a small number of large arrays (rather than a lot of small ones).
* To be more precise - for use in ColumnVector. * 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. * 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. * 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. * Some methods using allocator have TAllocatorParams variadic arguments.
* These arguments will be passed to corresponding methods of TAllocator. * These arguments will be passed to corresponding methods of TAllocator.
* Example: pointer to Arena, that is used for allocations. * Example: pointer to Arena, that is used for allocations.
@ -49,16 +58,25 @@ namespace DB
* TODO Pass alignment to Allocator. * TODO Pass alignment to Allocator.
* TODO Allow greater alignment than alignof(T). Example: array of char aligned to page size. * TODO Allow greater alignment than alignof(T). Example: array of char aligned to page size.
*/ */
template <typename T, size_t INITIAL_SIZE = 4096, typename TAllocator = Allocator<false>, size_t pad_right_ = 0> static constexpr size_t EmptyPODArraySize = 1024;
extern const char EmptyPODArray[EmptyPODArraySize];
template <typename T, size_t INITIAL_SIZE = 4096, typename TAllocator = Allocator<false>, size_t pad_right_ = 0, size_t pad_left_ = 0>
class PODArray : private boost::noncopyable, private TAllocator /// empty base optimization class PODArray : private boost::noncopyable, private TAllocator /// empty base optimization
{ {
protected: protected:
/// Round padding up to an whole number of elements to simplify arithmetic. /// 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<char *>(EmptyPODArray) + EmptyPODArraySize : nullptr;
char * c_start = nullptr; static_assert(pad_left <= EmptyPODArraySize && "Left Padding exceeds EmptyPODArraySize. Is the element size too large?");
char * c_end = nullptr;
char * c_end_of_storage = nullptr; /// Does not include pad_right. 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<T *>(c_start); } T * t_start() { return reinterpret_cast<T *>(c_start); }
T * t_end() { return reinterpret_cast<T *>(c_end); } T * t_end() { return reinterpret_cast<T *>(c_end); }
@ -72,7 +90,7 @@ protected:
static size_t byte_size(size_t num_elements) { return num_elements * sizeof(T); } 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. /// 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) void alloc_for_num_elements(size_t num_elements)
{ {
@ -82,22 +100,24 @@ protected:
template <typename ... TAllocatorParams> template <typename ... TAllocatorParams>
void alloc(size_t bytes, TAllocatorParams &&... allocator_params) void alloc(size_t bytes, TAllocatorParams &&... allocator_params)
{ {
c_start = c_end = reinterpret_cast<char *>(TAllocator::alloc(bytes, std::forward<TAllocatorParams>(allocator_params)...)); c_start = c_end = reinterpret_cast<char *>(TAllocator::alloc(bytes, std::forward<TAllocatorParams>(allocator_params)...)) + pad_left;
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] = {};
} }
void dealloc() void dealloc()
{ {
if (c_start == nullptr) if (c_start == null)
return; return;
TAllocator::free(c_start, allocated_bytes()); TAllocator::free(c_start - pad_left, allocated_bytes());
} }
template <typename ... TAllocatorParams> template <typename ... TAllocatorParams>
void realloc(size_t bytes, TAllocatorParams &&... allocator_params) void realloc(size_t bytes, TAllocatorParams &&... allocator_params)
{ {
if (c_start == nullptr) if (c_start == null)
{ {
alloc(bytes, std::forward<TAllocatorParams>(allocator_params)...); alloc(bytes, std::forward<TAllocatorParams>(allocator_params)...);
return; return;
@ -105,15 +125,18 @@ protected:
ptrdiff_t end_diff = c_end - c_start; ptrdiff_t end_diff = c_end - c_start;
c_start = reinterpret_cast<char *>(TAllocator::realloc(c_start, allocated_bytes(), bytes, std::forward<TAllocatorParams>(allocator_params)...)); c_start = reinterpret_cast<char *>(
TAllocator::realloc(c_start - pad_left, allocated_bytes(), bytes, std::forward<TAllocatorParams>(allocator_params)...))
+ pad_left;
c_end = c_start + end_diff; 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 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 bool isAllocatedFromStack() const
@ -139,7 +162,7 @@ protected:
public: public:
using value_type = T; 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. /// You can not just use `typedef`, because there is ambiguity for the constructors and `assign` functions.
struct iterator : public boost::iterator_adaptor<iterator, T*> struct iterator : public boost::iterator_adaptor<iterator, T*>
@ -378,9 +401,9 @@ public:
memcpy(dest.c_start, src.c_start, byte_size(src.size())); memcpy(dest.c_start, src.c_start, byte_size(src.size()));
dest.c_end = dest.c_start + (src.c_end - src.c_start); dest.c_end = dest.c_start + (src.c_end - src.c_start);
src.c_start = nullptr; src.c_start = null;
src.c_end = nullptr; src.c_end = null;
src.c_end_of_storage = nullptr; src.c_end_of_storage = null;
} }
else else
{ {
@ -504,15 +527,9 @@ void swap(PODArray<T, INITIAL_SIZE, TAllocator, pad_right_> & lhs, PODArray<T, I
/** For columns. Padding is enough to read and write xmm-register at the address of the last element. */ /** For columns. Padding is enough to read and write xmm-register at the address of the last element. */
template <typename T, size_t INITIAL_SIZE = 4096, typename TAllocator = Allocator<false>> template <typename T, size_t INITIAL_SIZE = 4096, typename TAllocator = Allocator<false>>
using PaddedPODArray = PODArray<T, INITIAL_SIZE, TAllocator, 15>; using PaddedPODArray = PODArray<T, INITIAL_SIZE, TAllocator, 15, 16>;
inline constexpr size_t integerRound(size_t value, size_t dividend)
{
return ((value + dividend - 1) / dividend) * dividend;
}
template <typename T, size_t stack_size_in_bytes> template <typename T, size_t stack_size_in_bytes>
using PODArrayWithStackMemory = PODArray<T, 0, AllocatorWithStackMemory<Allocator<false>, integerRound(stack_size_in_bytes, sizeof(T))>>; using PODArrayWithStackMemory = PODArray<T, 0, AllocatorWithStackMemory<Allocator<false>, integerRoundUp(stack_size_in_bytes, sizeof(T))>>;
} }

View File

@ -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) ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, Coordination::WatchCallback caller_watch_callback)
{ {
zkutil::ZooKeeperPtr zookeeper;
std::unordered_set<std::string> invalidated_paths; std::unordered_set<std::string> invalidated_paths;
{ {
std::lock_guard<std::mutex> lock(context->mutex); std::lock_guard<std::mutex> lock(context->mutex);
if (!context->zookeeper) if (context->all_paths_invalidated)
{ {
/// Possibly, there was a previous session and it has expired. Clear the cache. /// Possibly, there was a previous session and it has expired. Clear the cache.
path_to_cached_znode.clear(); path_to_cached_znode.clear();
context->all_paths_invalidated = false;
context->zookeeper = get_zookeeper();
} }
zookeeper = context->zookeeper;
invalidated_paths.swap(context->invalidated_paths); invalidated_paths.swap(context->invalidated_paths);
} }
zkutil::ZooKeeperPtr zookeeper = get_zookeeper();
if (!zookeeper) if (!zookeeper)
throw DB::Exception("Could not get znode: `" + path + "'. ZooKeeper not configured.", DB::ErrorCodes::NO_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; changed = owned_context->invalidated_paths.emplace(response.path).second;
else if (response.state == Coordination::EXPIRED_SESSION) else if (response.state == Coordination::EXPIRED_SESSION)
{ {
owned_context->zookeeper = nullptr;
owned_context->invalidated_paths.clear(); owned_context->invalidated_paths.clear();
owned_context->all_paths_invalidated = true;
changed = true; changed = true;
} }
} }

View File

@ -53,8 +53,8 @@ private:
struct Context struct Context
{ {
std::mutex mutex; std::mutex mutex;
zkutil::ZooKeeperPtr zookeeper;
std::unordered_set<std::string> invalidated_paths; std::unordered_set<std::string> invalidated_paths;
bool all_paths_invalidated = false;
}; };
std::shared_ptr<Context> context; std::shared_ptr<Context> context;

View File

@ -59,7 +59,7 @@ void DataTypeArray::serializeBinary(const IColumn & column, size_t row_num, Writ
const ColumnArray & column_array = static_cast<const ColumnArray &>(column); const ColumnArray & column_array = static_cast<const ColumnArray &>(column);
const ColumnArray::Offsets & offsets = column_array.getOffsets(); 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 next_offset = offsets[row_num];
size_t size = next_offset - offset; size_t size = next_offset - offset;
@ -113,7 +113,7 @@ namespace
? offset + limit ? offset + limit
: size; : 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) for (size_t i = offset; i < end; ++i)
{ {
ColumnArray::Offset current_offset = offset_values[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<const ColumnArray &>(column); const ColumnArray & column_array = static_cast<const ColumnArray &>(column);
const ColumnArray::Offsets & offsets = column_array.getOffsets(); 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 next_offset = offsets[row_num];
const IColumn & nested_column = column_array.getData(); 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<const ColumnArray &>(column); const ColumnArray & column_array = static_cast<const ColumnArray &>(column);
const ColumnArray::Offsets & offsets = column_array.getOffsets(); 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 next_offset = offsets[row_num];
const IColumn & nested_column = column_array.getData(); 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<const ColumnArray &>(column); const ColumnArray & column_array = static_cast<const ColumnArray &>(column);
const ColumnArray::Offsets & offsets = column_array.getOffsets(); 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 next_offset = offsets[row_num];
const IColumn & nested_column = column_array.getData(); const IColumn & nested_column = column_array.getData();

View File

@ -1142,32 +1142,24 @@ struct ToIntMonotonicity
if (!type.isValueRepresentedByNumber()) if (!type.isValueRepresentedByNumber())
return {}; return {};
size_t size_of_type = type.getSizeOfValueInMemory(); /// If type is same, the conversion is always monotonic.
/// (Enum has separate case, because it is different data type)
/// 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<T> || type.isValueRepresentedByUnsignedInteger())
return {true, true, true};
/// If arguments from the same half, then function is monotonic.
if ((left.get<Int64>() >= 0) == (right.get<Int64>() >= 0))
return {true, true, true};
}
/// If type is same, too. (Enum has separate case, because it is different data type)
if (checkAndGetDataType<DataTypeNumber<T>>(&type) || if (checkAndGetDataType<DataTypeNumber<T>>(&type) ||
checkAndGetDataType<DataTypeEnum<T>>(&type)) checkAndGetDataType<DataTypeEnum<T>>(&type))
return { true, true, true }; return { true, true, true };
/// In other cases, if range is unbounded, we don't know, whether function is monotonic or not. /// Float cases.
if (left.isNull() || right.isNull())
return {};
/// 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<T>)
return {true, true, true};
/// If converting from Float, for monotonicity, arguments must fit in range of result type.
if (WhichDataType(type).isFloat()) if (WhichDataType(type).isFloat())
{ {
if (left.isNull() || right.isNull())
return {};
Float64 left_float = left.get<Float64>(); Float64 left_float = left.get<Float64>();
Float64 right_float = right.get<Float64>(); Float64 right_float = right.get<Float64>();
@ -1178,18 +1170,79 @@ struct ToIntMonotonicity
return {}; return {};
} }
/// If signedness of type is changing, or converting from Date, DateTime, then arguments must be from same half, /// Integer cases.
/// and after conversion, resulting values must be from same half.
/// Just in case, it is required in rest of cases too.
if ((left.get<Int64>() >= 0) != (right.get<Int64>() >= 0)
|| (T(left.get<Int64>()) >= 0) != (T(right.get<Int64>()) >= 0))
return {};
/// If type is shrinked, then for monotonicity, all bits other than that fits, must be same. const bool from_is_unsigned = type.isValueRepresentedByUnsignedInteger();
if (divideByRangeOfType(left.get<UInt64>()) != divideByRangeOfType(right.get<UInt64>())) const bool to_is_unsigned = std::is_unsigned_v<T>;
return {};
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<Int64>() >= 0);
const bool right_in_first_half = right.isNull()
? !from_is_unsigned
: (right.get<Int64>() >= 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<UInt64>()) == divideByRangeOfType(right.get<UInt64>()))
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<Int64>()) >= 0) == (T(right.get<Int64>()) >= 0)
&& divideByRangeOfType(left.get<UInt64>()) == divideByRangeOfType(right.get<UInt64>()))
return {true};
return {};
}
}
__builtin_unreachable();
} }
}; };

View File

@ -139,7 +139,7 @@ struct IntegerRoundingComputation
static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out)
{ {
if (scale > size_t(std::numeric_limits<T>::max())) if (sizeof(T) <= sizeof(scale) && scale > size_t(std::numeric_limits<T>::max()))
*out = 0; *out = 0;
else else
*out = compute(*in, scale); *out = compute(*in, scale);
@ -331,10 +331,10 @@ public:
template <size_t scale> template <size_t scale>
static NO_INLINE void applyImpl(const PaddedPODArray<T> & in, typename ColumnVector<T>::Container & out) static NO_INLINE void applyImpl(const PaddedPODArray<T> & in, typename ColumnVector<T>::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(); const T * __restrict p_in = in.data();
T* __restrict p_out = out.data(); T * __restrict p_out = out.data();
while (p_in < end_in) while (p_in < end_in)
{ {

View File

@ -225,7 +225,7 @@ bool FunctionArrayReverse::executeString(const IColumn & src_data, const ColumnA
{ {
size_t j_reversed = array_size - j - 1; 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; 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); memcpySmallAllowReadWriteOverflow15(&res_chars[res_string_prev_offset], &src_data[src_pos], string_size);

View File

@ -271,7 +271,7 @@ struct AggregationMethodString
Arena & /*pool*/) const Arena & /*pool*/) const
{ {
return StringRef( return StringRef(
&(*chars)[i == 0 ? 0 : (*offsets)[i - 1]], &(*chars)[(*offsets)[i - 1]],
(i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1); (i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1);
} }
}; };

View File

@ -61,7 +61,7 @@ DNSCacheUpdater::DNSCacheUpdater(Context & context_)
task_handle = pool.addTask([this] () { return run(); }); task_handle = pool.addTask([this] () { return run(); });
} }
bool DNSCacheUpdater::run() BackgroundProcessingPoolTaskResult DNSCacheUpdater::run()
{ {
/// TODO: Ensusre that we get global counter (not thread local) /// 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); 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_num_network_erros = num_current_network_exceptions;
last_update_time = time(nullptr); last_update_time = time(nullptr);
return true; return BackgroundProcessingPoolTaskResult::SUCCESS;
} }
catch (...) catch (...)
{ {
/// Do not increment ProfileEvents::NetworkErrors twice /// Do not increment ProfileEvents::NetworkErrors twice
if (isNetworkError()) if (isNetworkError())
return false; return BackgroundProcessingPoolTaskResult::ERROR;
throw; throw;
} }
} }
/// According to BackgroundProcessingPool logic, if task has done work, it could be executed again immediately. /// According to BackgroundProcessingPool logic, if task has done work, it could be executed again immediately.
return false; return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO;
} }
DNSCacheUpdater::~DNSCacheUpdater() DNSCacheUpdater::~DNSCacheUpdater()

View File

@ -11,6 +11,7 @@ namespace DB
class Context; class Context;
class BackgroundProcessingPool; class BackgroundProcessingPool;
class BackgroundProcessingPoolTaskInfo; class BackgroundProcessingPoolTaskInfo;
enum class BackgroundProcessingPoolTaskResult;
/// Add a task to BackgroundProcessingPool that watch for ProfileEvents::NetworkErrors and updates DNS cache if it has increased /// 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(); static bool incrementNetworkErrorEventsIfNeeded();
private: private:
bool run(); BackgroundProcessingPoolTaskResult run();
Context & context; Context & context;
BackgroundProcessingPool & pool; BackgroundProcessingPool & pool;

View File

@ -337,56 +337,6 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block &
} }
} }
bool ExpressionAnalyzer::isThereArrayJoin(const ASTPtr & ast)
{
if (typeid_cast<ASTIdentifier *>(ast.get()))
{
return false;
}
else if (ASTFunction * node = typeid_cast<ASTFunction *>(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<ASTLiteral *>(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) 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();
}
} }

View File

@ -150,7 +150,7 @@ public:
/** Get a set of columns that are enough to read from the table to evaluate the expression. /** 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. * 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. /** 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; 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. /// If ast is ASTSelectQuery with JOIN, add actions for JOIN key columns.
void getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, ExpressionActionsPtr & actions); void getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, ExpressionActionsPtr & actions);

View File

@ -75,7 +75,7 @@ struct JoinKeyGetterString
const Sizes &) const const Sizes &) const
{ {
return StringRef( return StringRef(
&(*chars)[i == 0 ? 0 : (*offsets)[i - 1]], &(*chars)[(*offsets)[i - 1]],
(i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1); (i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1);
} }

View File

@ -86,7 +86,7 @@ struct SetMethodString
const Sizes &) const const Sizes &) const
{ {
return StringRef( return StringRef(
&(*chars)[i == 0 ? 0 : (*offsets)[i - 1]], &(*chars)[(*offsets)[i - 1]],
(i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1); (i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1);
} }
}; };

View File

@ -25,6 +25,7 @@ namespace DB
static constexpr double thread_sleep_seconds = 10; static constexpr double thread_sleep_seconds = 10;
static constexpr double thread_sleep_seconds_random_part = 1.0; 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. /// For exponential backoff.
static constexpr double task_sleep_seconds_when_no_work_min = 10; static constexpr double task_sleep_seconds_when_no_work_min = 10;
@ -146,7 +147,7 @@ void BackgroundProcessingPool::threadFunction()
while (!shutdown) while (!shutdown)
{ {
bool done_work = false; TaskResult task_result = TaskResult::ERROR;
TaskHandle task; TaskHandle task;
try try
@ -198,7 +199,7 @@ void BackgroundProcessingPool::threadFunction()
{ {
CurrentMetrics::Increment metric_increment{CurrentMetrics::BackgroundPoolTask}; CurrentMetrics::Increment metric_increment{CurrentMetrics::BackgroundPoolTask};
done_work = task->function(); task_result = task->function();
} }
} }
catch (...) catch (...)
@ -216,7 +217,7 @@ void BackgroundProcessingPool::threadFunction()
if (task->removed) if (task->removed)
continue; continue;
if (done_work) if (task_result == TaskResult::SUCCESS)
task->count_no_work_done = 0; task->count_no_work_done = 0;
else else
++task->count_no_work_done; ++task->count_no_work_done;
@ -225,11 +226,13 @@ void BackgroundProcessingPool::threadFunction()
/// If not, add delay before next run. /// If not, add delay before next run.
Poco::Timestamp next_time_to_execute; /// current time Poco::Timestamp next_time_to_execute; /// current time
if (!done_work) if (task_result == TaskResult::ERROR)
next_time_to_execute += 1000000 * (std::min( next_time_to_execute += 1000000 * (std::min(
task_sleep_seconds_when_no_work_max, 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)) 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<double>(0, task_sleep_seconds_when_no_work_random_part)(rng)); + std::uniform_real_distribution<double>(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); tasks.erase(task->iterator);
task->iterator = tasks.emplace(next_time_to_execute, task); task->iterator = tasks.emplace(next_time_to_execute, task);

View File

@ -21,6 +21,12 @@ namespace DB
class BackgroundProcessingPool; class BackgroundProcessingPool;
class BackgroundProcessingPoolTaskInfo; 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. /** 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. * In this case, one task can run simultaneously from different threads.
* Designed for tasks that perform continuous background work (for example, merge). * Designed for tasks that perform continuous background work (for example, merge).
@ -31,11 +37,13 @@ class BackgroundProcessingPool
{ {
public: public:
/// Returns true, if some useful work was done. In that case, thread will not sleep before next run of this task. /// 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<bool()>; using TaskResult = BackgroundProcessingPoolTaskResult;
using Task = std::function<TaskResult()>;
using TaskInfo = BackgroundProcessingPoolTaskInfo; using TaskInfo = BackgroundProcessingPoolTaskInfo;
using TaskHandle = std::shared_ptr<TaskInfo>; using TaskHandle = std::shared_ptr<TaskInfo>;
BackgroundProcessingPool(int size_); BackgroundProcessingPool(int size_);
size_t getNumberOfThreads() const size_t getNumberOfThreads() const

View File

@ -581,13 +581,13 @@ bool StorageMergeTree::tryMutatePart()
} }
bool StorageMergeTree::backgroundTask() BackgroundProcessingPoolTaskResult StorageMergeTree::backgroundTask()
{ {
if (shutdown_called) if (shutdown_called)
return false; return BackgroundProcessingPoolTaskResult::ERROR;
if (merger_mutator.actions_blocker.isCancelled()) if (merger_mutator.actions_blocker.isCancelled())
return false; return BackgroundProcessingPoolTaskResult::ERROR;
try try
{ {
@ -601,16 +601,19 @@ bool StorageMergeTree::backgroundTask()
///TODO: read deduplicate option from table config ///TODO: read deduplicate option from table config
if (merge(false /*aggressive*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/)) 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) catch (Exception & e)
{ {
if (e.code() == ErrorCodes::ABORTED) if (e.code() == ErrorCodes::ABORTED)
{ {
LOG_INFO(log, e.message()); LOG_INFO(log, e.message());
return false; return BackgroundProcessingPoolTaskResult::ERROR;
} }
throw; throw;

View File

@ -137,7 +137,7 @@ private:
/// Try and find a single part to mutate and mutate it. If some part was successfully mutated, return true. /// Try and find a single part to mutate and mutate it. If some part was successfully mutated, return true.
bool tryMutatePart(); bool tryMutatePart();
bool backgroundTask(); BackgroundProcessingPoolTaskResult backgroundTask();
Int64 getCurrentMutationVersion( Int64 getCurrentMutationVersion(
const MergeTreeData::DataPartPtr & part, const MergeTreeData::DataPartPtr & part,

View File

@ -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 replication queue is stopped exit immediately as we successfully executed the task
if (queue.actions_blocker.isCancelled()) if (queue.actions_blocker.isCancelled())
{ {
std::this_thread::sleep_for(std::chrono::milliseconds(5)); 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. /// This object will mark the element of the queue as running.
@ -2077,7 +2077,7 @@ bool StorageReplicatedMergeTree::queueTask()
LogEntryPtr & entry = selected.first; LogEntryPtr & entry = selected.first;
if (!entry) if (!entry)
return false; return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO;
time_t prev_attempt_time = entry->last_attempt_time; 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); bool need_sleep = !res && (entry->last_attempt_time - prev_attempt_time < 10);
/// If there was no exception, you do not need to sleep. /// If there was no exception, you do not need to sleep.
return !need_sleep; return need_sleep ? BackgroundProcessingPoolTaskResult::ERROR : BackgroundProcessingPoolTaskResult::SUCCESS;
} }

View File

@ -427,7 +427,7 @@ private:
/** Performs actions from the queue. /** Performs actions from the queue.
*/ */
bool queueTask(); BackgroundProcessingPoolTaskResult queueTask();
/// Postcondition: /// Postcondition:
/// either leader_election is fully initialized (node in ZK is created and the watching thread is launched) /// either leader_election is fully initialized (node in ZK is created and the watching thread is launched)

View File

@ -0,0 +1,50 @@
<test>
<name>cryptographic_hashes</name>
<type>once</type>
<stop_conditions>
<all_of>
<total_time_ms>10000</total_time_ms>
</all_of>
<any_of>
<average_speed_not_changing_for_ms>5000</average_speed_not_changing_for_ms>
<total_time_ms>20000</total_time_ms>
</any_of>
</stop_conditions>
<main_metric>
<max_bytes_per_second/>
</main_metric>
<substitutions>
<substitution>
<name>crypto_hash_func</name>
<values>
<value>MD5</value>
<value>SHA1</value>
<value>SHA224</value>
<value>SHA256</value>
<value>halfMD5</value>
<value>sipHash64</value>
<value>sipHash128</value>
</values>
</substitution>
<substitution>
<name>string</name>
<values>
<value>materialize('')</value>
<value>toString(1000000000+number)</value>
<value>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')</value>
</values>
</substitution>
<substitution>
<name>table</name>
<values>
<value>numbers</value>
<value>numbers_mt</value>
</values>
</substitution>
</substitutions>
<query>SELECT ignore({crypto_hash_func}({string})) FROM system.{table} LIMIT 10000000</query>
</test>

View File

@ -0,0 +1,55 @@
<test>
<name>general_purpose_hashes</name>
<type>once</type>
<stop_conditions>
<all_of>
<total_time_ms>10000</total_time_ms>
</all_of>
<any_of>
<average_speed_not_changing_for_ms>3000</average_speed_not_changing_for_ms>
<total_time_ms>20000</total_time_ms>
</any_of>
</stop_conditions>
<main_metric>
<max_bytes_per_second/>
</main_metric>
<substitutions>
<substitution>
<name>gp_hash_func</name>
<values>
<value>cityHash64</value>
<value>farmHash64</value>
<value>metroHash64</value>
<value>murmurHash2_32</value>
<value>murmurHash2_64</value>
<value>murmurHash3_32</value>
<value>murmurHash3_64</value>
<value>murmurHash3_128</value>
<value>javaHash</value>
<value>hiveHash</value>
<value>xxHash32</value>
<value>xxHash64</value>
</values>
</substitution>
<substitution>
<name>string</name>
<values>
<value>materialize('')</value>
<value>toString(1000000000+number)</value>
<value>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')</value>
</values>
</substitution>
<substitution>
<name>table</name>
<values>
<value>numbers</value>
<value>numbers_mt</value>
</values>
</substitution>
</substitutions>
<query>SELECT ignore({gp_hash_func}({string})) FROM system.{table} LIMIT 10000000</query>
</test>

View File

@ -0,0 +1,37 @@
<test>
<name>left pad test</name>
<tags>
<tag>string</tag>
</tags>
<preconditions>
<table_exists>hashfile</table_exists>
</preconditions>
<type>loop</type>
<stop_conditions>
<all_of>
<iterations>5</iterations>
<min_time_not_changing_for_ms>10000</min_time_not_changing_for_ms>
</all_of>
<any_of>
<iterations>50</iterations>
<total_time_ms>60000</total_time_ms>
</any_of>
</stop_conditions>
<main_metric>
<min_time/>
</main_metric>
<query><![CDATA[SELECT max(length(MobilePhoneModel)) FROM hashfile]]></query>
<query><![CDATA[SELECT max(length(Params)) FROM hashfile]]></query>
<query><![CDATA[SELECT max(length(Title)) FROM hashfile]]></query>
<query><![CDATA[SELECT max(length(URLDomain)) FROM hashfile]]></query>
<query><![CDATA[SELECT max(length(PageCharset)) FROM hashfile]]></query>
<query><![CDATA[SELECT max(length(Referer)) FROM hashfile]]></query>
<query><![CDATA[SELECT max(length(URL)) FROM hashfile]]></query>
<query><![CDATA[SELECT max(length(UTMSource)) FROM hashfile]]></query>
</test>

View File

@ -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.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);" ${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.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" ${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 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 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 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 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" ${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 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 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 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 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" ${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"

View File

@ -1 +0,0 @@
SELECT count() FROM test.mouse_clicks

View File

@ -48,7 +48,7 @@ SELECT
## Working with data types ## 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')`). 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')`).

View File

@ -8,7 +8,7 @@ Includes the `Enum8` and `Enum16` types. `Enum` saves the finite set of pairs of
## Usage examples ## 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 CREATE TABLE t_enum

View File

@ -5,7 +5,7 @@
Types are equivalent to types of C: Types are equivalent to types of C:
- `Float32` - `float` - `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. 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.

View File

@ -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 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 ### Data Selection

View File

@ -44,7 +44,7 @@ void reinsert(const Value & x)
memcpy(&buf[place_value], &x, sizeof(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 ```cpp
for (size_t i = 0; i < rows; i += storage.index_granularity) 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 ```cpp
template <typename T> template <typename T>
@ -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. **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. 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 ## 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 ```cpp
size_t max_block_size; 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; 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 ```cpp
#define MAX_SRC_TABLE_NAMES_TO_STORE 1000 #define MAX_SRC_TABLE_NAMES_TO_STORE 1000

View File

@ -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. 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. Tests should use (create, drop, etc) only tables in `test` database that is assumed to be created beforehand; also tests can use temporary tables.

View File

@ -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 Q4. The number of delays by carrier for 2007
``` sql ``` sql
SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC 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 Q5. The percentage of delays by carrier for 2007

View File

@ -10,7 +10,7 @@ make
There will be some warnings during the process, but this is normal. 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: 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 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. Next, create MergeTree tables on each server and a Distributed above them.
Downloading data (change 'customer' to 'customerd' in the distributed version): Downloading data (change 'customer' to 'customerd' in the distributed version):

View File

@ -79,7 +79,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va
### Command Line Options ### 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. - `--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. - `--user, -u` The username. Default value: default.
- `--password` The password. Default value: empty string. - `--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. - `--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. - `--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. - `--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 ### 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. - Defined in the `-config-file` parameter.
- `./clickhouse-client.xml` - `./clickhouse-client.xml`

View File

@ -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. 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 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 <20> 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 <20> 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. `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). 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. See also the JSONEachRow format.

View File

@ -1,6 +1,6 @@
# Yandex.Metrica Use Case # 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. 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.

View File

@ -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. 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. 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.

View File

@ -42,7 +42,7 @@ Block field `<case>`:
- ``min_part_size_ratio`` The ratio of the minimum size of a table part to the full size of the table. - ``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). - ``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 `<case>` 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 `<case>` matches, ClickHouse applies the `lz4` compression algorithm.
**Example** **Example**
@ -456,7 +456,7 @@ Keys for server/client settings:
- requireTLSv1 Require a TLSv1.2 connection. Acceptable values: `true`, `false`. - requireTLSv1 Require a TLSv1.2 connection. Acceptable values: `true`, `false`.
- fips Activates OpenSSL FIPS mode. Supported if the library's OpenSSL version supports FIPS. - 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: ``<privateKeyPassphraseHandler>``, ``<name>KeyFileHandler</name>``, ``<options><password>test</password></options>``, ``</privateKeyPassphraseHandler>``. - privateKeyPassphraseHandler Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ``<privateKeyPassphraseHandler>``, ``<name>KeyFileHandler</name>``, ``<options><password>test</password></options>``, ``</privateKeyPassphraseHandler>``.
- invalidCertificateHandler Class (subclass of CertificateHandler) for verifying invalid certificates. For example: `` <invalidCertificateHandler> <name>ConsoleCertificateHandler</name> </invalidCertificateHandler>`` . - invalidCertificateHandler Class (subclass of CertificateHandler) for verifying invalid certificates. For example: `` <invalidCertificateHandler> <name>ConsoleCertificateHandler</name> </invalidCertificateHandler>`` .
- disableProtocols Protocols that are not allowed to use. - disableProtocols Protocols that are not allowed to use.
- preferServerCiphers Preferred server ciphers on the client. - 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} ## 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** **Example**

View File

@ -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. 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). 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. 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 ## 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. 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 ## 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). 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} ## 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 ## 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. The default value is 7500.
@ -326,7 +326,7 @@ Replica lag is not controlled.
Enable compilation of queries. By default, 0 (disabled). 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). 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 ## min_count_to_compile
@ -344,7 +344,7 @@ It works for JSONEachRow and TSKV formats.
## output_format_json_quote_64bit_integers ## 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} ## 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. 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** **ClickHouse generates an exception**

View File

@ -2,7 +2,7 @@
# Settings profiles # Settings profiles
A settings profile is a collection of settings grouped under the same name. Each ClickHouse user has a profile. 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: Example:
@ -58,9 +58,9 @@ Example:
</profiles> </profiles>
``` ```
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/) <!--hide--> [Original article](https://clickhouse.yandex/docs/en/operations/settings/settings_profiles/) <!--hide-->

View File

@ -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. - 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. - 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: Columns:
- `name String` — The name of the node. - `name String` — The name of the node.
- `path String` — The path to the node. - `path String` — The path to the node.
- `value String` — Node value. - `value String` — Node value.
- `dataLength Int32` — Size of the value. - `dataLength Int32` — Size of the value.

View File

@ -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). 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'`. 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'`.

View File

@ -24,7 +24,7 @@ Kafka SETTINGS
kafka_topic_list = 'topic1,topic2', kafka_topic_list = 'topic1,topic2',
kafka_group_name = 'group1', kafka_group_name = 'group1',
kafka_format = 'JSONEachRow', kafka_format = 'JSONEachRow',
kafka_row_delimiter = '\n' kafka_row_delimiter = '\n',
kafka_schema = '', kafka_schema = '',
kafka_num_consumers = 2 kafka_num_consumers = 2
``` ```
@ -39,7 +39,7 @@ Required parameters:
Optional parameters: Optional parameters:
- `kafka_row_delimiter` - Character-delimiter of records (rows), which ends the message. - `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. - `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: 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). 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 ```xml
<!-- Global configuration options for all tables of Kafka engine type --> <!-- Global configuration options for all tables of Kafka engine type -->
<kafka> <kafka>
<debug>cgrp</debug> <debug>cgrp</debug>
<auto_offset_reset>smallest</auto_offset_reset> <auto_offset_reset>smallest</auto_offset_reset>

View File

@ -1,8 +1,8 @@
# Merge # 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. 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: Example:
@ -10,17 +10,17 @@ Example:
Merge(hits, '^WatchLog') 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()`. 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. 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. 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. 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. 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: 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. 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/) <!--hide--> [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/merge/) <!--hide-->

View File

@ -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). 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. 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: If a sampling expression is used, the primary key must contain it. Example:
`SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`.

View File

@ -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. 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 ## Creating a Table
@ -33,7 +33,7 @@ For a description of request parameters, see [request description](../../query_l
**Query clauses** **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.
<details markdown="1"><summary>Deprecated Method for Creating a Table</summary> <details markdown="1"><summary>Deprecated Method for Creating a Table</summary>

View File

@ -1,9 +1,9 @@
# SummingMergeTree # 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 ## 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** **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.
<details markdown="1"><summary>Deprecated Method for Creating a Table</summary> <details markdown="1"><summary>Deprecated Method for Creating a Table</summary>
@ -72,7 +72,7 @@ Insert data to it:
:) INSERT INTO summtt Values(1,1),(1,2),(2,1) :) 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 ```sql
SELECT key, sum(value) FROM summtt GROUP BY key SELECT key, sum(value) FROM summtt GROUP BY key

View File

@ -18,7 +18,7 @@ You can use `turbostat` to view the CPU's actual clock rate under a load.
## CPU Scaling Governor ## 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 ```bash
sudo echo 'performance' | tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor sudo echo 'performance' | tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor

View File

@ -37,7 +37,7 @@ Pattern syntax:
Any quantity of any type of events is allowed over the specified time. 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. Any number may be specified in place of 1800.

View File

@ -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). 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) ## uniqExact(x)
@ -382,8 +382,8 @@ For its purpose (calculating quantiles of page loading times), using this functi
## quantileTimingWeighted(level)(x, weight) ## quantileTimingWeighted(level)(x, weight)
Differs from the `quantileTiming` function in that it has a second argument, "weights". Weight is a non-negative integer. 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. The result is calculated as if the `x` value were passed `weight` number of times to the `quantileTiming` function.
## quantileExact(level)(x) ## 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). 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) topK(N)(column)

View File

@ -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. 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. 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 ```bash
$ ls -l /var/lib/clickhouse/data/test/visits/ $ ls -l /var/lib/clickhouse/data/test/visits/
total 48 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_2_2_0
drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_4_4_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 drwxrwxrwx 2 clickhouse clickhouse 4096 May 5 02:55 detached
-rw-rw-rw- 1 clickhouse clickhouse 2 May 5 02:58 increment.txt -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. 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. 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. 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: 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. 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. 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). 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).

View File

@ -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. 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: Example: The table contains discounts for each advertiser in the format:

View File

@ -15,7 +15,7 @@ Example of settings:
</dictionary> </dictionary>
``` ```
Setting ` <lifetime> 0</lifetime> ` prevents updating dictionaries. Setting ` <lifetime> 0</lifetime> ` 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. 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.

View File

@ -243,7 +243,7 @@ This function can also be used in higher-order functions. For example, you can u
## arrayEnumerateUniq(arr, ...) ## 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. 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. This function is useful when using ARRAY JOIN and aggregation of array elements.
Example: Example:
@ -378,7 +378,7 @@ arrayPushFront(array, single_value)
**Parameters** **Parameters**
- `array` Array. - `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** **Example**

View File

@ -16,7 +16,7 @@ For information on connecting and configuring external dictionaries, see "[Exter
`dictGetT('dict_name', 'attr_name', id)` `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. If there is no `id` key in the dictionary, it returns the default value specified in the dictionary description.
## dictGetTOrDefault ## dictGetTOrDefault

View File

@ -102,7 +102,7 @@ coalesce(x,...)
**Returned values** **Returned values**
- The first non-`NULL` argument. - The first non-`NULL` argument.
- `NULL`, if all arguments are `NULL`. - `NULL`, if all arguments are `NULL`.
**Example** **Example**
@ -207,7 +207,7 @@ SELECT nullIf(1, 2)
## assumeNotNull ## 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) assumeNotNull(x)

View File

@ -77,7 +77,7 @@ pointInPolygon((x, y), [(a, b), (c, d) ...], ...)
**Input values** **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. - `[(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. - 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.

View File

@ -336,7 +336,7 @@ toColumnTypeName(value)
**Returned values** **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`** **Example of the difference between` toTypeName ' and ' toColumnTypeName`**
@ -376,7 +376,7 @@ dumpColumnStructure(value)
**Returned values** **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** **Example**

View File

@ -18,7 +18,7 @@ For a case-insensitive search, use the function `positionCaseInsensitiveUTF8`.
## match(haystack, pattern) ## 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. Returns 0 if it doesn't match, or 1 if it matches.

View File

@ -34,7 +34,7 @@ For example, `cutToFirstSignificantSubdomain('https://news.yandex.com.tr/') = 'y
### path ### 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 ### pathFull
@ -42,7 +42,7 @@ The same as above, but including query string and fragment. Example: /top/news.h
### queryString ### 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 ### fragment

View File

@ -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. 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`. 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`. However, you can delete old data using `ALTER TABLE ... DROP PARTITION`.

View File

@ -76,7 +76,7 @@ There is no `DETACH DATABASE` query.
## DROP ## DROP
This query has two types: `DROP DATABASE` and `DROP TABLE`. This query has two types: `DROP DATABASE` and `DROP TABLE`.
``` sql ``` sql
DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster] DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster]

View File

@ -5,7 +5,7 @@ Groups of operators are listed in order of priority (the higher it is in the lis
## Access Operators ## 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. `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 ## Associativity
All binary operators have left associativity. For example, `1 + 2 + 3` is transformed to `plus(plus(1, 2), 3)`. 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. 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.

View File

@ -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`. 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 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. 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`). 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 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 ### 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 #### 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: Here is an example with the `t_null` table:
@ -847,18 +847,18 @@ FROM t_null
#### Distributed Subqueries {#select-distributed-subqueries} #### 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 !!! attention
Remember that the algorithms described below may work differently depending on the [settings](../operations/settings/settings.md) `distributed_product_mode` setting. 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 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`. 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. 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.

View File

@ -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 and C-style comments are supported.
SQL-style comments: from `--` to the end of the line. The space after `--` can be omitted. 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 ## Keywords

View File

@ -4,7 +4,7 @@
`url(URL, format, structure)` - returns a table created from the `URL` with given `url(URL, format, structure)` - returns a table created from the `URL` with given
`format` and `structure`. `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. format - [format](../../interfaces/formats.md#formats) of the data.

View File

@ -7,7 +7,7 @@
Type های float در ClickHouse مشابه C می باشد: Type های float در ClickHouse مشابه C می باشد:
- `Float32` - `float` - `Float32` - `float`
- `Float64` - `double` - `Float64` - `double`
توصیه می کنیم که داده ها را هرزمان که امکان پذیره است به جای float به صورت int ذخیره کنید. برای مثال: تبدیل دقت اعداد به یک مقدار int، مثل سرعت page load در قالب میلی ثانیه. توصیه می کنیم که داده ها را هرزمان که امکان پذیره است به جای float به صورت int ذخیره کنید. برای مثال: تبدیل دقت اعداد به یک مقدار int، مثل سرعت page load در قالب میلی ثانیه.

View File

@ -2,7 +2,7 @@
# AggregateFunction(name, types_of_arguments...) # AggregateFunction(name, types_of_arguments...)
حالت متوسط از توابع aggregate. برای دریافت آن، از توابع aggregate به همراه پسوند '-State' استفاده کنید. برای اطلاعات بیشتر قسمت "AggregatingMergeTree" را ببینید. حالت متوسط از توابع aggregate. برای دریافت آن، از توابع aggregate به همراه پسوند '-State' استفاده کنید. برای اطلاعات بیشتر قسمت "AggregatingMergeTree" را ببینید.
</div> </div>
[مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/nested_data_structures/aggregatefunction/) <!--hide--> [مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/nested_data_structures/aggregatefunction/) <!--hide-->

View File

@ -69,7 +69,7 @@ LIMIT 10
<div dir="rtl" markdown="1"> <div dir="rtl" markdown="1">
ساده ترین راه برای فکر کردن به یک ساختار داده nestet این است که، یک nestet مجموعه ای از آرایه های چند ستونی با طول ثابت است. ساده ترین راه برای فکر کردن به یک ساختار داده nestet این است که، یک nestet مجموعه ای از آرایه های چند ستونی با طول ثابت است.
تنها جایی که یک دستور SELECT می تواند کل ساختار داده ی nested را به جای مشخص کردن ستون های آن قرار دهد، عبارت ARRAY JOIN است. برای اطلاعات بیشتر "ARRAY JOIN clouse" را ببینید. مثال: تنها جایی که یک دستور SELECT می تواند کل ساختار داده ی nested را به جای مشخص کردن ستون های آن قرار دهد، عبارت ARRAY JOIN است. برای اطلاعات بیشتر "ARRAY JOIN clouse" را ببینید. مثال:

View File

@ -1,6 +1,6 @@
<div dir="rtl" markdown="1"> <div dir="rtl" markdown="1">
# بنچمارک AMPLab Big Data # بنچمارک AMPLab Big Data
ببینید <https://amplab.cs.berkeley.edu/benchmark/> ببینید <https://amplab.cs.berkeley.edu/benchmark/>

File diff suppressed because one or more lines are too long

View File

@ -197,7 +197,7 @@ Q4. تعداد تاخیرها براساس carrier در سال 78
</div> </div>
``` sql ``` 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
``` ```
<div dir="rtl" markdown="1"> <div dir="rtl" markdown="1">

View File

@ -14,7 +14,7 @@ make
<div dir="rtl" markdown="1"> <div dir="rtl" markdown="1">
در هنگام پردازش چند warnings نمایش داده می شود که مشکلی نیست و طبیعی است. در هنگام پردازش چند warnings نمایش داده می شود که مشکلی نیست و طبیعی است.
`dbgen` و ` dists.dss` را در یک جا با 800 گیگابایت فضای حالی دیسک قرار دهید. `dbgen` و ` dists.dss` را در یک جا با 800 گیگابایت فضای حالی دیسک قرار دهید.

View File

@ -46,7 +46,7 @@ sudo apt-get install clickhouse-client clickhouse-server
شما همچنین می توانید از طریق لینک زیر پکیج ClickHouse را به صورت دستی دانلود و نصب کنید: <https://repo.yandex.ru/clickhouse/deb/stable/main/>. شما همچنین می توانید از طریق لینک زیر پکیج ClickHouse را به صورت دستی دانلود و نصب کنید: <https://repo.yandex.ru/clickhouse/deb/stable/main/>.
ClickHouse دارای تنظیمات محدودیت دسترسی می باشد. این تنظیمات در فایل 'users.xml' (کنار 'config.xml') می باشد. به صورت پیش فرض دسترسی برای کاربر 'default' از همه جا بدون نیاز به پسورد وجود دارد. 'user/default/networks' را مشاهده کنید. برای اطلاعات بیشتر قسمت "تنظیمات فایل ها" را مشاهده کنید. ClickHouse دارای تنظیمات محدودیت دسترسی می باشد. این تنظیمات در فایل 'users.xml' (کنار 'config.xml') می باشد. به صورت پیش فرض دسترسی برای کاربر 'default' از همه جا بدون نیاز به پسورد وجود دارد. 'user/default/networks' را مشاهده کنید. برای اطلاعات بیشتر قسمت "تنظیمات فایل ها" را مشاهده کنید.
### نصب از طریق Source ### نصب از طریق Source

View File

@ -45,7 +45,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA
در حالت intercative، شما یک command line برای درج query های خود دریافت می کنید. در حالت 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 درخواست می شد. اگر چند خطی (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 نوشته می شوند. 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" برای خروج از کلاینت، 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 خارج می شوید. شما میتوانید query های طولانی را با فشردن Ctrl-C کنسل کنید. هر چند، بعد از این کار همچنان نیاز به انتظار چند ثانیه ای برای قطع کردن درخواست توسط سرور می باشید. امکان کنسل کردن یک query در مراحل خاص وجود ندارد. اگر شما صبر نکنید و برای بار دوم Ctrl+C را وارد کنید از client خارج می شوید.
کلاینت commant-line اجازه ی پاس دادن داده های external (جداول موقت external) را برای query ها می دهد. برای اطلاعات بیشتر به بخش "داده های External برای پردازش query" مراجعه کنید. کلاینت commant-line اجازه ی پاس دادن داده های external (جداول موقت external) را برای query ها می دهد. برای اطلاعات بیشتر به بخش "داده های External برای پردازش query" مراجعه کنید.
## پیکربندی {#interfaces_cli_configuration} ## پیکربندی {#interfaces_cli_configuration}

View File

@ -43,9 +43,9 @@ Format | INSERT | SELECT
Dates با فرمت YYY-MM-DD نوشته می شوند و به همین حالت پارس می شوند، اما با هر کاراکتری به عنوان جدا کننده. Dates به همراه زمان با فرمت YYYY-MM-DD hh:mm:ss نوشته می شوند و با همین فرمت پارس می شوند، اما با هر کاراکتری به عنوان جداکننده. این در منطقه زمان سیستم در زمانی که کلاینت یا سرور شروع می شود (بسته به اینکه کدام یک از داده ها را تشکیل می دهد) رخ می دهد. برای تاریخ همراه با زمان DST مشخص نمی شود. پس اگر یک دامپ دارای زمان DST باشد، دامپ، داده ها را به طور غیرمستقیم مطابقت نمی دهد و پارسینگ، یکی از دو ساعت را انتخاب خواهد کرد. در طول عملیات خواندن، تاریخ ها و تاریخ و ساعت های نادرست می توانند به صورت null و یا natural overflow پارس شوند، بدون اینکه پیغام خطایی نمایش دهند. 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 می تواند به هر یک از حالات زیر پارس شود::
</div> </div>
@ -146,7 +146,7 @@ SearchPhrase=baku count()=1000
Comma Separated Values format ([RFC](https://tools.ietf.org/html/rfc4180)). 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 از دست میروند)
</div> </div>
@ -340,7 +340,7 @@ JSON با جاوااسکریپت سازگار است. برای اطمینان ا
## Pretty ## 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 نمایش داده شده است): فرمت Pretty از total values (هنگام استفاده از WITH TOTALS) و extreme (هنگام که 'extremes' برابر با 1 است) برای خروجی پشتیبانی می کند. در این موارد، total values و extreme values بعد از نمایش داده های اصلی در جداول جدا، چاپ می شوند. مثال (برای فرمت PrettyCompact نمایش داده شده است):

View File

@ -87,7 +87,7 @@ $ echo 'SELECT 1 FORMAT Pretty' | curl 'http://localhost:8123/?' --data-binary @
<div dir="rtl" markdown="1"> <div dir="rtl" markdown="1">
برای 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 نصب شده است) شما می توانید از فرمت فشرده سازی داخلی 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` را به درخواست خود اضافه کنید. همچنین استفاده از فشرده سازی استاندارد gzip در HTTP ممکن است. برای ارسال درخواست POST و فشرده سازی آن به صورت gzip، هدر `Content-Encoding: gzip` را به request خود اضافه کنید. برای اینکه ClickHouse، response فشرده شده به صورت gzip برای شما ارسال کند، ابتدا باید `enable_http_compression` را در تنظیمات ClickHouse فعال کنید و در ادامه هدر `Accept-Encoding: gzip` را به درخواست خود اضافه کنید.

View File

@ -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. در این سیستم ها، شما توان عملیاتی حدود صدها هزار سطر در ثانیه را دارید، اما نه صدها میلیون سطر در ثانیه. این بسیار قابل توجه است چون سیستم هایی وجود دارند که توانایی ذخیره سازی مقادیر ستون ها را به صورت جداگانه دارند، اما به دلیل بهینه سازی آنها برای دیگر سناریو ها، نمیتوانند به طور موثر پردازش های تحیلی انجام دهند. برای مثال HBase، BigTable، Cassandra و HyperTable. در این سیستم ها، شما توان عملیاتی حدود صدها هزار سطر در ثانیه را دارید، اما نه صدها میلیون سطر در ثانیه.

View File

@ -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).
Пример автоматического определения типа данных: Пример автоматического определения типа данных:

View File

@ -44,13 +44,13 @@ void reinsert(const Value & x)
memcpy(&buf[place_value], &x, sizeof(x)); memcpy(&buf[place_value], &x, sizeof(x));
``` ```
**6.** В выражениях `if`, `for`, `while` и т.д. перед открывающей скобкой ставится пробел (в отличие от вызовов функций). **6.** В выражениях `if`, `for`, `while` и т.д. перед открывающей скобкой ставится пробел (в отличие от вызовов функций).
```cpp ```cpp
for (size_t i = 0; i < rows; i += storage.index_granularity) for (size_t i = 0; i < rows; i += storage.index_granularity)
``` ```
**7.** Вокруг бинарных операторов (`+`, `-`, `*`, `/`, `%`, ...), а также тернарного оператора `?:` ставятся пробелы. **7.** Вокруг бинарных операторов (`+`, `-`, `*`, `/`, `%`, ...), а также тернарного оператора `?:` ставятся пробелы.
```cpp ```cpp
UInt16 year = (s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0'); 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'); UInt8 day = (s[8] - '0') * 10 + (s[9] - '0');
``` ```
**8.** Если ставится перенос строки, то оператор пишется на новой строке, и перед ним увеличивается отступ. **8.** Если ставится перенос строки, то оператор пишется на новой строке, и перед ним увеличивается отступ.
```cpp ```cpp
if (elapsed_ns) if (elapsed_ns)
@ -67,7 +67,7 @@ if (elapsed_ns)
<< bytes_read_on_server * 1000.0 / elapsed_ns << " MB/s.) "; << bytes_read_on_server * 1000.0 / elapsed_ns << " MB/s.) ";
``` ```
**9.** Внутри строки можно, выполнять выравнивание с помощью пробелов. **9.** Внутри строки можно, выполнять выравнивание с помощью пробелов.
```cpp ```cpp
dst.ClickLogID = click.LogID; dst.ClickLogID = click.LogID;
@ -85,7 +85,7 @@ dst.ClickGoodEvent = click.GoodEvent;
**13.** Оператор `[]` не отделяется пробелами. **13.** Оператор `[]` не отделяется пробелами.
**14.** В выражении `template <...>`, между `template` и `<` ставится пробел, а после `<` и до `>` не ставится. **14.** В выражении `template <...>`, между `template` и `<` ставится пробел, а после `<` и до `>` не ставится.
```cpp ```cpp
template <typename TKey, typename TValue> template <typename TKey, typename TValue>
@ -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.** Нельзя писать мусорные комментарии (автор, дата создания...) в начале каждого файла. **7.** Нельзя писать мусорные комментарии (автор, дата создания...) в начале каждого файла.
@ -305,7 +305,7 @@ void executeQuery(
size_t max_block_size; size_t max_block_size;
``` ```
**2.** Имена функций (методов) camelCase с маленькой буквы. **2.** Имена функций (методов) camelCase с маленькой буквы.
```cpp ```cpp
std::string getName() const override { return "Memory"; } std::string getName() const override { return "Memory"; }
@ -348,7 +348,7 @@ class IBlockInputStream
bool info_successfully_loaded = false; bool info_successfully_loaded = false;
``` ```
**9.** В именах `define` и глобальных констант используется ALL_CAPS с подчёркиванием. **9.** В именах `define` и глобальных констант используется ALL_CAPS с подчёркиванием.
```cpp ```cpp
#define MAX_SRC_TABLE_NAMES_TO_STORE 1000 #define MAX_SRC_TABLE_NAMES_TO_STORE 1000

File diff suppressed because one or more lines are too long

View File

@ -166,7 +166,7 @@ SELECT Origin, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND
Q4. Количество задержек по перевозчикам за 2007 год Q4. Количество задержек по перевозчикам за 2007 год
``` sql ``` 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 год Q5. Процент задержек по перевозчикам за 2007 год

View File

@ -72,9 +72,9 @@ ClickHouse - столбцовая система управления базам
### По вводу-выводу ### По вводу-выводу
1. Для выполнения аналитического запроса, требуется прочитать небольшое количество столбцов таблицы. В столбцовой БД для этого можно читать только нужные данные. Например, если вам требуется только 5 столбцов из 100, то следует рассчитывать на 20-кратное уменьшение ввода-вывода. 1. Для выполнения аналитического запроса, требуется прочитать небольшое количество столбцов таблицы. В столбцовой БД для этого можно читать только нужные данные. Например, если вам требуется только 5 столбцов из 100, то следует рассчитывать на 20-кратное уменьшение ввода-вывода.
2. Так как данные читаются пачками, то их проще сжимать. Данные, лежащие по столбцам также лучше сжимаются. За счёт этого, дополнительно уменьшается объём ввода-вывода. 2. Так как данные читаются пачками, то их проще сжимать. Данные, лежащие по столбцам также лучше сжимаются. За счёт этого, дополнительно уменьшается объём ввода-вывода.
3. За счёт уменьшения ввода-вывода, больше данных влезает в системный кэш. 3. За счёт уменьшения ввода-вывода, больше данных влезает в системный кэш.
Например, для запроса "посчитать количество записей для каждой рекламной системы", требуется прочитать один столбец "идентификатор рекламной системы", который занимает 1 байт в несжатом виде. Если большинство переходов было не с рекламных систем, то можно рассчитывать хотя бы на десятикратное сжатие этого столбца. При использовании быстрого алгоритма сжатия, возможно разжатие данных со скоростью более нескольких гигабайт несжатых данных в секунду. То есть, такой запрос может выполняться со скоростью около нескольких миллиардов строк в секунду на одном сервере. На практике, такая скорость действительно достигается. Например, для запроса "посчитать количество записей для каждой рекламной системы", требуется прочитать один столбец "идентификатор рекламной системы", который занимает 1 байт в несжатом виде. Если большинство переходов было не с рекламных систем, то можно рассчитывать хотя бы на десятикратное сжатие этого столбца. При использовании быстрого алгоритма сжатия, возможно разжатие данных со скоростью более нескольких гигабайт несжатых данных в секунду. То есть, такой запрос может выполняться со скоростью около нескольких миллиардов строк в секунду на одном сервере. На практике, такая скорость действительно достигается.

Some files were not shown because too many files have changed in this diff Show More