mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
Merge remote-tracking branch 'upstream/master' into nikvas0/set_opt
This commit is contained in:
commit
8e65c9a094
@ -1,11 +1,11 @@
|
||||
# This strings autochanged from release_lib.sh:
|
||||
set(VERSION_REVISION 54415)
|
||||
set(VERSION_REVISION 54417)
|
||||
set(VERSION_MAJOR 19)
|
||||
set(VERSION_MINOR 3)
|
||||
set(VERSION_PATCH 4)
|
||||
set(VERSION_GITHASH 263e69e861b769eae7e2bcc79d87673e3a08d376)
|
||||
set(VERSION_DESCRIBE v19.3.4-testing)
|
||||
set(VERSION_STRING 19.3.4)
|
||||
set(VERSION_MINOR 5)
|
||||
set(VERSION_PATCH 1)
|
||||
set(VERSION_GITHASH 628ed349c335b79a441a1bd6e4bc791d61dfe62c)
|
||||
set(VERSION_DESCRIBE v19.5.1.1-testing)
|
||||
set(VERSION_STRING 19.5.1.1)
|
||||
# end of autochange
|
||||
|
||||
set(VERSION_EXTRA "" CACHE STRING "")
|
||||
|
@ -163,7 +163,7 @@ public:
|
||||
size_t old_size = data_to.size();
|
||||
data_to.resize(data_to.size() + size);
|
||||
|
||||
data.getManyFloat(levels.levels.data(), levels.permutation.data(), size, &data_to[old_size]);
|
||||
data.getManyFloat(levels.levels.data(), levels.permutation.data(), size, data_to.data() + old_size);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -171,7 +171,7 @@ public:
|
||||
size_t old_size = data_to.size();
|
||||
data_to.resize(data_to.size() + size);
|
||||
|
||||
data.getMany(levels.levels.data(), levels.permutation.data(), size, &data_to[old_size]);
|
||||
data.getMany(levels.levels.data(), levels.permutation.data(), size, data_to.data() + old_size);
|
||||
}
|
||||
}
|
||||
else
|
||||
|
@ -152,7 +152,7 @@ void ColumnAggregateFunction::insertRangeFrom(const IColumn & from, size_t start
|
||||
|
||||
size_t old_size = data.size();
|
||||
data.resize(old_size + length);
|
||||
memcpy(&data[old_size], &from_concrete.data[start], length * sizeof(data[0]));
|
||||
memcpy(data.data() + old_size, &from_concrete.data[start], length * sizeof(data[0]));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -140,7 +140,7 @@ void ColumnDecimal<T>::insertRangeFrom(const IColumn & src, size_t start, size_t
|
||||
|
||||
size_t old_size = data.size();
|
||||
data.resize(old_size + length);
|
||||
memcpy(&data[old_size], &src_vec.data[start], length * sizeof(data[0]));
|
||||
memcpy(data.data() + old_size, &src_vec.data[start], length * sizeof(data[0]));
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
|
@ -55,7 +55,7 @@ void ColumnFixedString::insert(const Field & x)
|
||||
|
||||
size_t old_size = chars.size();
|
||||
chars.resize_fill(old_size + n);
|
||||
memcpy(&chars[old_size], s.data(), s.size());
|
||||
memcpy(chars.data() + old_size, s.data(), s.size());
|
||||
}
|
||||
|
||||
void ColumnFixedString::insertFrom(const IColumn & src_, size_t index)
|
||||
@ -67,7 +67,7 @@ void ColumnFixedString::insertFrom(const IColumn & src_, size_t index)
|
||||
|
||||
size_t old_size = chars.size();
|
||||
chars.resize(old_size + n);
|
||||
memcpySmallAllowReadWriteOverflow15(&chars[old_size], &src.chars[n * index], n);
|
||||
memcpySmallAllowReadWriteOverflow15(chars.data() + old_size, &src.chars[n * index], n);
|
||||
}
|
||||
|
||||
void ColumnFixedString::insertData(const char * pos, size_t length)
|
||||
@ -77,7 +77,7 @@ void ColumnFixedString::insertData(const char * pos, size_t length)
|
||||
|
||||
size_t old_size = chars.size();
|
||||
chars.resize_fill(old_size + n);
|
||||
memcpy(&chars[old_size], pos, length);
|
||||
memcpy(chars.data() + old_size, pos, length);
|
||||
}
|
||||
|
||||
StringRef ColumnFixedString::serializeValueIntoArena(size_t index, Arena & arena, char const *& begin) const
|
||||
@ -91,7 +91,7 @@ const char * ColumnFixedString::deserializeAndInsertFromArena(const char * pos)
|
||||
{
|
||||
size_t old_size = chars.size();
|
||||
chars.resize(old_size + n);
|
||||
memcpy(&chars[old_size], pos, n);
|
||||
memcpy(chars.data() + old_size, pos, n);
|
||||
return pos + n;
|
||||
}
|
||||
|
||||
@ -151,7 +151,7 @@ void ColumnFixedString::insertRangeFrom(const IColumn & src, size_t start, size_
|
||||
|
||||
size_t old_size = chars.size();
|
||||
chars.resize(old_size + length * n);
|
||||
memcpy(&chars[old_size], &src_concrete.chars[start * n], length * n);
|
||||
memcpy(chars.data() + old_size, &src_concrete.chars[start * n], length * n);
|
||||
}
|
||||
|
||||
ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result_size_hint) const
|
||||
|
@ -185,7 +185,7 @@ const char * ColumnString::deserializeAndInsertFromArena(const char * pos)
|
||||
const size_t old_size = chars.size();
|
||||
const size_t new_size = old_size + string_size;
|
||||
chars.resize(new_size);
|
||||
memcpy(&chars[old_size], pos, string_size);
|
||||
memcpy(chars.data() + old_size, pos, string_size);
|
||||
|
||||
offsets.push_back(new_size);
|
||||
return pos + string_size;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <string.h>
|
||||
#include <cstring>
|
||||
#include <cassert>
|
||||
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Common/PODArray.h>
|
||||
@ -71,21 +72,25 @@ public:
|
||||
|
||||
Field operator[](size_t n) const override
|
||||
{
|
||||
assert(n < size());
|
||||
return Field(&chars[offsetAt(n)], sizeAt(n) - 1);
|
||||
}
|
||||
|
||||
void get(size_t n, Field & res) const override
|
||||
{
|
||||
assert(n < size());
|
||||
res.assignString(&chars[offsetAt(n)], sizeAt(n) - 1);
|
||||
}
|
||||
|
||||
StringRef getDataAt(size_t n) const override
|
||||
{
|
||||
assert(n < size());
|
||||
return StringRef(&chars[offsetAt(n)], sizeAt(n) - 1);
|
||||
}
|
||||
|
||||
StringRef getDataAtWithTerminatingZero(size_t n) const override
|
||||
{
|
||||
assert(n < size());
|
||||
return StringRef(&chars[offsetAt(n)], sizeAt(n));
|
||||
}
|
||||
|
||||
@ -103,7 +108,7 @@ public:
|
||||
const size_t new_size = old_size + size_to_append;
|
||||
|
||||
chars.resize(new_size);
|
||||
memcpy(&chars[old_size], s.c_str(), size_to_append);
|
||||
memcpy(chars.data() + old_size, s.c_str(), size_to_append);
|
||||
offsets.push_back(new_size);
|
||||
}
|
||||
|
||||
@ -132,7 +137,7 @@ public:
|
||||
const size_t new_size = old_size + size_to_append;
|
||||
|
||||
chars.resize(new_size);
|
||||
memcpySmallAllowReadWriteOverflow15(&chars[old_size], &src.chars[offset], size_to_append);
|
||||
memcpySmallAllowReadWriteOverflow15(chars.data() + old_size, &src.chars[offset], size_to_append);
|
||||
offsets.push_back(new_size);
|
||||
}
|
||||
}
|
||||
@ -143,7 +148,7 @@ public:
|
||||
const size_t new_size = old_size + size_to_append;
|
||||
|
||||
chars.resize(new_size);
|
||||
memcpySmallAllowReadWriteOverflow15(&chars[old_size], &src.chars[0], size_to_append);
|
||||
memcpySmallAllowReadWriteOverflow15(chars.data() + old_size, &src.chars[0], size_to_append);
|
||||
offsets.push_back(new_size);
|
||||
}
|
||||
}
|
||||
@ -155,7 +160,7 @@ public:
|
||||
|
||||
chars.resize(new_size);
|
||||
if (length)
|
||||
memcpy(&chars[old_size], pos, length);
|
||||
memcpy(chars.data() + old_size, pos, length);
|
||||
chars[old_size + length] = 0;
|
||||
offsets.push_back(new_size);
|
||||
}
|
||||
@ -167,7 +172,7 @@ public:
|
||||
const size_t new_size = old_size + length;
|
||||
|
||||
chars.resize(new_size);
|
||||
memcpy(&chars[old_size], pos, length);
|
||||
memcpy(chars.data() + old_size, pos, length);
|
||||
offsets.push_back(new_size);
|
||||
}
|
||||
|
||||
|
@ -141,7 +141,7 @@ void ColumnVector<T>::insertRangeFrom(const IColumn & src, size_t start, size_t
|
||||
|
||||
size_t old_size = data.size();
|
||||
data.resize(old_size + length);
|
||||
memcpy(&data[old_size], &src_vec.data[start], length * sizeof(data[0]));
|
||||
memcpy(data.data() + old_size, &src_vec.data[start], length * sizeof(data[0]));
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <string.h>
|
||||
#include <cstddef>
|
||||
#include <cassert>
|
||||
#include <algorithm>
|
||||
#include <memory>
|
||||
|
||||
@ -159,6 +160,7 @@ public:
|
||||
size_t size() const { return (c_end - c_start) / ELEMENT_SIZE; }
|
||||
size_t capacity() const { return (c_end_of_storage - c_start) / ELEMENT_SIZE; }
|
||||
|
||||
/// This method is safe to use only for information about memory usage.
|
||||
size_t allocated_bytes() const { return c_end_of_storage - c_start + pad_right + pad_left; }
|
||||
|
||||
void clear() { c_end = c_start; }
|
||||
@ -271,8 +273,18 @@ public:
|
||||
const T * data() const { return t_start(); }
|
||||
|
||||
/// The index is signed to access -1th element without pointer overflow.
|
||||
T & operator[] (ssize_t n) { return t_start()[n]; }
|
||||
const T & operator[] (ssize_t n) const { return t_start()[n]; }
|
||||
T & operator[] (ssize_t n)
|
||||
{
|
||||
/// <= size, because taking address of one element past memory range is Ok in C++ (expression like &arr[arr.size()] is perfectly valid).
|
||||
assert((n >= (static_cast<ssize_t>(pad_left_) ? -1 : 0)) && (n <= static_cast<ssize_t>(this->size())));
|
||||
return t_start()[n];
|
||||
}
|
||||
|
||||
const T & operator[] (ssize_t n) const
|
||||
{
|
||||
assert((n >= (static_cast<ssize_t>(pad_left_) ? -1 : 0)) && (n <= static_cast<ssize_t>(this->size())));
|
||||
return t_start()[n];
|
||||
}
|
||||
|
||||
T & front() { return t_start()[0]; }
|
||||
T & back() { return t_end()[-1]; }
|
||||
|
@ -85,7 +85,7 @@ CompressionCodecPtr CompressionCodecFactory::get(const UInt8 byte_code) const
|
||||
const auto family_code_and_creator = family_code_with_codec.find(byte_code);
|
||||
|
||||
if (family_code_and_creator == family_code_with_codec.end())
|
||||
throw Exception("Unknown codec family code : " + toString(byte_code), ErrorCodes::UNKNOWN_CODEC);
|
||||
throw Exception("Unknown codec family code: " + toString(byte_code), ErrorCodes::UNKNOWN_CODEC);
|
||||
|
||||
return family_code_and_creator->second({}, nullptr);
|
||||
}
|
||||
|
@ -6,9 +6,14 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
LimitBlockInputStream::LimitBlockInputStream(const BlockInputStreamPtr & input, UInt64 limit_, UInt64 offset_, bool always_read_till_end_)
|
||||
LimitBlockInputStream::LimitBlockInputStream(const BlockInputStreamPtr & input, UInt64 limit_, UInt64 offset_, bool always_read_till_end_, bool use_limit_as_total_rows_approx)
|
||||
: limit(limit_), offset(offset_), always_read_till_end(always_read_till_end_)
|
||||
{
|
||||
if (use_limit_as_total_rows_approx)
|
||||
{
|
||||
addTotalRowsApprox(static_cast<size_t>(limit));
|
||||
}
|
||||
|
||||
children.push_back(input);
|
||||
}
|
||||
|
||||
|
@ -16,8 +16,9 @@ public:
|
||||
* returns an empty block, and this causes the query to be canceled.
|
||||
* If always_read_till_end = true - reads all the data to the end, but ignores them. This is necessary in rare cases:
|
||||
* when otherwise, due to the cancellation of the request, we would not have received the data for GROUP BY WITH TOTALS from the remote server.
|
||||
* If use_limit_as_total_rows_approx = true, then addTotalRowsApprox is called to use the limit in progress & stats
|
||||
*/
|
||||
LimitBlockInputStream(const BlockInputStreamPtr & input, UInt64 limit_, UInt64 offset_, bool always_read_till_end_ = false);
|
||||
LimitBlockInputStream(const BlockInputStreamPtr & input, UInt64 limit_, UInt64 offset_, bool always_read_till_end_ = false, bool use_limit_as_total_rows_approx = false);
|
||||
|
||||
String getName() const override { return "Limit"; }
|
||||
|
||||
|
@ -69,7 +69,7 @@ void DataTypeFixedString::deserializeBinary(IColumn & column, ReadBuffer & istr)
|
||||
data.resize(old_size + n);
|
||||
try
|
||||
{
|
||||
istr.readStrict(reinterpret_cast<char *>(&data[old_size]), n);
|
||||
istr.readStrict(reinterpret_cast<char *>(data.data() + old_size), n);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -130,9 +130,9 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt
|
||||
|
||||
if (size)
|
||||
{
|
||||
#ifdef __x86_64__
|
||||
#ifdef __SSE2__
|
||||
/// An optimistic branch in which more efficient copying is possible.
|
||||
if (offset + 16 * UNROLL_TIMES <= data.allocated_bytes() && istr.position() + size + 16 * UNROLL_TIMES <= istr.buffer().end())
|
||||
if (offset + 16 * UNROLL_TIMES <= data.capacity() && istr.position() + size + 16 * UNROLL_TIMES <= istr.buffer().end())
|
||||
{
|
||||
const __m128i * sse_src_pos = reinterpret_cast<const __m128i *>(istr.position());
|
||||
const __m128i * sse_src_end = sse_src_pos + (size + (16 * UNROLL_TIMES - 1)) / 16 / UNROLL_TIMES * UNROLL_TIMES;
|
||||
@ -140,22 +140,11 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt
|
||||
|
||||
while (sse_src_pos < sse_src_end)
|
||||
{
|
||||
/// NOTE gcc 4.9.2 unrolls the loop, but for some reason uses only one xmm register.
|
||||
/// for (size_t j = 0; j < UNROLL_TIMES; ++j)
|
||||
/// _mm_storeu_si128(sse_dst_pos + j, _mm_loadu_si128(sse_src_pos + j));
|
||||
for (size_t j = 0; j < UNROLL_TIMES; ++j)
|
||||
_mm_storeu_si128(sse_dst_pos + j, _mm_loadu_si128(sse_src_pos + j));
|
||||
|
||||
sse_src_pos += UNROLL_TIMES;
|
||||
sse_dst_pos += UNROLL_TIMES;
|
||||
|
||||
if (UNROLL_TIMES >= 4) __asm__("movdqu %0, %%xmm0" :: "m"(sse_src_pos[-4]));
|
||||
if (UNROLL_TIMES >= 3) __asm__("movdqu %0, %%xmm1" :: "m"(sse_src_pos[-3]));
|
||||
if (UNROLL_TIMES >= 2) __asm__("movdqu %0, %%xmm2" :: "m"(sse_src_pos[-2]));
|
||||
if (UNROLL_TIMES >= 1) __asm__("movdqu %0, %%xmm3" :: "m"(sse_src_pos[-1]));
|
||||
|
||||
if (UNROLL_TIMES >= 4) __asm__("movdqu %%xmm0, %0" : "=m"(sse_dst_pos[-4]));
|
||||
if (UNROLL_TIMES >= 3) __asm__("movdqu %%xmm1, %0" : "=m"(sse_dst_pos[-3]));
|
||||
if (UNROLL_TIMES >= 2) __asm__("movdqu %%xmm2, %0" : "=m"(sse_dst_pos[-2]));
|
||||
if (UNROLL_TIMES >= 1) __asm__("movdqu %%xmm3, %0" : "=m"(sse_dst_pos[-1]));
|
||||
}
|
||||
|
||||
istr.position() += size;
|
||||
|
@ -77,7 +77,7 @@ void RegionsNames::reload()
|
||||
throw Poco::Exception("Logical error. Maybe size estimate of " + names_source->getSourceName() + " is wrong.");
|
||||
|
||||
new_chars.resize(old_size + name_entry.name.length() + 1);
|
||||
memcpy(&new_chars[old_size], name_entry.name.c_str(), name_entry.name.length() + 1);
|
||||
memcpy(new_chars.data() + old_size, name_entry.name.c_str(), name_entry.name.length() + 1);
|
||||
|
||||
if (name_entry.id > max_region_id)
|
||||
{
|
||||
@ -92,7 +92,7 @@ void RegionsNames::reload()
|
||||
while (name_entry.id >= new_names_refs.size())
|
||||
new_names_refs.resize(new_names_refs.size() * 2, StringRef("", 0));
|
||||
|
||||
new_names_refs[name_entry.id] = StringRef(&new_chars[old_size], name_entry.name.length());
|
||||
new_names_refs[name_entry.id] = StringRef(new_chars.data() + old_size, name_entry.name.length());
|
||||
}
|
||||
|
||||
chars[language_id].swap(new_chars);
|
||||
|
@ -59,7 +59,7 @@ namespace
|
||||
{
|
||||
size_t old_size = buf.size();
|
||||
buf.reserve(old_size + MAX_VARINT_SIZE);
|
||||
UInt8 * ptr = &buf[old_size];
|
||||
UInt8 * ptr = buf.data() + old_size;
|
||||
ptr = writeVarint(value, ptr);
|
||||
buf.resize_assume_reserved(ptr - buf.data());
|
||||
}
|
||||
@ -200,7 +200,7 @@ void ProtobufWriter::SimpleWriter::writeUInt(UInt32 field_number, UInt64 value)
|
||||
{
|
||||
size_t old_size = buffer.size();
|
||||
buffer.reserve(old_size + 2 * MAX_VARINT_SIZE);
|
||||
UInt8 * ptr = &buffer[old_size];
|
||||
UInt8 * ptr = buffer.data() + old_size;
|
||||
ptr = writeFieldNumber(field_number, VARINT, ptr);
|
||||
ptr = writeVarint(value, ptr);
|
||||
buffer.resize_assume_reserved(ptr - buffer.data());
|
||||
@ -223,7 +223,7 @@ void ProtobufWriter::SimpleWriter::writeFixed(UInt32 field_number, T value)
|
||||
constexpr WireType wire_type = (sizeof(T) == 4) ? BITS32 : BITS64;
|
||||
size_t old_size = buffer.size();
|
||||
buffer.reserve(old_size + MAX_VARINT_SIZE + sizeof(T));
|
||||
UInt8 * ptr = &buffer[old_size];
|
||||
UInt8 * ptr = buffer.data() + old_size;
|
||||
ptr = writeFieldNumber(field_number, wire_type, ptr);
|
||||
memcpy(ptr, &value, sizeof(T));
|
||||
ptr += sizeof(T);
|
||||
@ -234,7 +234,7 @@ void ProtobufWriter::SimpleWriter::writeString(UInt32 field_number, const String
|
||||
{
|
||||
size_t old_size = buffer.size();
|
||||
buffer.reserve(old_size + 2 * MAX_VARINT_SIZE + str.size);
|
||||
UInt8 * ptr = &buffer[old_size];
|
||||
UInt8 * ptr = buffer.data() + old_size;
|
||||
ptr = writeFieldNumber(field_number, LENGTH_DELIMITED, ptr);
|
||||
ptr = writeVarint(str.size, ptr);
|
||||
memcpy(ptr, str.data, str.size);
|
||||
@ -294,7 +294,7 @@ void ProtobufWriter::SimpleWriter::addFixedToRepeatedPack(T value)
|
||||
static_assert((sizeof(T) == 4) || (sizeof(T) == 8));
|
||||
size_t old_size = buffer.size();
|
||||
buffer.resize(old_size + sizeof(T));
|
||||
memcpy(&buffer[old_size], &value, sizeof(T));
|
||||
memcpy(buffer.data() + old_size, &value, sizeof(T));
|
||||
}
|
||||
|
||||
|
||||
|
@ -54,7 +54,7 @@ public:
|
||||
array_column = checkAndGetColumn<ColumnArray>(temp_column.get());
|
||||
}
|
||||
block.getByPosition(result).column
|
||||
= ColumnArray::create(first_column->replicate(array_column->getOffsets()), array_column->getOffsetsPtr());
|
||||
= ColumnArray::create(first_column->replicate(array_column->getOffsets())->convertToFullColumnIfConst(), array_column->getOffsetsPtr());
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -55,14 +55,11 @@ struct Progress
|
||||
/// Each value separately is changed atomically (but not whole object).
|
||||
bool incrementPiecewiseAtomically(const Progress & rhs)
|
||||
{
|
||||
if (!rhs.rows)
|
||||
return false;
|
||||
|
||||
rows += rhs.rows;
|
||||
bytes += rhs.bytes;
|
||||
total_rows += rhs.total_rows;
|
||||
|
||||
return true;
|
||||
return rhs.rows ? true : false;
|
||||
}
|
||||
|
||||
void reset()
|
||||
|
@ -34,7 +34,7 @@ private:
|
||||
|
||||
size_t old_size = vector.size();
|
||||
vector.resize(old_size * 2);
|
||||
internal_buffer = Buffer(reinterpret_cast<Position>(&vector[old_size]), reinterpret_cast<Position>(vector.data() + vector.size()));
|
||||
internal_buffer = Buffer(reinterpret_cast<Position>(vector.data() + old_size), reinterpret_cast<Position>(vector.data() + vector.size()));
|
||||
working_buffer = internal_buffer;
|
||||
}
|
||||
|
||||
|
@ -49,7 +49,6 @@ StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multi
|
||||
setColumns(ColumnsDescription({{"number", std::make_shared<DataTypeUInt64>()}}));
|
||||
}
|
||||
|
||||
|
||||
BlockInputStreams StorageSystemNumbers::read(
|
||||
const Names & column_names,
|
||||
const SelectQueryInfo &,
|
||||
@ -75,7 +74,7 @@ BlockInputStreams StorageSystemNumbers::read(
|
||||
res[i] = std::make_shared<NumbersBlockInputStream>(max_block_size, offset + i * max_block_size, num_streams * max_block_size);
|
||||
|
||||
if (limit) /// This formula is how to split 'limit' elements to 'num_streams' chunks almost uniformly.
|
||||
res[i] = std::make_shared<LimitBlockInputStream>(res[i], *limit * (i + 1) / num_streams - *limit * i / num_streams, 0);
|
||||
res[i] = std::make_shared<LimitBlockInputStream>(res[i], *limit * (i + 1) / num_streams - *limit * i / num_streams, 0, false, true);
|
||||
}
|
||||
|
||||
return res;
|
||||
|
@ -1,3 +1,7 @@
|
||||
< X-ClickHouse-Progress: {"read_rows":"0","read_bytes":"0","total_rows":"10"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"5","read_bytes":"40","total_rows":"10"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"10","read_bytes":"80","total_rows":"10"}
|
||||
9
|
||||
< X-ClickHouse-Progress: {"read_rows":"1","read_bytes":"8","total_rows":"0"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"2","read_bytes":"16","total_rows":"0"}
|
||||
< X-ClickHouse-Progress: {"read_rows":"3","read_bytes":"24","total_rows":"0"}
|
||||
|
@ -3,6 +3,7 @@
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?max_block_size=5&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d 'SELECT max(number) FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]'
|
||||
# This test will fail with external poco (progress not supported)
|
||||
|
||||
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d 'SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]'
|
||||
|
@ -0,0 +1,4 @@
|
||||
b1 b1
|
||||
b1 b1
|
||||
b1 b1
|
||||
b1 b1
|
15
dbms/tests/queries/0_stateless/00914_join_bgranvea.sql
Normal file
15
dbms/tests/queries/0_stateless/00914_join_bgranvea.sql
Normal file
@ -0,0 +1,15 @@
|
||||
USE test;
|
||||
|
||||
DROP TABLE IF EXISTS table1;
|
||||
DROP TABLE IF EXISTS table2;
|
||||
|
||||
CREATE TABLE table1 (A String, B String, ts DateTime) ENGINE = MergeTree PARTITION BY toStartOfDay(ts) ORDER BY (ts, A, B);
|
||||
CREATE TABLE table2 (B String, ts DateTime) ENGINE = MergeTree PARTITION BY toStartOfDay(ts) ORDER BY (ts, B);
|
||||
|
||||
insert into table1 values('a1','b1','2019-02-05 16:50:00'),('a1','b1','2019-02-05 16:55:00');
|
||||
insert into table2 values('b1','2019-02-05 16:50:00'),('b1','2019-02-05 16:55:00');
|
||||
|
||||
SELECT t1.B, t2.B FROM table1 t1 ALL INNER JOIN table2 t2 ON t1.B = t2.B ORDER BY t1.B, t2.B;
|
||||
|
||||
DROP TABLE table1;
|
||||
DROP TABLE table2;
|
1
dbms/tests/queries/0_stateless/00914_replicate.reference
Normal file
1
dbms/tests/queries/0_stateless/00914_replicate.reference
Normal file
@ -0,0 +1 @@
|
||||
[[\'a\']]
|
1
dbms/tests/queries/0_stateless/00914_replicate.sql
Normal file
1
dbms/tests/queries/0_stateless/00914_replicate.sql
Normal file
@ -0,0 +1 @@
|
||||
SELECT CAST(replicate(['a'], [1]) AS String);
|
4
debian/changelog
vendored
4
debian/changelog
vendored
@ -1,5 +1,5 @@
|
||||
clickhouse (19.3.4) unstable; urgency=low
|
||||
clickhouse (19.5.1.1) unstable; urgency=low
|
||||
|
||||
* Modified source code
|
||||
|
||||
-- <root@yandex-team.ru> Fri, 15 Feb 2019 14:50:36 +0300
|
||||
-- clickhouse-release <clickhouse-release@yandex-team.ru> Sat, 09 Mar 2019 10:45:02 +300
|
||||
|
@ -1,7 +1,7 @@
|
||||
FROM ubuntu:18.04
|
||||
|
||||
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
|
||||
ARG version=19.3.4
|
||||
ARG version=19.5.1.1
|
||||
|
||||
RUN apt-get update \
|
||||
&& apt-get install --yes --no-install-recommends \
|
||||
|
@ -1,7 +1,7 @@
|
||||
FROM ubuntu:18.04
|
||||
|
||||
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
|
||||
ARG version=19.3.4
|
||||
ARG version=19.5.1.1
|
||||
ARG gosu_ver=1.10
|
||||
|
||||
RUN apt-get update \
|
||||
|
@ -1,7 +1,7 @@
|
||||
FROM ubuntu:18.04
|
||||
|
||||
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
|
||||
ARG version=19.3.4
|
||||
ARG version=19.5.1.1
|
||||
|
||||
RUN apt-get update && \
|
||||
apt-get install -y apt-transport-https dirmngr && \
|
||||
|
Loading…
Reference in New Issue
Block a user