mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge
This commit is contained in:
commit
85d2dbcd76
@ -84,7 +84,7 @@ public:
|
||||
|
||||
void ignore(size_t n)
|
||||
{
|
||||
while (!eof() && n != 0)
|
||||
while (n != 0 && !eof())
|
||||
{
|
||||
size_t bytes_to_ignore = std::min(static_cast<size_t>(working_buffer.end() - pos), n);
|
||||
pos += bytes_to_ignore;
|
||||
|
@ -5,10 +5,9 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
|
||||
/** Запрос с секцией FORMAT.
|
||||
*/
|
||||
*/
|
||||
class ASTQueryWithOutput : public IAST
|
||||
{
|
||||
public:
|
||||
@ -16,6 +15,11 @@ public:
|
||||
|
||||
ASTQueryWithOutput() = default;
|
||||
ASTQueryWithOutput(const StringRange range_) : IAST(range_) {}
|
||||
|
||||
/** Возвращает указатель на формат. Если типом объекта является ASTSelectQuery,
|
||||
* то эта функция возвращает указатель на формат из последнего SELECT'а цепочки UNION ALL.
|
||||
*/
|
||||
virtual const IAST * getFormat() const { return format.get(); }
|
||||
};
|
||||
|
||||
|
||||
|
@ -34,6 +34,9 @@ public:
|
||||
|
||||
ASTPtr clone() const override;
|
||||
|
||||
/// Возвращает указатель на формат из последнего SELECT'а цепочки UNION ALL.
|
||||
const IAST * getFormat() const override;
|
||||
|
||||
public:
|
||||
bool distinct = false;
|
||||
ASTPtr select_expression_list;
|
||||
|
@ -15,115 +15,13 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wsign-compare"
|
||||
|
||||
/** Более точное сравнение.
|
||||
* Отличается от Field::operator< и Field::operator== тем, что сравнивает значения разных числовых типов между собой.
|
||||
* Правила сравнения - такие же, что и в FunctionsComparison.
|
||||
* В том числе, сравнение знаковых и беззнаковых оставляем UB.
|
||||
*/
|
||||
class FieldVisitorAccurateEquals : public StaticVisitor<bool>
|
||||
{
|
||||
public:
|
||||
bool operator() (const Null & l, const Null & r) const { return true; }
|
||||
bool operator() (const Null & l, const UInt64 & r) const { return false; }
|
||||
bool operator() (const Null & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const Null & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const Null & l, const String & r) const { return false; }
|
||||
bool operator() (const Null & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const UInt64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; }
|
||||
bool operator() (const UInt64 & l, const Int64 & r) const { return l == r; }
|
||||
bool operator() (const UInt64 & l, const Float64 & r) const { return l == r; }
|
||||
bool operator() (const UInt64 & l, const String & r) const { return false; }
|
||||
bool operator() (const UInt64 & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const Int64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const Int64 & l, const UInt64 & r) const { return l == r; }
|
||||
bool operator() (const Int64 & l, const Int64 & r) const { return l == r; }
|
||||
bool operator() (const Int64 & l, const Float64 & r) const { return l == r; }
|
||||
bool operator() (const Int64 & l, const String & r) const { return false; }
|
||||
bool operator() (const Int64 & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const Float64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const Float64 & l, const UInt64 & r) const { return l == r; }
|
||||
bool operator() (const Float64 & l, const Int64 & r) const { return l == r; }
|
||||
bool operator() (const Float64 & l, const Float64 & r) const { return l == r; }
|
||||
bool operator() (const Float64 & l, const String & r) const { return false; }
|
||||
bool operator() (const Float64 & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const String & l, const Null & r) const { return false; }
|
||||
bool operator() (const String & l, const UInt64 & r) const { return false; }
|
||||
bool operator() (const String & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const String & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const String & l, const String & r) const { return l == r; }
|
||||
bool operator() (const String & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const Array & l, const Null & r) const { return false; }
|
||||
bool operator() (const Array & l, const UInt64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const String & r) const { return false; }
|
||||
bool operator() (const Array & l, const Array & r) const { return l == r; }
|
||||
};
|
||||
|
||||
class FieldVisitorAccurateLess : public StaticVisitor<bool>
|
||||
{
|
||||
public:
|
||||
bool operator() (const Null & l, const Null & r) const { return false; }
|
||||
bool operator() (const Null & l, const UInt64 & r) const { return true; }
|
||||
bool operator() (const Null & l, const Int64 & r) const { return true; }
|
||||
bool operator() (const Null & l, const Float64 & r) const { return true; }
|
||||
bool operator() (const Null & l, const String & r) const { return true; }
|
||||
bool operator() (const Null & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const UInt64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; }
|
||||
bool operator() (const UInt64 & l, const Int64 & r) const { return l < r; }
|
||||
bool operator() (const UInt64 & l, const Float64 & r) const { return l < r; }
|
||||
bool operator() (const UInt64 & l, const String & r) const { return true; }
|
||||
bool operator() (const UInt64 & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const Int64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const Int64 & l, const UInt64 & r) const { return l < r; }
|
||||
bool operator() (const Int64 & l, const Int64 & r) const { return l < r; }
|
||||
bool operator() (const Int64 & l, const Float64 & r) const { return l < r; }
|
||||
bool operator() (const Int64 & l, const String & r) const { return true; }
|
||||
bool operator() (const Int64 & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const Float64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const Float64 & l, const UInt64 & r) const { return l < r; }
|
||||
bool operator() (const Float64 & l, const Int64 & r) const { return l < r; }
|
||||
bool operator() (const Float64 & l, const Float64 & r) const { return l < r; }
|
||||
bool operator() (const Float64 & l, const String & r) const { return true; }
|
||||
bool operator() (const Float64 & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const String & l, const Null & r) const { return false; }
|
||||
bool operator() (const String & l, const UInt64 & r) const { return false; }
|
||||
bool operator() (const String & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const String & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const String & l, const String & r) const { return l < r; }
|
||||
bool operator() (const String & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const Array & l, const Null & r) const { return false; }
|
||||
bool operator() (const Array & l, const UInt64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const String & r) const { return false; }
|
||||
bool operator() (const Array & l, const Array & r) const { return l < r; }
|
||||
};
|
||||
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
/** Диапазон с открытыми или закрытыми концами; возможно, неограниченный.
|
||||
*/
|
||||
struct Range
|
||||
{
|
||||
private:
|
||||
static bool equals(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateEquals(), lhs, rhs); }
|
||||
static bool less(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateLess(), lhs, rhs); }
|
||||
static bool equals(const Field & lhs, const Field & rhs);
|
||||
static bool less(const Field & lhs, const Field & rhs);
|
||||
|
||||
public:
|
||||
Field left; /// левая граница, если есть
|
||||
|
@ -850,11 +850,11 @@ private:
|
||||
/// Формат может быть указан в запросе.
|
||||
if (ASTQueryWithOutput * query_with_output = dynamic_cast<ASTQueryWithOutput *>(&*parsed_query))
|
||||
{
|
||||
if (query_with_output->format)
|
||||
if (query_with_output->getFormat() != nullptr)
|
||||
{
|
||||
if (has_vertical_output_suffix)
|
||||
throw Exception("Output format already specified", ErrorCodes::CLIENT_OUTPUT_FORMAT_SPECIFIED);
|
||||
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(&*query_with_output->format))
|
||||
if (const ASTIdentifier * id = typeid_cast<const ASTIdentifier *>(query_with_output->getFormat()))
|
||||
current_format = id->name;
|
||||
}
|
||||
}
|
||||
|
@ -81,6 +81,10 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
|
||||
{
|
||||
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
|
||||
|
||||
const auto & lifetime = dict_ptr->getLifetime();
|
||||
std::uniform_int_distribution<std::uint64_t> distribution{lifetime.min_sec, lifetime.max_sec};
|
||||
update_times[name] = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
|
||||
|
||||
const auto dict_it = dictionaries.find(name);
|
||||
if (dict_it->second.dict)
|
||||
dict_it->second.dict->set(dict_ptr.release());
|
||||
@ -90,10 +94,6 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
|
||||
/// erase stored exception on success
|
||||
dict_it->second.exception = std::exception_ptr{};
|
||||
|
||||
const auto & lifetime = dict_ptr->getLifetime();
|
||||
std::uniform_int_distribution<std::uint64_t> distribution{lifetime.min_sec, lifetime.max_sec};
|
||||
update_times[name] = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
|
||||
|
||||
recreated_failed_dictionaries.push_back(name);
|
||||
}
|
||||
}
|
||||
|
@ -211,8 +211,8 @@ void executeQuery(
|
||||
{
|
||||
const ASTQueryWithOutput * ast_query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get());
|
||||
|
||||
String format_name = ast_query_with_output && ast_query_with_output->format
|
||||
? typeid_cast<const ASTIdentifier &>(*ast_query_with_output->format).name
|
||||
String format_name = ast_query_with_output && (ast_query_with_output->getFormat() != nullptr)
|
||||
? typeid_cast<const ASTIdentifier &>(*ast_query_with_output->getFormat()).name
|
||||
: context.getDefaultFormat();
|
||||
|
||||
BlockOutputStreamPtr out = context.getFormatFactory().getOutput(format_name, ostr, streams.in_sample);
|
||||
|
@ -172,5 +172,14 @@ ASTPtr ASTSelectQuery::clone() const
|
||||
|
||||
return ptr;
|
||||
}
|
||||
|
||||
const IAST * ASTSelectQuery::getFormat() const
|
||||
{
|
||||
const ASTSelectQuery * query = this;
|
||||
while (!query->next_union_all.isNull())
|
||||
query = static_cast<const ASTSelectQuery *>(query->next_union_all.get());
|
||||
return query->format.get();
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
|
@ -8,7 +8,6 @@
|
||||
#include <DB/Parsers/ParserSetQuery.h>
|
||||
#include <DB/Parsers/ParserSelectQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -296,6 +295,8 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
|
||||
bool has_format = false;
|
||||
|
||||
/// FORMAT format_name
|
||||
if (s_format.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
@ -308,6 +309,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
typeid_cast<ASTIdentifier &>(*select_query->format).kind = ASTIdentifier::Format;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
has_format = true;
|
||||
}
|
||||
|
||||
// UNION ALL select query
|
||||
@ -317,6 +319,13 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
|
||||
if (s_all.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
if (has_format)
|
||||
{
|
||||
/// FORMAT может быть задан только в последнем запросе цепочки UNION ALL.
|
||||
expected = "FORMAT only in the last SELECT of the UNION ALL chain";
|
||||
return false;
|
||||
}
|
||||
|
||||
ParserSelectQuery select_p;
|
||||
if (!select_p.parse(pos, end, select_query->next_union_all, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
@ -11,7 +11,6 @@
|
||||
#include <DB/IO/CompressedReadBuffer.h>
|
||||
#include <DB/IO/HashingReadBuffer.h>
|
||||
#include <DB/Columns/ColumnsNumber.h>
|
||||
#include <statdaemons/ext/scope_guard.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -116,12 +115,6 @@ struct Stream
|
||||
readIntBinary(mrk_mark.offset_in_compressed_file, mrk_hashing_buf);
|
||||
readIntBinary(mrk_mark.offset_in_decompressed_block, mrk_hashing_buf);
|
||||
|
||||
/// На всякий случай, сохраним смещение в файле и размер предыдущего блока.
|
||||
SCOPE_EXIT(
|
||||
prev_offset_in_compressed_file = mrk_mark.offset_in_compressed_file;
|
||||
prev_buffer_size = uncompressed_hashing_buf.buffer().size();
|
||||
);
|
||||
|
||||
bool has_alternative_mark = false;
|
||||
MarkInCompressedFile alternative_data_mark;
|
||||
MarkInCompressedFile data_mark;
|
||||
@ -145,18 +138,6 @@ struct Stream
|
||||
if (uncompressed_hashing_buf.eof())
|
||||
return;
|
||||
}
|
||||
else if (uncompressed_hashing_buf.offset() == 0)
|
||||
{
|
||||
/// Восстановим засечку на конец предыдущего блока по сохраненным данным
|
||||
has_alternative_mark = true;
|
||||
alternative_data_mark.offset_in_compressed_file = prev_offset_in_compressed_file;
|
||||
alternative_data_mark.offset_in_decompressed_block = prev_buffer_size;
|
||||
|
||||
if (mrk_mark == alternative_data_mark)
|
||||
return;
|
||||
}
|
||||
|
||||
std::cout << "mrk_mark " << mrk_mark.offset_in_compressed_file << ' ' << mrk_mark.offset_in_decompressed_block << std::endl;
|
||||
|
||||
data_mark.offset_in_compressed_file = compressed_hashing_buf.count() - uncompressing_buf.getSizeCompressed();
|
||||
data_mark.offset_in_decompressed_block = uncompressed_hashing_buf.offset();
|
||||
@ -180,10 +161,6 @@ struct Stream
|
||||
checksums.files[name + ".mrk"] = MergeTreeData::DataPart::Checksums::Checksum(
|
||||
mrk_hashing_buf.count(), mrk_hashing_buf.getHash());
|
||||
}
|
||||
|
||||
private:
|
||||
size_t prev_offset_in_compressed_file{};
|
||||
size_t prev_buffer_size{};
|
||||
};
|
||||
|
||||
/// Возвращает количество строк. Добавляет в checksums чексуммы всех файлов столбца.
|
||||
|
@ -8,6 +8,136 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/// Преобразование строки с датой или датой-с-временем в UInt64, содержащим числовое значение даты или даты-с-временем.
|
||||
UInt64 stringToDateOrDateTime(const String & s)
|
||||
{
|
||||
ReadBufferFromString in(s);
|
||||
|
||||
if (s.size() == strlen("YYYY-MM-DD"))
|
||||
{
|
||||
DayNum_t date{};
|
||||
readDateText(date, in);
|
||||
return UInt64(date);
|
||||
}
|
||||
else
|
||||
{
|
||||
time_t date_time{};
|
||||
readDateTimeText(date_time, in);
|
||||
if (!in.eof())
|
||||
throw Exception("String is too long for DateTime: " + s);
|
||||
return UInt64(date_time);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wsign-compare"
|
||||
|
||||
/** Более точное сравнение.
|
||||
* Отличается от Field::operator< и Field::operator== тем, что сравнивает значения разных числовых типов между собой.
|
||||
* Правила сравнения - такие же, что и в FunctionsComparison.
|
||||
* В том числе, сравнение знаковых и беззнаковых оставляем UB.
|
||||
*/
|
||||
class FieldVisitorAccurateEquals : public StaticVisitor<bool>
|
||||
{
|
||||
public:
|
||||
bool operator() (const Null & l, const Null & r) const { return true; }
|
||||
bool operator() (const Null & l, const UInt64 & r) const { return false; }
|
||||
bool operator() (const Null & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const Null & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const Null & l, const String & r) const { return false; }
|
||||
bool operator() (const Null & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const UInt64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; }
|
||||
bool operator() (const UInt64 & l, const Int64 & r) const { return l == r; }
|
||||
bool operator() (const UInt64 & l, const Float64 & r) const { return l == r; }
|
||||
bool operator() (const UInt64 & l, const String & r) const { return l == stringToDateOrDateTime(r); }
|
||||
bool operator() (const UInt64 & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const Int64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const Int64 & l, const UInt64 & r) const { return l == r; }
|
||||
bool operator() (const Int64 & l, const Int64 & r) const { return l == r; }
|
||||
bool operator() (const Int64 & l, const Float64 & r) const { return l == r; }
|
||||
bool operator() (const Int64 & l, const String & r) const { return false; }
|
||||
bool operator() (const Int64 & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const Float64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const Float64 & l, const UInt64 & r) const { return l == r; }
|
||||
bool operator() (const Float64 & l, const Int64 & r) const { return l == r; }
|
||||
bool operator() (const Float64 & l, const Float64 & r) const { return l == r; }
|
||||
bool operator() (const Float64 & l, const String & r) const { return false; }
|
||||
bool operator() (const Float64 & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const String & l, const Null & r) const { return false; }
|
||||
bool operator() (const String & l, const UInt64 & r) const { return stringToDateOrDateTime(l) == r; }
|
||||
bool operator() (const String & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const String & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const String & l, const String & r) const { return l == r; }
|
||||
bool operator() (const String & l, const Array & r) const { return false; }
|
||||
|
||||
bool operator() (const Array & l, const Null & r) const { return false; }
|
||||
bool operator() (const Array & l, const UInt64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const String & r) const { return false; }
|
||||
bool operator() (const Array & l, const Array & r) const { return l == r; }
|
||||
};
|
||||
|
||||
class FieldVisitorAccurateLess : public StaticVisitor<bool>
|
||||
{
|
||||
public:
|
||||
bool operator() (const Null & l, const Null & r) const { return false; }
|
||||
bool operator() (const Null & l, const UInt64 & r) const { return true; }
|
||||
bool operator() (const Null & l, const Int64 & r) const { return true; }
|
||||
bool operator() (const Null & l, const Float64 & r) const { return true; }
|
||||
bool operator() (const Null & l, const String & r) const { return true; }
|
||||
bool operator() (const Null & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const UInt64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; }
|
||||
bool operator() (const UInt64 & l, const Int64 & r) const { return l < r; }
|
||||
bool operator() (const UInt64 & l, const Float64 & r) const { return l < r; }
|
||||
bool operator() (const UInt64 & l, const String & r) const { return l < stringToDateOrDateTime(r); }
|
||||
bool operator() (const UInt64 & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const Int64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const Int64 & l, const UInt64 & r) const { return l < r; }
|
||||
bool operator() (const Int64 & l, const Int64 & r) const { return l < r; }
|
||||
bool operator() (const Int64 & l, const Float64 & r) const { return l < r; }
|
||||
bool operator() (const Int64 & l, const String & r) const { return true; }
|
||||
bool operator() (const Int64 & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const Float64 & l, const Null & r) const { return false; }
|
||||
bool operator() (const Float64 & l, const UInt64 & r) const { return l < r; }
|
||||
bool operator() (const Float64 & l, const Int64 & r) const { return l < r; }
|
||||
bool operator() (const Float64 & l, const Float64 & r) const { return l < r; }
|
||||
bool operator() (const Float64 & l, const String & r) const { return true; }
|
||||
bool operator() (const Float64 & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const String & l, const Null & r) const { return false; }
|
||||
bool operator() (const String & l, const UInt64 & r) const { return stringToDateOrDateTime(l) < r; }
|
||||
bool operator() (const String & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const String & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const String & l, const String & r) const { return l < r; }
|
||||
bool operator() (const String & l, const Array & r) const { return true; }
|
||||
|
||||
bool operator() (const Array & l, const Null & r) const { return false; }
|
||||
bool operator() (const Array & l, const UInt64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const Int64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const Float64 & r) const { return false; }
|
||||
bool operator() (const Array & l, const String & r) const { return false; }
|
||||
bool operator() (const Array & l, const Array & r) const { return l < r; }
|
||||
};
|
||||
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
|
||||
inline bool Range::equals(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateEquals(), lhs, rhs); }
|
||||
inline bool Range::less(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateLess(), lhs, rhs); }
|
||||
|
||||
|
||||
PKCondition::PKCondition(ASTPtr query, const Context & context_, const NamesAndTypesList & all_columns, const SortDescription & sort_descr_)
|
||||
: sort_descr(sort_descr_)
|
||||
{
|
||||
|
18
dbms/tests/queries/0_stateless/00098_k_union_all.reference
Normal file
18
dbms/tests/queries/0_stateless/00098_k_union_all.reference
Normal file
@ -0,0 +1,18 @@
|
||||
[1m1[0m
|
||||
|
||||
1
|
||||
[1m1[0m
|
||||
|
||||
1
|
||||
[1m1[0m
|
||||
|
||||
1
|
||||
[1m1[0m
|
||||
|
||||
1
|
||||
[1m1[0m
|
||||
|
||||
1
|
||||
[1m1[0m
|
||||
|
||||
1
|
3
dbms/tests/queries/0_stateless/00098_k_union_all.sql
Normal file
3
dbms/tests/queries/0_stateless/00098_k_union_all.sql
Normal file
@ -0,0 +1,3 @@
|
||||
SELECT 1 FORMAT PrettySpace;
|
||||
SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace;
|
||||
SELECT 1 UNION ALL SELECT 1 UNION ALL SELECT 1 FORMAT PrettySpace;
|
@ -0,0 +1,14 @@
|
||||
inf
|
||||
0
|
||||
inf
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
inf
|
||||
0
|
||||
0
|
||||
0
|
||||
inf
|
||||
0
|
@ -0,0 +1,143 @@
|
||||
DROP TABLE IF EXISTS series;
|
||||
|
||||
CREATE TABLE series(i UInt32, x_value Float64, y_value Float64) ENGINE = Memory;
|
||||
|
||||
/* Тестовые данные */
|
||||
|
||||
INSERT INTO series(i, x_value, y_value) VALUES (1, 5.6,-4.4),(2, -9.6,3),(3, -1.3,-4),(4, 5.3,9.7),(5, 4.4,0.037),(6, -8.6,-7.8),(7, 5.1,9.3),(8, 7.9,-3.6),(9, -8.2,0.62),(10, -3,7.3);
|
||||
|
||||
/* varSamp */
|
||||
|
||||
SELECT varSamp(x_value) FROM (SELECT x_value FROM series LIMIT 0);
|
||||
SELECT varSamp(x_value) FROM (SELECT x_value FROM series LIMIT 1);
|
||||
|
||||
SELECT round(abs(res1 - res2), 6) FROM
|
||||
(
|
||||
SELECT
|
||||
varSamp(x_value) AS res1,
|
||||
(sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / (count() - 1) AS res2
|
||||
FROM series
|
||||
);
|
||||
|
||||
/* stddevSamp */
|
||||
|
||||
SELECT stddevSamp(x_value) FROM (SELECT x_value FROM series LIMIT 0);
|
||||
SELECT stddevSamp(x_value) FROM (SELECT x_value FROM series LIMIT 1);
|
||||
|
||||
SELECT round(abs(res1 - res2), 6) FROM
|
||||
(
|
||||
SELECT
|
||||
stddevSamp(x_value) AS res1,
|
||||
sqrt((sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / (count() - 1)) AS res2
|
||||
FROM series
|
||||
);
|
||||
|
||||
/* varPop */
|
||||
|
||||
SELECT varPop(x_value) FROM (SELECT x_value FROM series LIMIT 0);
|
||||
SELECT varPop(x_value) FROM (SELECT x_value FROM series LIMIT 1);
|
||||
|
||||
SELECT round(abs(res1 - res2), 6) FROM
|
||||
(
|
||||
SELECT
|
||||
varPop(x_value) AS res1,
|
||||
(sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / count() AS res2
|
||||
FROM series
|
||||
);
|
||||
|
||||
/* stddevPop */
|
||||
|
||||
SELECT stddevPop(x_value) FROM (SELECT x_value FROM series LIMIT 0);
|
||||
SELECT stddevPop(x_value) FROM (SELECT x_value FROM series LIMIT 1);
|
||||
|
||||
SELECT round(abs(res1 - res2), 6) FROM
|
||||
(
|
||||
SELECT
|
||||
stddevPop(x_value) AS res1,
|
||||
sqrt((sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / count()) AS res2
|
||||
FROM series
|
||||
);
|
||||
|
||||
/* covarSamp */
|
||||
|
||||
SELECT covarSamp(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0);
|
||||
SELECT covarSamp(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1);
|
||||
|
||||
SELECT round(abs(COVAR1 - COVAR2), 6)
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
arrayJoin([1]) AS ID2,
|
||||
covarSamp(x_value, y_value) AS COVAR1
|
||||
FROM series
|
||||
) ANY INNER JOIN
|
||||
(
|
||||
SELECT
|
||||
arrayJoin([1]) AS ID2,
|
||||
sum(VAL) / (count() - 1) AS COVAR2
|
||||
FROM
|
||||
(
|
||||
SELECT (X - AVG_X) * (Y - AVG_Y) AS VAL
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
toUInt32(arrayJoin([1, 2, 3, 4, 5, 6, 7, 8, 9, 10])) AS ID,
|
||||
avg(x_value) AS AVG_X,
|
||||
avg(y_value) AS AVG_Y
|
||||
FROM series
|
||||
) ANY INNER JOIN
|
||||
(
|
||||
SELECT
|
||||
i AS ID,
|
||||
x_value AS X,
|
||||
y_value AS Y
|
||||
FROM series
|
||||
) USING ID
|
||||
)
|
||||
) USING ID2;
|
||||
|
||||
/* covarPop */
|
||||
|
||||
SELECT covarPop(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0);
|
||||
SELECT covarPop(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1);
|
||||
|
||||
SELECT round(abs(COVAR1 - COVAR2), 6)
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
arrayJoin([1]) AS ID2,
|
||||
covarPop(x_value, y_value) AS COVAR1
|
||||
FROM series
|
||||
) ANY INNER JOIN
|
||||
(
|
||||
SELECT
|
||||
arrayJoin([1]) AS ID2,
|
||||
sum(VAL) / count() AS COVAR2
|
||||
FROM
|
||||
(
|
||||
SELECT (X - AVG_X) * (Y - AVG_Y) AS VAL
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
toUInt32(arrayJoin([1, 2, 3, 4, 5, 6, 7, 8, 9, 10])) AS ID,
|
||||
avg(x_value) AS AVG_X,
|
||||
avg(y_value) AS AVG_Y
|
||||
FROM series
|
||||
) ANY INNER JOIN
|
||||
(
|
||||
SELECT
|
||||
i AS ID,
|
||||
x_value AS X,
|
||||
y_value AS Y
|
||||
FROM series
|
||||
) USING ID
|
||||
)
|
||||
) USING ID2;
|
||||
|
||||
/* corr */
|
||||
|
||||
SELECT corr(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0);
|
||||
SELECT corr(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1);
|
||||
|
||||
SELECT round(abs(corr(x_value, y_value) - covarPop(x_value, y_value) / (stddevPop(x_value) * stddevPop(y_value))), 6) FROM series;
|
||||
|
Loading…
Reference in New Issue
Block a user