mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Merge branch 'master' into fix-bug-orc
This commit is contained in:
commit
81e8cb4be6
@ -5,8 +5,9 @@
|
||||
#include <type_traits>
|
||||
|
||||
|
||||
/** \brief Returns value `from` converted to type `To` while retaining bit representation.
|
||||
* `To` and `From` must satisfy `CopyConstructible`.
|
||||
/** Returns value `from` converted to type `To` while retaining bit representation.
|
||||
* `To` and `From` must satisfy `CopyConstructible`.
|
||||
* In contrast to std::bit_cast can cast types of different width.
|
||||
*/
|
||||
template <typename To, typename From>
|
||||
std::decay_t<To> bit_cast(const From & from)
|
||||
@ -15,13 +16,3 @@ std::decay_t<To> bit_cast(const From & from)
|
||||
memcpy(static_cast<void*>(&res), &from, std::min(sizeof(res), sizeof(from)));
|
||||
return res;
|
||||
}
|
||||
|
||||
/** \brief Returns value `from` converted to type `To` while retaining bit representation.
|
||||
* `To` and `From` must satisfy `CopyConstructible`.
|
||||
*/
|
||||
template <typename To, typename From>
|
||||
std::decay_t<To> safe_bit_cast(const From & from)
|
||||
{
|
||||
static_assert(sizeof(To) == sizeof(From), "bit cast on types of different width");
|
||||
return bit_cast<To, From>(from);
|
||||
}
|
||||
|
2
contrib/krb5
vendored
2
contrib/krb5
vendored
@ -1 +1 @@
|
||||
Subproject commit d879821c7a4c70b0c3ad739d9951d1a2b1903df7
|
||||
Subproject commit b89e20367b074bd02dd118a6534099b21e88b3c3
|
2
contrib/replxx
vendored
2
contrib/replxx
vendored
@ -1 +1 @@
|
||||
Subproject commit 3fd0e3c9364a589447453d9906d854ebd8d385c5
|
||||
Subproject commit 5d04501f93a4fb7f0bb8b73b8f614bc986f9e25b
|
@ -2,8 +2,10 @@ set (SOURCE_DIR ${CMAKE_SOURCE_DIR}/contrib/zlib-ng)
|
||||
|
||||
add_definitions(-DZLIB_COMPAT)
|
||||
add_definitions(-DWITH_GZFILEOP)
|
||||
add_definitions(-DUNALIGNED_OK)
|
||||
add_definitions(-DUNALIGNED64_OK)
|
||||
if(NOT ARCH_S390X)
|
||||
add_definitions(-DUNALIGNED_OK)
|
||||
add_definitions(-DUNALIGNED64_OK)
|
||||
endif()
|
||||
|
||||
set (HAVE_UNISTD_H 1)
|
||||
add_definitions(-D_LARGEFILE64_SOURCE=1 -D__USE_LARGEFILE64)
|
||||
|
@ -213,9 +213,10 @@ If the `WITH TOTALS` modifier is specified, another row will be calculated. This
|
||||
|
||||
This extra row is only produced in `JSON*`, `TabSeparated*`, and `Pretty*` formats, separately from the other rows:
|
||||
|
||||
- In `JSON*` formats, this row is output as a separate ‘totals’ field.
|
||||
- In `TabSeparated*` formats, the row comes after the main result, preceded by an empty row (after the other data).
|
||||
- In `XML` and `JSON*` formats, this row is output as a separate ‘totals’ field.
|
||||
- In `TabSeparated*`, `CSV*` and `Vertical` formats, the row comes after the main result, preceded by an empty row (after the other data).
|
||||
- In `Pretty*` formats, the row is output as a separate table after the main result.
|
||||
- In `Template` format, the row is output according to specified template.
|
||||
- In the other formats it is not available.
|
||||
|
||||
:::note
|
||||
|
@ -135,9 +135,9 @@ In all other cases, we do not recommend using the asterisk, since it only gives
|
||||
|
||||
In addition to results, you can also get minimum and maximum values for the results columns. To do this, set the **extremes** setting to 1. Minimums and maximums are calculated for numeric types, dates, and dates with times. For other columns, the default values are output.
|
||||
|
||||
An extra two rows are calculated – the minimums and maximums, respectively. These extra two rows are output in `JSON*`, `TabSeparated*`, and `Pretty*` [formats](../../../interfaces/formats.md), separate from the other rows. They are not output for other formats.
|
||||
An extra two rows are calculated – the minimums and maximums, respectively. These extra two rows are output in `XML`, `JSON*`, `TabSeparated*`, `CSV*`, `Vertical`, `Template` and `Pretty*` [formats](../../../interfaces/formats.md), separate from the other rows. They are not output for other formats.
|
||||
|
||||
In `JSON*` formats, the extreme values are output in a separate ‘extremes’ field. In `TabSeparated*` formats, the row comes after the main result, and after ‘totals’ if present. It is preceded by an empty row (after the other data). In `Pretty*` formats, the row is output as a separate table after the main result, and after `totals` if present.
|
||||
In `JSON*` and `XML` formats, the extreme values are output in a separate ‘extremes’ field. In `TabSeparated*`, `CSV*` and `Vertical` formats, the row comes after the main result, and after ‘totals’ if present. It is preceded by an empty row (after the other data). In `Pretty*` formats, the row is output as a separate table after the main result, and after `totals` if present. In `Template` format the extreme values are output according to specified template.
|
||||
|
||||
Extreme values are calculated for rows before `LIMIT`, but after `LIMIT BY`. However, when using `LIMIT offset, size`, the rows before `offset` are included in `extremes`. In stream requests, the result may also include a small number of rows that passed through `LIMIT`.
|
||||
|
||||
|
@ -402,6 +402,36 @@ void checkHarmfulEnvironmentVariables(char ** argv)
|
||||
}
|
||||
|
||||
|
||||
/// Don't allow dlopen in the main ClickHouse binary, because it is harmful and insecure.
|
||||
/// We don't use it. But it can be used by some libraries for implementation of "plugins".
|
||||
/// We absolutely discourage the ancient technique of loading
|
||||
/// 3rd-party uncontrolled dangerous libraries into the process address space,
|
||||
/// because it is insane.
|
||||
|
||||
extern "C"
|
||||
{
|
||||
void * dlopen(const char *, int)
|
||||
{
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
void * dlmopen(long, const char *, int) // NOLINT
|
||||
{
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
int dlclose(void *)
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
const char * dlerror()
|
||||
{
|
||||
return "ClickHouse does not allow dynamic library loading";
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/// This allows to implement assert to forbid initialization of a class in static constructors.
|
||||
/// Usage:
|
||||
///
|
||||
|
@ -137,9 +137,41 @@ Field QueryFuzzer::fuzzField(Field field)
|
||||
break;
|
||||
}
|
||||
}
|
||||
else if (type == Field::Types::Array || type == Field::Types::Tuple)
|
||||
else if (type == Field::Types::Array)
|
||||
{
|
||||
auto & arr = field.reinterpret<FieldVector>();
|
||||
auto & arr = field.get<Array>();
|
||||
|
||||
if (fuzz_rand() % 5 == 0 && !arr.empty())
|
||||
{
|
||||
size_t pos = fuzz_rand() % arr.size();
|
||||
arr.erase(arr.begin() + pos);
|
||||
std::cerr << "erased\n";
|
||||
}
|
||||
|
||||
if (fuzz_rand() % 5 == 0)
|
||||
{
|
||||
if (!arr.empty())
|
||||
{
|
||||
size_t pos = fuzz_rand() % arr.size();
|
||||
arr.insert(arr.begin() + pos, fuzzField(arr[pos]));
|
||||
std::cerr << fmt::format("inserted (pos {})\n", pos);
|
||||
}
|
||||
else
|
||||
{
|
||||
arr.insert(arr.begin(), getRandomField(0));
|
||||
std::cerr << "inserted (0)\n";
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
for (auto & element : arr)
|
||||
{
|
||||
element = fuzzField(element);
|
||||
}
|
||||
}
|
||||
else if (type == Field::Types::Tuple)
|
||||
{
|
||||
auto & arr = field.get<Tuple>();
|
||||
|
||||
if (fuzz_rand() % 5 == 0 && !arr.empty())
|
||||
{
|
||||
|
@ -1,4 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Columns/IColumnUnique.h>
|
||||
#include <Columns/IColumnImpl.h>
|
||||
#include <Columns/ReverseIndex.h>
|
||||
@ -7,16 +8,17 @@
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/NumberTraits.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <base/range.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
#include <base/range.h>
|
||||
#include <base/unaligned.h>
|
||||
#include "Columns/ColumnConst.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -305,17 +307,52 @@ size_t ColumnUnique<ColumnType>::getNullValueIndex() const
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
class FieldVisitorGetData : public StaticVisitor<>
|
||||
{
|
||||
public:
|
||||
StringRef res;
|
||||
|
||||
[[noreturn]] static void throwUnsupported()
|
||||
{
|
||||
throw Exception("Unsupported field type", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
[[noreturn]] void operator() (const Null &) { throwUnsupported(); }
|
||||
[[noreturn]] void operator() (const Array &) { throwUnsupported(); }
|
||||
[[noreturn]] void operator() (const Tuple &) { throwUnsupported(); }
|
||||
[[noreturn]] void operator() (const Map &) { throwUnsupported(); }
|
||||
[[noreturn]] void operator() (const Object &) { throwUnsupported(); }
|
||||
[[noreturn]] void operator() (const AggregateFunctionStateData &) { throwUnsupported(); }
|
||||
void operator() (const String & x) { res = {x.data(), x.size()}; }
|
||||
void operator() (const UInt64 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const UInt128 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const UInt256 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const Int64 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const Int128 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const Int256 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const UUID & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const Float64 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const DecimalField<Decimal32> & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const DecimalField<Decimal64> & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const DecimalField<Decimal128> & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const DecimalField<Decimal256> & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const bool & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
template <typename ColumnType>
|
||||
size_t ColumnUnique<ColumnType>::uniqueInsert(const Field & x)
|
||||
{
|
||||
if (x.isNull())
|
||||
return getNullValueIndex();
|
||||
|
||||
if (valuesHaveFixedSize())
|
||||
return uniqueInsertData(&x.reinterpret<char>(), size_of_value_if_fixed);
|
||||
|
||||
const auto & val = x.get<String>();
|
||||
return uniqueInsertData(val.data(), val.size());
|
||||
FieldVisitorGetData visitor;
|
||||
applyVisitor(visitor, x);
|
||||
return uniqueInsertData(visitor.res.data, visitor.res.size);
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
|
@ -94,21 +94,7 @@ public:
|
||||
T operator() (const DecimalField<U> & x) const
|
||||
{
|
||||
if constexpr (std::is_floating_point_v<T>)
|
||||
return x.getValue(). template convertTo<T>() / x.getScaleMultiplier(). template convertTo<T>();
|
||||
else if constexpr (std::is_same_v<T, UInt128>)
|
||||
{
|
||||
if constexpr (sizeof(U) < 16)
|
||||
{
|
||||
return UInt128(0, (x.getValue() / x.getScaleMultiplier()).value);
|
||||
}
|
||||
else if constexpr (sizeof(U) == 16)
|
||||
{
|
||||
auto tmp = (x.getValue() / x.getScaleMultiplier()).value;
|
||||
return UInt128(tmp >> 64, UInt64(tmp));
|
||||
}
|
||||
else
|
||||
throw Exception("No conversion to old UInt128 from " + demangle(typeid(U).name()), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
return x.getValue().template convertTo<T>() / x.getScaleMultiplier().template convertTo<T>();
|
||||
else
|
||||
return (x.getValue() / x.getScaleMultiplier()). template convertTo<T>();
|
||||
}
|
||||
@ -134,4 +120,3 @@ public:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -15,7 +15,7 @@ FieldVisitorSum::FieldVisitorSum(const Field & rhs_) : rhs(rhs_) {}
|
||||
bool FieldVisitorSum::operator() (Int64 & x) const { return this->operator()(reinterpret_cast<UInt64 &>(x)); }
|
||||
bool FieldVisitorSum::operator() (UInt64 & x) const
|
||||
{
|
||||
x += rhs.reinterpret<UInt64>();
|
||||
x += applyVisitor(FieldVisitorConvertToNumber<UInt64>(), rhs);
|
||||
return x != 0;
|
||||
}
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -41,7 +42,7 @@ public:
|
||||
requires is_big_int_v<T>
|
||||
bool operator() (T & x) const
|
||||
{
|
||||
x += rhs.reinterpret<T>();
|
||||
x += applyVisitor(FieldVisitorConvertToNumber<T>(), rhs);
|
||||
return x != T(0);
|
||||
}
|
||||
};
|
||||
|
@ -115,7 +115,13 @@ protected:
|
||||
}
|
||||
|
||||
/// 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 + pad_left; } /// NOLINT
|
||||
static size_t minimum_memory_for_elements(size_t num_elements)
|
||||
{
|
||||
size_t amount;
|
||||
if (__builtin_add_overflow(byte_size(num_elements), pad_left + pad_right, &amount))
|
||||
throw Exception("Amount of memory requested to allocate is more than allowed", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
return amount;
|
||||
}
|
||||
|
||||
void alloc_for_num_elements(size_t num_elements) /// NOLINT
|
||||
{
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/isLocalAddress.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Common/getMultipleKeysFromConfig.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -94,6 +95,14 @@ KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfigur
|
||||
continue;
|
||||
|
||||
std::string full_prefix = config_prefix + ".raft_configuration." + server_key;
|
||||
|
||||
if (getMultipleValuesFromConfig(config, full_prefix, "id").size() > 1
|
||||
|| getMultipleValuesFromConfig(config, full_prefix, "hostname").size() > 1
|
||||
|| getMultipleValuesFromConfig(config, full_prefix, "port").size() > 1)
|
||||
{
|
||||
throw Exception(ErrorCodes::RAFT_ERROR, "Multiple <id> or <hostname> or <port> specified for a single <server>");
|
||||
}
|
||||
|
||||
int new_server_id = config.getInt(full_prefix + ".id");
|
||||
std::string hostname = config.getString(full_prefix + ".hostname");
|
||||
int port = config.getInt(full_prefix + ".port");
|
||||
|
@ -425,16 +425,6 @@ public:
|
||||
bool isNegativeInfinity() const { return which == Types::Null && get<Null>().isNegativeInfinity(); }
|
||||
bool isPositiveInfinity() const { return which == Types::Null && get<Null>().isPositiveInfinity(); }
|
||||
|
||||
template <typename T>
|
||||
T & reinterpret();
|
||||
|
||||
template <typename T>
|
||||
const T & reinterpret() const
|
||||
{
|
||||
auto * mutable_this = const_cast<std::decay_t<decltype(*this)> *>(this);
|
||||
return mutable_this->reinterpret<T>();
|
||||
}
|
||||
|
||||
template <typename T> bool tryGet(T & result)
|
||||
{
|
||||
const Types::Which requested = TypeToEnum<std::decay_t<T>>::value;
|
||||
@ -552,7 +542,7 @@ public:
|
||||
case Types::Float64:
|
||||
{
|
||||
// Compare as UInt64 so that NaNs compare as equal.
|
||||
return reinterpret<UInt64>() == rhs.reinterpret<UInt64>();
|
||||
return std::bit_cast<UInt64>(get<Float64>()) == std::bit_cast<UInt64>(rhs.get<Float64>());
|
||||
}
|
||||
case Types::UUID: return get<UUID>() == rhs.get<UUID>();
|
||||
case Types::String: return get<String>() == rhs.get<String>();
|
||||
@ -843,30 +833,6 @@ auto & Field::safeGet()
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
T & Field::reinterpret()
|
||||
{
|
||||
assert(which != Types::String); // See specialization for char
|
||||
using ValueType = std::decay_t<T>;
|
||||
ValueType * MAY_ALIAS ptr = reinterpret_cast<ValueType *>(&storage);
|
||||
return *ptr;
|
||||
}
|
||||
|
||||
// Specialize reinterpreting to char (used in ColumnUnique) to make sure Strings are reinterpreted correctly
|
||||
// inline to avoid multiple definitions
|
||||
template <>
|
||||
inline char & Field::reinterpret<char>()
|
||||
{
|
||||
if (which == Types::String)
|
||||
{
|
||||
// For String we want to return a pointer to the data, not the start of the class
|
||||
// as the layout of std::string depends on the STD version and options
|
||||
char * ptr = reinterpret_cast<String *>(&storage)->data();
|
||||
return *ptr;
|
||||
}
|
||||
return *reinterpret_cast<char *>(&storage);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
Field::Field(T && rhs, enable_if_not_field_or_bool_or_stringlike_t<T>) //-V730
|
||||
{
|
||||
|
@ -20,8 +20,13 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_READ_ALL_DATA;
|
||||
extern const int CANNOT_READ_ARRAY_FROM_TEXT;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int TOO_LARGE_ARRAY_SIZE;
|
||||
}
|
||||
|
||||
static constexpr size_t MAX_ARRAY_SIZE = 1ULL << 30;
|
||||
static constexpr size_t MAX_ARRAYS_SIZE = 1ULL << 40;
|
||||
|
||||
|
||||
void SerializationArray::serializeBinary(const Field & field, WriteBuffer & ostr) const
|
||||
{
|
||||
const Array & a = field.get<const Array &>();
|
||||
@ -125,7 +130,12 @@ namespace
|
||||
{
|
||||
ColumnArray::Offset current_size = 0;
|
||||
readIntBinary(current_size, istr);
|
||||
current_offset += current_size;
|
||||
|
||||
if (unlikely(current_size > MAX_ARRAY_SIZE))
|
||||
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size is too large: {}", current_size);
|
||||
if (unlikely(__builtin_add_overflow(current_offset, current_size, ¤t_offset)))
|
||||
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Deserialization of array offsets will lead to overflow");
|
||||
|
||||
offset_values[i] = current_offset;
|
||||
++i;
|
||||
}
|
||||
@ -348,6 +358,9 @@ void SerializationArray::deserializeBinaryBulkWithMultipleStreams(
|
||||
throw Exception("Nested column is longer than last offset", ErrorCodes::LOGICAL_ERROR);
|
||||
size_t nested_limit = last_offset - nested_column->size();
|
||||
|
||||
if (unlikely(nested_limit > MAX_ARRAYS_SIZE))
|
||||
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array sizes are too large: {}", nested_limit);
|
||||
|
||||
/// Adjust value size hint. Divide it to the average array size.
|
||||
settings.avg_value_size_hint = nested_limit ? settings.avg_value_size_hint / nested_limit * offset_values.size() : 0;
|
||||
|
||||
|
@ -24,6 +24,8 @@ namespace ErrorCodes
|
||||
extern const int TOO_LARGE_STRING_SIZE;
|
||||
}
|
||||
|
||||
static constexpr size_t MAX_STRINGS_SIZE = 1ULL << 30;
|
||||
|
||||
void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer & ostr) const
|
||||
{
|
||||
const String & s = field.get<const String &>();
|
||||
@ -85,8 +87,17 @@ void SerializationFixedString::deserializeBinaryBulk(IColumn & column, ReadBuffe
|
||||
ColumnFixedString::Chars & data = typeid_cast<ColumnFixedString &>(column).getChars();
|
||||
|
||||
size_t initial_size = data.size();
|
||||
size_t max_bytes = limit * n;
|
||||
data.resize(initial_size + max_bytes);
|
||||
size_t max_bytes;
|
||||
size_t new_data_size;
|
||||
|
||||
if (unlikely(__builtin_mul_overflow(limit, n, &max_bytes)))
|
||||
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Deserializing FixedString will lead to overflow");
|
||||
if (unlikely(max_bytes > MAX_STRINGS_SIZE))
|
||||
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large sizes of FixedString to deserialize: {}", max_bytes);
|
||||
if (unlikely(__builtin_add_overflow(initial_size, max_bytes, &new_data_size)))
|
||||
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Deserializing FixedString will lead to overflow");
|
||||
|
||||
data.resize(new_data_size);
|
||||
size_t read_bytes = istr.readBig(reinterpret_cast<char *>(&data[initial_size]), max_bytes);
|
||||
|
||||
if (read_bytes % n != 0)
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <emmintrin.h>
|
||||
#endif
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -156,6 +156,8 @@ void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std::
|
||||
e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF ||
|
||||
e.code() == ErrorCodes::CANNOT_READ_ALL_DATA)
|
||||
return;
|
||||
|
||||
throw;
|
||||
}
|
||||
catch (const fs::filesystem_error & e)
|
||||
{
|
||||
|
@ -63,9 +63,10 @@ ColumnsDescription readSchemaFromFormat(
|
||||
{
|
||||
names_and_types = external_schema_reader->readSchema();
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
catch (Exception & e)
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file. Error: {}. You can specify the structure manually", format_name, e.message());
|
||||
e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name));
|
||||
throw;
|
||||
}
|
||||
}
|
||||
else if (FormatFactory::instance().checkIfFormatHasSchemaReader(format_name))
|
||||
@ -85,6 +86,12 @@ ColumnsDescription readSchemaFromFormat(
|
||||
break;
|
||||
is_eof = buf->eof();
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage(fmt::format(
|
||||
"Cannot extract table structure from {} format file. You can specify the structure manually", format_name));
|
||||
throw;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
auto exception_message = getCurrentExceptionMessage(false);
|
||||
@ -136,7 +143,21 @@ ColumnsDescription readSchemaFromFormat(
|
||||
}
|
||||
|
||||
if (!retry || !isRetryableSchemaInferenceError(getCurrentExceptionCode()))
|
||||
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file. Error: {}. You can specify the structure manually", format_name, exception_message);
|
||||
{
|
||||
try
|
||||
{
|
||||
throw;
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name));
|
||||
throw;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file. Error: {}. You can specify the structure manually", format_name, exception_message);
|
||||
}
|
||||
}
|
||||
|
||||
exception_messages += "\n" + exception_message;
|
||||
}
|
||||
|
@ -2381,7 +2381,7 @@ using FunctionToDate = FunctionConvert<DataTypeDate, NameToDate, ToDateMonotonic
|
||||
using FunctionToDate32 = FunctionConvert<DataTypeDate32, NameToDate32, ToDateMonotonicity>;
|
||||
using FunctionToDateTime = FunctionConvert<DataTypeDateTime, NameToDateTime, ToDateTimeMonotonicity>;
|
||||
using FunctionToDateTime32 = FunctionConvert<DataTypeDateTime, NameToDateTime32, ToDateTimeMonotonicity>;
|
||||
using FunctionToDateTime64 = FunctionConvert<DataTypeDateTime64, NameToDateTime64, UnknownMonotonicity>;
|
||||
using FunctionToDateTime64 = FunctionConvert<DataTypeDateTime64, NameToDateTime64, ToDateTimeMonotonicity>;
|
||||
using FunctionToUUID = FunctionConvert<DataTypeUUID, NameToUUID, ToNumberMonotonicity<UInt128>>;
|
||||
using FunctionToString = FunctionConvert<DataTypeString, NameToString, ToStringMonotonicity>;
|
||||
using FunctionToUnixTimestamp = FunctionConvert<DataTypeUInt32, NameToUnixTimestamp, ToNumberMonotonicity<UInt32>>;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <mutex>
|
||||
#include <base/bit_cast.h>
|
||||
|
||||
#include <Common/FieldVisitorDump.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
@ -920,8 +921,7 @@ private:
|
||||
ColumnString::Offset current_dst_default_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
Field key = src[i];
|
||||
const auto * it = table.find(key.reinterpret<UInt64>());
|
||||
const auto * it = table.find(bit_cast<UInt64>(src[i]));
|
||||
StringRef ref;
|
||||
|
||||
if (it)
|
||||
@ -1081,6 +1081,22 @@ private:
|
||||
|
||||
mutable Cache cache;
|
||||
|
||||
|
||||
static UInt64 bitCastToUInt64(const Field & x)
|
||||
{
|
||||
switch (x.getType())
|
||||
{
|
||||
case Field::Types::UInt64: return x.get<UInt64>();
|
||||
case Field::Types::Int64: return x.get<Int64>();
|
||||
case Field::Types::Float64: return std::bit_cast<UInt64>(x.get<Float64>());
|
||||
case Field::Types::Bool: return x.get<bool>();
|
||||
case Field::Types::Decimal32: return x.get<DecimalField<Decimal32>>().getValue();
|
||||
case Field::Types::Decimal64: return x.get<DecimalField<Decimal64>>().getValue();
|
||||
default:
|
||||
throw Exception("Unexpected type in function 'transform'", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
}
|
||||
|
||||
/// Can be called from different threads. It works only on the first call.
|
||||
void initialize(const Array & from, const Array & to, const ColumnsWithTypeAndName & arguments) const
|
||||
{
|
||||
@ -1151,20 +1167,8 @@ private:
|
||||
if (key.isNull())
|
||||
continue;
|
||||
|
||||
// Field may be of Float type, but for the purpose of bitwise
|
||||
// equality we can treat them as UInt64, hence the reinterpret().
|
||||
if (to[0].getType() ==Field::Types::Decimal32)
|
||||
{
|
||||
table[key.reinterpret<Decimal32>()] = (*used_to)[i].reinterpret<Decimal32>();
|
||||
}
|
||||
else if (to[0].getType() ==Field::Types::Decimal64)
|
||||
{
|
||||
table[key.reinterpret<Decimal32>()] = (*used_to)[i].reinterpret<Decimal64>();
|
||||
}
|
||||
else
|
||||
{
|
||||
table[key.reinterpret<UInt64>()] = (*used_to)[i].reinterpret<UInt64>();
|
||||
}
|
||||
/// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64
|
||||
table[bitCastToUInt64(key)] = bitCastToUInt64((*used_to)[i]);
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -1179,7 +1183,7 @@ private:
|
||||
|
||||
const String & str_to = to[i].get<const String &>();
|
||||
StringRef ref{cache.string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1};
|
||||
table[key.reinterpret<UInt64>()] = ref;
|
||||
table[bitCastToUInt64(key)] = ref;
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1193,7 +1197,7 @@ private:
|
||||
{
|
||||
const String & str_from = from[i].get<const String &>();
|
||||
StringRef ref{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
|
||||
table[ref] = (*used_to)[i].reinterpret<UInt64>();
|
||||
table[ref] = bitCastToUInt64((*used_to)[i]);
|
||||
}
|
||||
}
|
||||
else
|
||||
|
@ -14,6 +14,7 @@
|
||||
#include <cstdlib>
|
||||
#include <unistd.h>
|
||||
#include <sys/mman.h>
|
||||
#include <dlfcn.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -25,6 +26,7 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int CANNOT_ALLOCATE_MEMORY;
|
||||
extern const int CANNOT_DLOPEN;
|
||||
}
|
||||
|
||||
|
||||
@ -136,7 +138,7 @@ public:
|
||||
}
|
||||
else if (mode == "access context")
|
||||
{
|
||||
(void)context.getCurrentQueryId();
|
||||
(void)context->getCurrentQueryId();
|
||||
}
|
||||
else if (mode == "stack overflow")
|
||||
{
|
||||
@ -166,6 +168,12 @@ public:
|
||||
maps.push_back(map);
|
||||
}
|
||||
}
|
||||
else if (mode == "dlopen")
|
||||
{
|
||||
void * handle = dlopen("libc.so.6", RTLD_NOW);
|
||||
if (!handle)
|
||||
throw Exception(ErrorCodes::CANNOT_DLOPEN, "Cannot dlopen: ({})", dlerror()); // NOLINT(concurrency-mt-unsafe) // MT-Safe on Linux, see man dlerror
|
||||
}
|
||||
else
|
||||
throw Exception("Unknown trap mode", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
@ -6,26 +6,24 @@
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeObject.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
|
||||
#include <Core/AccurateComparison.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Common/DateLUT.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -223,7 +221,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
|
||||
&& (which_from_type.isNativeInt() || which_from_type.isNativeUInt() || which_from_type.isDate() || which_from_type.isDate32() || which_from_type.isDateTime() || which_from_type.isDateTime64()))
|
||||
{
|
||||
const auto scale = static_cast<const DataTypeDateTime64 &>(type).getScale();
|
||||
const auto decimal_value = DecimalUtils::decimalFromComponents<DateTime64>(src.reinterpret<Int64>(), 0, scale);
|
||||
const auto decimal_value = DecimalUtils::decimalFromComponents<DateTime64>(applyVisitor(FieldVisitorConvertToNumber<Int64>(), src), 0, scale);
|
||||
return Field(DecimalField<DateTime64>(decimal_value, scale));
|
||||
}
|
||||
}
|
||||
|
@ -40,6 +40,9 @@ void IRowOutputFormat::consume(DB::Chunk chunk)
|
||||
|
||||
void IRowOutputFormat::consumeTotals(DB::Chunk chunk)
|
||||
{
|
||||
if (!supportTotals())
|
||||
return;
|
||||
|
||||
auto num_rows = chunk.getNumRows();
|
||||
if (num_rows != 1)
|
||||
throw Exception("Got " + toString(num_rows) + " in totals chunk, expected 1", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -53,6 +56,9 @@ void IRowOutputFormat::consumeTotals(DB::Chunk chunk)
|
||||
|
||||
void IRowOutputFormat::consumeExtremes(DB::Chunk chunk)
|
||||
{
|
||||
if (!supportExtremes())
|
||||
return;
|
||||
|
||||
auto num_rows = chunk.getNumRows();
|
||||
const auto & columns = chunk.getColumns();
|
||||
if (num_rows != 2)
|
||||
|
@ -32,6 +32,9 @@ protected:
|
||||
void consumeTotals(Chunk chunk) override;
|
||||
void consumeExtremes(Chunk chunk) override;
|
||||
|
||||
virtual bool supportTotals() const { return false; }
|
||||
virtual bool supportExtremes() const { return false; }
|
||||
|
||||
/** Write a row.
|
||||
* Default implementation calls methods to write single values and delimiters
|
||||
* (except delimiter between rows (writeRowBetweenDelimiter())).
|
||||
|
@ -34,6 +34,10 @@ private:
|
||||
void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override;
|
||||
void writeFieldDelimiter() override;
|
||||
void writeRowEndDelimiter() override;
|
||||
|
||||
bool supportTotals() const override { return true; }
|
||||
bool supportExtremes() const override { return true; }
|
||||
|
||||
void writeBeforeTotals() override;
|
||||
void writeBeforeExtremes() override;
|
||||
|
||||
|
@ -36,9 +36,8 @@ private:
|
||||
void writeRowStartDelimiter() override;
|
||||
void writeRowEndDelimiter() override;
|
||||
|
||||
bool supportTotals() const override { return true; }
|
||||
void consumeTotals(Chunk) override;
|
||||
/// No extremes.
|
||||
void consumeExtremes(Chunk) override {}
|
||||
|
||||
void writeLine(const std::vector<String> & values);
|
||||
|
||||
|
@ -31,6 +31,9 @@ private:
|
||||
void writeRowStartDelimiter() override;
|
||||
void writeRowEndDelimiter() override;
|
||||
|
||||
bool supportTotals() const override { return true; }
|
||||
bool supportExtremes() const override { return true; }
|
||||
|
||||
void writeBeforeTotals() override;
|
||||
void writeAfterTotals() override;
|
||||
|
||||
|
@ -39,10 +39,6 @@ protected:
|
||||
void writePrefix() override;
|
||||
void writeSuffix() override;
|
||||
|
||||
/// No totals and extremes.
|
||||
void consumeTotals(Chunk) override {}
|
||||
void consumeExtremes(Chunk) override {}
|
||||
|
||||
size_t field_number = 0;
|
||||
|
||||
private:
|
||||
|
@ -56,6 +56,9 @@ protected:
|
||||
void writeMaxExtreme(const Columns & columns, size_t row_num) override;
|
||||
void writeTotals(const Columns & columns, size_t row_num) override;
|
||||
|
||||
bool supportTotals() const override { return true; }
|
||||
bool supportExtremes() const override { return true; }
|
||||
|
||||
void writeBeforeTotals() override;
|
||||
void writeAfterTotals() override;
|
||||
void writeBeforeExtremes() override;
|
||||
|
@ -486,13 +486,16 @@ void ORCBlockOutputFormat::consume(Chunk chunk)
|
||||
{
|
||||
if (!writer)
|
||||
prepareWriter();
|
||||
|
||||
size_t columns_num = chunk.getNumColumns();
|
||||
size_t rows_num = chunk.getNumRows();
|
||||
|
||||
/// getMaxColumnSize is needed to write arrays.
|
||||
/// The size of the batch must be no less than total amount of array elements
|
||||
/// and no less than the number of rows (ORC writes a null bit for every row).
|
||||
std::unique_ptr<orc::ColumnVectorBatch> batch = writer->createRowBatch(getMaxColumnSize(chunk));
|
||||
orc::StructVectorBatch & root = dynamic_cast<orc::StructVectorBatch &>(*batch);
|
||||
|
||||
auto columns = chunk.detachColumns();
|
||||
for (auto & column : columns)
|
||||
column = recursiveRemoveLowCardinality(column);
|
||||
|
@ -8,11 +8,13 @@
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <orc/OrcFile.hh>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class WriteBuffer;
|
||||
|
||||
|
||||
/// orc::Writer writes only in orc::OutputStream
|
||||
class ORCOutputStream : public orc::OutputStream
|
||||
{
|
||||
@ -21,7 +23,7 @@ public:
|
||||
|
||||
uint64_t getLength() const override;
|
||||
uint64_t getNaturalWriteSize() const override;
|
||||
void write(const void* buf, size_t length) override;
|
||||
void write(const void * buf, size_t length) override;
|
||||
|
||||
void close() override {}
|
||||
const std::string& getName() const override { return name; }
|
||||
@ -31,6 +33,7 @@ private:
|
||||
std::string name = "ORCOutputStream";
|
||||
};
|
||||
|
||||
|
||||
class ORCBlockOutputFormat : public IOutputFormat
|
||||
{
|
||||
public:
|
||||
|
@ -22,6 +22,10 @@ private:
|
||||
void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override;
|
||||
void writeRowEndDelimiter() override;
|
||||
|
||||
/// Disable totals and extremes, because they are enabled in TSV.
|
||||
bool supportTotals() const override { return false; }
|
||||
bool supportExtremes() const override { return false; }
|
||||
|
||||
NamesAndTypes fields;
|
||||
size_t field_number = 0;
|
||||
};
|
||||
|
@ -37,6 +37,10 @@ protected:
|
||||
void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override;
|
||||
void writeFieldDelimiter() override final;
|
||||
void writeRowEndDelimiter() override;
|
||||
|
||||
bool supportTotals() const override { return true; }
|
||||
bool supportExtremes() const override { return true; }
|
||||
|
||||
void writeBeforeTotals() override final;
|
||||
void writeBeforeExtremes() override final;
|
||||
|
||||
|
@ -32,6 +32,9 @@ private:
|
||||
void writeMaxExtreme(const Columns & columns, size_t row_num) override;
|
||||
void writeTotals(const Columns & columns, size_t row_num) override;
|
||||
|
||||
bool supportTotals() const override { return true; }
|
||||
bool supportExtremes() const override { return true; }
|
||||
|
||||
void writeBeforeTotals() override;
|
||||
void writeBeforeExtremes() override;
|
||||
|
||||
|
@ -32,6 +32,9 @@ private:
|
||||
void writeMaxExtreme(const Columns & columns, size_t row_num) override;
|
||||
void writeTotals(const Columns & columns, size_t row_num) override;
|
||||
|
||||
bool supportTotals() const override { return true; }
|
||||
bool supportExtremes() const override { return true; }
|
||||
|
||||
void writeBeforeTotals() override;
|
||||
void writeAfterTotals() override;
|
||||
void writeBeforeExtremes() override;
|
||||
|
@ -223,6 +223,27 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare()
|
||||
.part_log_writer = {}
|
||||
};
|
||||
}
|
||||
else if (!storage.findReplicaHavingCoveringPart(entry.new_part_name, /* active */ false, dummy).empty())
|
||||
{
|
||||
/// Why this if still needed? We can check for part in zookeeper, don't find it and sleep for any amount of time. During this sleep part will be actually committed from other replica
|
||||
/// and exclusive zero copy lock will be released. We will take the lock and execute merge one more time, while it was possible just to download the part from other replica.
|
||||
///
|
||||
/// It's also possible just because reads in [Zoo]Keeper are not lineariazable.
|
||||
///
|
||||
/// NOTE: In case of mutation and hardlinks it can even lead to extremely rare dataloss (we will produce new part with the same hardlinks, don't fetch the same from other replica), so this check is important.
|
||||
zero_copy_lock->lock->unlock();
|
||||
|
||||
LOG_DEBUG(log, "We took zero copy lock, but merge of part {} finished by some other replica, will release lock and download merged part to avoid data duplication", entry.new_part_name);
|
||||
return PrepareResult{
|
||||
.prepared_successfully = false,
|
||||
.need_to_check_missing_part_in_fetch = true,
|
||||
.part_log_writer = {}
|
||||
};
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_DEBUG(log, "Zero copy lock taken, will merge part {}", entry.new_part_name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -242,11 +242,21 @@ bool MergeTreeIndexConditionBloomFilter::traverseAtomAST(const ASTPtr & node, Bl
|
||||
DataTypePtr const_type;
|
||||
if (KeyCondition::getConstant(node, block_with_constants, const_value, const_type))
|
||||
{
|
||||
if (const_value.getType() == Field::Types::UInt64 || const_value.getType() == Field::Types::Int64 ||
|
||||
const_value.getType() == Field::Types::Float64)
|
||||
if (const_value.getType() == Field::Types::UInt64)
|
||||
{
|
||||
/// Zero in all types is represented in memory the same way as in UInt64.
|
||||
out.function = const_value.reinterpret<UInt64>() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE;
|
||||
out.function = const_value.get<UInt64>() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE;
|
||||
return true;
|
||||
}
|
||||
|
||||
if (const_value.getType() == Field::Types::Int64)
|
||||
{
|
||||
out.function = const_value.get<Int64>() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE;
|
||||
return true;
|
||||
}
|
||||
|
||||
if (const_value.getType() == Field::Types::Float64)
|
||||
{
|
||||
out.function = const_value.get<Float64>() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
@ -134,6 +134,29 @@ ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare()
|
||||
.part_log_writer = {}
|
||||
};
|
||||
}
|
||||
else if (!storage.findReplicaHavingCoveringPart(entry.new_part_name, /* active */ false, dummy).empty())
|
||||
{
|
||||
/// Why this if still needed? We can check for part in zookeeper, don't find it and sleep for any amount of time. During this sleep part will be actually committed from other replica
|
||||
/// and exclusive zero copy lock will be released. We will take the lock and execute mutation one more time, while it was possible just to download the part from other replica.
|
||||
///
|
||||
/// It's also possible just because reads in [Zoo]Keeper are not lineariazable.
|
||||
///
|
||||
/// NOTE: In case of mutation and hardlinks it can even lead to extremely rare dataloss (we will produce new part with the same hardlinks, don't fetch the same from other replica), so this check is important.
|
||||
///
|
||||
/// In case of DROP_RANGE on fast replica and stale replica we can have some failed select queries in case of zero copy replication.
|
||||
zero_copy_lock->lock->unlock();
|
||||
|
||||
LOG_DEBUG(log, "We took zero copy lock, but mutation of part {} finished by some other replica, will release lock and download mutated part to avoid data duplication", entry.new_part_name);
|
||||
return PrepareResult{
|
||||
.prepared_successfully = false,
|
||||
.need_to_check_missing_part_in_fetch = true,
|
||||
.part_log_writer = {}
|
||||
};
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_DEBUG(log, "Zero copy lock taken, will mutate part {}", entry.new_part_name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeQueue.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeLogEntry.h>
|
||||
#include <Storages/MergeTree/ZeroCopyLock.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -18,7 +19,12 @@ public:
|
||||
ReplicatedMergeTreeQueue::SelectedEntryPtr selected_entry_,
|
||||
StorageReplicatedMergeTree & storage_,
|
||||
Callback && task_result_callback_)
|
||||
: ReplicatedMergeMutateTaskBase(&Poco::Logger::get("MutateFromLogEntryTask"), storage_, selected_entry_, task_result_callback_) {}
|
||||
: ReplicatedMergeMutateTaskBase(
|
||||
&Poco::Logger::get(storage_.getStorageID().getShortName() + "::" + selected_entry_->log_entry->new_part_name + "(MutateFromLogEntryTask)"),
|
||||
storage_,
|
||||
selected_entry_,
|
||||
task_result_callback_)
|
||||
{}
|
||||
|
||||
|
||||
UInt64 getPriority() override { return priority; }
|
||||
|
@ -172,6 +172,37 @@ NORMAL_CONFIG = """
|
||||
</clickhouse>
|
||||
"""
|
||||
|
||||
JUST_WRONG_CONFIG = """
|
||||
<clickhouse>
|
||||
<keeper_server>
|
||||
<tcp_port>9181</tcp_port>
|
||||
<server_id>1</server_id>
|
||||
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
|
||||
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
|
||||
|
||||
<coordination_settings>
|
||||
<operation_timeout_ms>5000</operation_timeout_ms>
|
||||
<session_timeout_ms>10000</session_timeout_ms>
|
||||
<raft_logs_level>trace</raft_logs_level>
|
||||
</coordination_settings>
|
||||
|
||||
<raft_configuration>
|
||||
<server>
|
||||
<id>1</id>
|
||||
<hostname>node1</hostname>
|
||||
<port>9234</port>
|
||||
<id>2</id>
|
||||
<hostname>node2</hostname>
|
||||
<port>9234</port>
|
||||
<id>3</id>
|
||||
<hostname>node3</hostname>
|
||||
<port>9234</port>
|
||||
</server>
|
||||
</raft_configuration>
|
||||
</keeper_server>
|
||||
</clickhouse>
|
||||
"""
|
||||
|
||||
|
||||
def test_duplicate_endpoint(started_cluster):
|
||||
node1.stop_clickhouse()
|
||||
@ -187,6 +218,7 @@ def test_duplicate_endpoint(started_cluster):
|
||||
assert_config_fails(DUPLICATE_ID_CONFIG)
|
||||
assert_config_fails(LOCALHOST_WITH_REMOTE)
|
||||
assert_config_fails(MULTIPLE_LOCAL_WITH_REMOTE)
|
||||
assert_config_fails(JUST_WRONG_CONFIG)
|
||||
|
||||
node1.replace_config(
|
||||
"/etc/clickhouse-server/config.d/enable_keeper1.xml", NORMAL_CONFIG
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: long
|
||||
-- Tags: long, no-s3-storage, no-asan
|
||||
|
||||
SET joined_subquery_requires_alias = 0;
|
||||
|
||||
|
@ -8,5 +8,4 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
|
||||
cp $CUR_DIR/data_orc/corrupted.orc $USER_FILES_PATH/
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="select * from file('corrupted.orc')" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="select * from file('corrupted.orc')" 2>&1 | grep -F -q 'Cannot extract table structure' && echo 'OK' || echo 'FAIL'
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest
|
||||
# Tags: no-fasttest, no-parallel
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
@ -12,7 +12,6 @@ DATA_FILE=$USER_FILES_PATH/$FILE_NAME
|
||||
cp $CUR_DIR/data_parquet_bad_column/metadata_0.parquet $DATA_FILE
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "desc file(test_02245.parquet)" 2>&1 | grep -qF "CANNOT_EXTRACT_TABLE_STRUCTURE" && echo "OK" || echo "FAIL"
|
||||
$CLICKHOUSE_CLIENT -q "desc file(test_02245.parquet)" 2>&1 | grep -qF "Cannot extract table structure" && echo "OK" || echo "FAIL"
|
||||
$CLICKHOUSE_CLIENT -q "desc file(test_02245.parquet) settings input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference=1"
|
||||
$CLICKHOUSE_CLIENT -q "select count(*) from file(test_02245.parquet) settings input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference=1"
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
-- Tags: no-fasttest, no-parallel
|
||||
|
||||
insert into function file('02267_data2.jsonl') select NULL as x;
|
||||
insert into function file('02267_data3.jsonl') select * from numbers(0);
|
||||
insert into function file('02267_data4.jsonl') select 1 as x;
|
||||
@ -7,4 +8,4 @@ select * from file('02267_data*.jsonl') order by x;
|
||||
insert into function file('02267_data1.jsonl', 'TSV') select 1 as x;
|
||||
insert into function file('02267_data1.jsonl', 'TSV') select [1,2,3] as x;
|
||||
|
||||
select * from file('02267_data*.jsonl') settings schema_inference_use_cache_for_file=0; --{serverError CANNOT_EXTRACT_TABLE_STRUCTURE}
|
||||
select * from file('02267_data*.jsonl') settings schema_inference_use_cache_for_file=0; --{serverError INCORRECT_DATA}
|
||||
|
@ -1,6 +1,7 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
insert into function file('02268_data.jsonl', 'TSV') select 1;
|
||||
select * from file('02268_data.jsonl'); --{serverError CANNOT_EXTRACT_TABLE_STRUCTURE}
|
||||
select * from file('02268_data.jsonl'); --{serverError 117}
|
||||
|
||||
insert into function file('02268_data.jsonCompactEachRow', 'TSV') select 1;
|
||||
select * from file('02268_data.jsonCompactEachRow'); --{serverError CANNOT_EXTRACT_TABLE_STRUCTURE}
|
||||
|
||||
select * from file('02268_data.jsonCompactEachRow'); --{serverError 117}
|
||||
|
@ -23,7 +23,7 @@ $CLICKHOUSE_CLIENT -q "desc file(dump1.sql, MySQLDump) settings input_format_mys
|
||||
$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test'"
|
||||
$CLICKHOUSE_CLIENT -q "desc file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'"
|
||||
$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test2'"
|
||||
$CLICKHOUSE_CLIENT -q "desc file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'
|
||||
$CLICKHOUSE_CLIENT -q "desc file(dump1.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'" 2>&1 | grep -F -q 'Cannot extract table structure' && echo 'OK' || echo 'FAIL'
|
||||
$CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32)') settings input_format_mysql_dump_table_name='test 3'" 2>&1 | grep -F -q 'EMPTY_DATA_PASSED' && echo 'OK' || echo 'FAIL'
|
||||
|
||||
echo "dump2"
|
||||
@ -146,4 +146,3 @@ $CLICKHOUSE_CLIENT -q "desc file(dump15.sql, MySQLDump) settings input_format_my
|
||||
$CLICKHOUSE_CLIENT -q "select * from file(dump15.sql, MySQLDump) settings input_format_mysql_dump_table_name='test 3'"
|
||||
|
||||
rm $USER_FILES_PATH/dump*.sql
|
||||
|
||||
|
@ -88,6 +88,4 @@ echo '
|
||||
}
|
||||
' > $DATA_FILE
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "desc file(data_02293, JSONColumns) settings input_format_max_rows_to_read_for_schema_inference=3" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL'
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "desc file(data_02293, JSONColumns) settings input_format_max_rows_to_read_for_schema_inference=3" 2>&1 | grep -F -q 'Cannot extract table structure' && echo 'OK' || echo 'FAIL'
|
||||
|
@ -15,11 +15,11 @@ mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR
|
||||
cp -r $CLIENT_SCHEMADIR/02327_* $SCHEMADIR/$SERVER_SCHEMADIR/
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="desc file(data.pb) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL';
|
||||
$CLICKHOUSE_CLIENT --query="desc file(data.capnp) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL';
|
||||
$CLICKHOUSE_CLIENT --query="desc file(data.pb) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'Cannot extract table structure' && echo 'OK' || echo 'FAIL';
|
||||
$CLICKHOUSE_CLIENT --query="desc file(data.capnp) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'Cannot extract table structure' && echo 'OK' || echo 'FAIL';
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="create table test_protobuf engine=File(Protobuf) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL';
|
||||
$CLICKHOUSE_CLIENT --query="create table test_capnp engine=File(CapnProto) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL';
|
||||
$CLICKHOUSE_CLIENT --query="create table test_protobuf engine=File(Protobuf) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'Cannot extract table structure' && echo 'OK' || echo 'FAIL';
|
||||
$CLICKHOUSE_CLIENT --query="create table test_capnp engine=File(CapnProto) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty'" 2>&1 | grep -F -q 'Cannot extract table structure' && echo 'OK' || echo 'FAIL';
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="desc file(data.pb) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty', input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference=1";
|
||||
$CLICKHOUSE_CLIENT --query="desc file(data.capnp) settings format_schema='$SERVER_SCHEMADIR/02327_schema:MessageWithEmpty', input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference=1";
|
||||
|
@ -0,0 +1,3 @@
|
||||
10000000
|
||||
8358400
|
||||
8358400
|
14
tests/queries/0_stateless/02373_datetime64_monotonicity.sql
Normal file
14
tests/queries/0_stateless/02373_datetime64_monotonicity.sql
Normal file
@ -0,0 +1,14 @@
|
||||
drop table if exists dt64_monot_test sync;
|
||||
|
||||
create table dt64_monot_test(date_time DateTime64(3), id String)
|
||||
Engine=MergeTree
|
||||
partition by toDate(date_time)
|
||||
order by date_time;
|
||||
|
||||
insert into dt64_monot_test select toDateTime64('2020-01-01 00:00:00.000',3)+number , '' from numbers(10000000);
|
||||
|
||||
SELECT count() FROM dt64_monot_test;
|
||||
SELECT count() FROM dt64_monot_test WHERE date_time >= toDateTime64('2020-01-20 00:00:00.000',3);
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-20 00:00:00.000',3);
|
||||
|
||||
drop table if exists dt64_monot_test sync;
|
@ -0,0 +1,57 @@
|
||||
drop table if exists dt64_monot_test;
|
||||
drop table if exists dt64_monot_test_string;
|
||||
CREATE TABLE dt64_monot_test(`date_time` DateTime64(3, 'Europe/Berlin'), `id` String) ENGINE = MergeTree PARTITION BY toDate(date_time, 'Europe/Berlin') ORDER BY date_time;
|
||||
insert into dt64_monot_test select toDateTime64('2020-01-01 00:00:00.000',3)+number , '' from numbers(10);
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime(date_time) >= toDateTime('2020-01-01 00:00:00') SETTINGS force_index_by_date = 1, force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time, 3) >= '2020-01-01 00:00:01.111' SETTINGS force_index_by_date = 1, force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time, 3) >= '2020-01-01 00:00:00.000' SETTINGS force_index_by_date = 1, force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001', 3) SETTINGS force_index_by_date = 1, force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001', 3, 'Europe/Berlin') SETTINGS force_index_by_date = 1, force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001',6) SETTINGS force_index_by_date = 1; -- { serverError 277}
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001',6, 'Europe/Berlin') SETTINGS force_primary_key = 1; -- { serverError 277}
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001',6) SETTINGS force_primary_key = 1; -- { serverError 277}
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) <= toDateTime64('2020-01-01 00:00:00.000001',3, 'Europe/Berlin') settings force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) <= toDateTime64('2020-01-01 00:00:00.000001',3) settings force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) = toDateTime64('2020-01-01 00:00:00.000000',6);
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) = toDateTime64('2020-01-01 00:00:00.000000',6, 'Europe/Berlin');
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) = toDateTime64('2020-01-01 00:00:00.000000',6) settings force_index_by_date = 1, force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) = toDateTime64('2020-01-01 00:00:00.000001',6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) > toDateTime64('2020-01-01 00:00:00.000001',6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) >= toDateTime64('2020-01-01 00:00:00.000001',6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) >= toDateTime64('2020-01-01 00:00:00.000001',6) settings force_index_by_date = 1, force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= toDateTime64('2020-01-01 00:00:00.000001',0, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= toDateTime64('2020-01-01 00:00:00.000001',0) settings force_index_by_date = 1, force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= '2020-01-01 00:00:00' settings force_index_by_date = 1, force_primary_key = 1;
|
||||
|
||||
SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= '2020-01-01 00:00:01.1' settings force_index_by_date = 1, force_primary_key = 1;
|
||||
|
||||
create table dt64_monot_test_string(date_time String, x String) Engine=MergeTree order by date_time;
|
||||
insert into dt64_monot_test_string select '2020-01-01 00:00', '' from numbers(1);
|
||||
insert into dt64_monot_test_string select '2020-01-01 00:00:00.000000' , '' from numbers(10);
|
||||
|
||||
SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,3) = '1970-01-01 00:00:00.000000000';
|
||||
SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,3) = '1970-01-01 00:00:00.000000001';
|
||||
SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,9) = '2020-01-01 00:00:00';
|
||||
|
||||
drop table dt64_monot_test;
|
||||
drop table dt64_monot_test_string;
|
@ -0,0 +1,92 @@
|
||||
Asia/Tehran
|
||||
10
|
||||
0
|
||||
0
|
||||
10
|
||||
0
|
||||
10
|
||||
1
|
||||
1
|
||||
0
|
||||
1
|
||||
0
|
||||
0
|
||||
0
|
||||
9
|
||||
0
|
||||
10
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
10
|
||||
|
||||
UTC
|
||||
10
|
||||
10
|
||||
10
|
||||
10
|
||||
10
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
1
|
||||
0
|
||||
10
|
||||
10
|
||||
9
|
||||
10
|
||||
10
|
||||
10
|
||||
10
|
||||
1
|
||||
1
|
||||
10
|
||||
|
||||
Canada/Atlantic
|
||||
10
|
||||
10
|
||||
10
|
||||
10
|
||||
10
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
1
|
||||
0
|
||||
10
|
||||
10
|
||||
9
|
||||
10
|
||||
10
|
||||
10
|
||||
10
|
||||
0
|
||||
0
|
||||
10
|
||||
|
||||
Europe/Berlin
|
||||
10
|
||||
8
|
||||
10
|
||||
10
|
||||
10
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
9
|
||||
9
|
||||
9
|
||||
10
|
||||
10
|
||||
10
|
||||
9
|
||||
0
|
||||
0
|
||||
10
|
||||
|
12
tests/queries/0_stateless/02373_datetime64_monotonicity1.sh
Executable file
12
tests/queries/0_stateless/02373_datetime64_monotonicity1.sh
Executable file
@ -0,0 +1,12 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
for tz in Asia/Tehran UTC Canada/Atlantic Europe/Berlin
|
||||
do
|
||||
echo "$tz"
|
||||
TZ=$tz $CLICKHOUSE_LOCAL -mn < ${CUR_DIR}/02373_datetime64_monotonicity1.queries
|
||||
echo ""
|
||||
done
|
Binary file not shown.
@ -0,0 +1,22 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
set output_format_write_statistics=0;
|
||||
|
||||
{% for format in ['CSV', 'TSV', 'XML', 'Vertical', 'Pretty', 'JSON', 'JSONCompact'] -%}
|
||||
|
||||
select '{{ format }}';
|
||||
select sum(number) from numbers(10) group by number % 2 with totals format {{ format }} settings extremes=1;
|
||||
select '';
|
||||
|
||||
{% endfor -%}
|
||||
|
||||
select 'Formats without totals and extremes:';
|
||||
|
||||
{% for format in ['CustomSeparated', 'JSONEachRow', 'JSONCompactEachRow', 'RowBinary', 'MsgPack', 'Markdown', 'SQLInsert', 'Values', 'TSKV'] -%}
|
||||
|
||||
select '{{ format }}';
|
||||
select sum(number) from numbers(10) group by number % 2 with totals format {{ format }} settings extremes=1;
|
||||
select '';
|
||||
|
||||
{% endfor -%}
|
||||
|
@ -0,0 +1,12 @@
|
||||
read_rows=1 read_bytes=8 written_rows=1 written_bytes=8 query=INSERT into target_1 FORMAT CSV
|
||||
read_rows=10 read_bytes=80 written_rows=10 written_bytes=80 query=INSERT INTO target_1 FORMAT Native\n
|
||||
read_rows=10 read_bytes=80 written_rows=10 written_bytes=80 query=INSERT INTO target_1 FORMAT RowBinary\n
|
||||
read_rows=5 read_bytes=40 written_rows=4 written_bytes=32 query=INSERT into floats FORMAT CSV
|
||||
read_rows=32 read_bytes=256 written_rows=40 written_bytes=320 query=INSERT INTO floats FORMAT Native\n
|
||||
read_rows=32 read_bytes=256 written_rows=40 written_bytes=320 query=INSERT INTO floats FORMAT RowBinary\n
|
||||
read_rows=1 read_bytes=8 written_rows=1 written_bytes=8 source_query=INSERT into floats FORMAT CSV view_query=SELECT * FROM default.floats
|
||||
read_rows=3 read_bytes=24 written_rows=2 written_bytes=16 source_query=INSERT into floats FORMAT CSV view_query=SELECT * FROM default.floats, numbers(2) AS n
|
||||
read_rows=10 read_bytes=80 written_rows=10 written_bytes=80 source_query=INSERT INTO floats FORMAT Native\n view_query=SELECT * FROM default.floats
|
||||
read_rows=12 read_bytes=96 written_rows=20 written_bytes=160 source_query=INSERT INTO floats FORMAT Native\n view_query=SELECT * FROM default.floats, numbers(2) AS n
|
||||
read_rows=10 read_bytes=80 written_rows=10 written_bytes=80 source_query=INSERT INTO floats FORMAT RowBinary\n view_query=SELECT * FROM default.floats
|
||||
read_rows=12 read_bytes=96 written_rows=20 written_bytes=160 source_query=INSERT INTO floats FORMAT RowBinary\n view_query=SELECT * FROM default.floats, numbers(2) AS n
|
64
tests/queries/0_stateless/02423_insert_stats_behaviour.sh
Executable file
64
tests/queries/0_stateless/02423_insert_stats_behaviour.sh
Executable file
@ -0,0 +1,64 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE floats (v Float64) Engine=MergeTree() ORDER BY tuple();"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE target_1 (v Float64) Engine=MergeTree() ORDER BY tuple();"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE target_2 (v Float64) Engine=MergeTree() ORDER BY tuple();"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW floats_to_target TO target_1 AS SELECT * FROM floats"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW floats_to_target_2 TO target_2 AS SELECT * FROM floats, numbers(2) n"
|
||||
|
||||
# Insertions into table without MVs
|
||||
$CLICKHOUSE_CLIENT -q "INSERT into target_1 FORMAT CSV 1.0"
|
||||
$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format Native | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT+INTO+target_1+FORMAT+Native" --data-binary @-
|
||||
$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format RowBinary | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT+INTO+target_1+FORMAT+RowBinary" --data-binary @-
|
||||
|
||||
# Insertions into table without 2 MVs (1:1 and 1:2 rows)
|
||||
$CLICKHOUSE_CLIENT -q "INSERT into floats FORMAT CSV 1.0"
|
||||
$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format Native | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT+INTO+floats+FORMAT+Native" --data-binary @-
|
||||
$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format RowBinary | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=INSERT+INTO+floats+FORMAT+RowBinary" --data-binary @-
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS"
|
||||
$CLICKHOUSE_CLIENT -q \
|
||||
"SELECT
|
||||
read_rows,
|
||||
read_bytes,
|
||||
written_rows,
|
||||
written_bytes,
|
||||
query
|
||||
FROM system.query_log
|
||||
WHERE
|
||||
event_date >= yesterday()
|
||||
AND event_time > now() - INTERVAL 600 SECOND
|
||||
AND type = 'QueryFinish'
|
||||
AND query_kind = 'Insert'
|
||||
AND current_database == currentDatabase()
|
||||
ORDER BY event_time_microseconds ASC
|
||||
FORMAT TSKV"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q \
|
||||
"SELECT
|
||||
read_rows,
|
||||
read_bytes,
|
||||
written_rows,
|
||||
written_bytes,
|
||||
ql.query as source_query,
|
||||
view_query
|
||||
FROM system.query_views_log
|
||||
INNER JOIN
|
||||
(
|
||||
SELECT
|
||||
query_id, query, event_time_microseconds
|
||||
FROM system.query_log
|
||||
WHERE
|
||||
event_date >= yesterday()
|
||||
AND event_time > now() - INTERVAL 600 SECOND
|
||||
AND type = 'QueryFinish'
|
||||
AND query_kind = 'Insert'
|
||||
AND current_database == currentDatabase()
|
||||
) ql
|
||||
ON system.query_views_log.initial_query_id = ql.query_id
|
||||
ORDER BY ql.event_time_microseconds ASC, view_query ASC
|
||||
FORMAT TSKV"
|
1
tests/queries/0_stateless/02424_pod_array_overflow.sql
Normal file
1
tests/queries/0_stateless/02424_pod_array_overflow.sql
Normal file
@ -0,0 +1 @@
|
||||
SELECT * FROM format(Native, '\x02\x02\x02\x6b\x30\x1a\x4d\x61\x70\x28\x46\x69\x78\x65\x64\x53\x74\x72\x69\x6e\x67\x28\x31\x29\x2c\x20\x49\x6e\x74\x36\x34\x29\x01\x00\x00\x00\x00\x00\x00\x00\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\x7f\x00\x7f\xff\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x64\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\xcf\x31\x3f\x56\x69\x11\x89\x25'); -- { serverError 128 }
|
@ -0,0 +1 @@
|
||||
4908278
|
@ -0,0 +1,8 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_LOCAL} --structure 'x LowCardinality(FixedString(2454139))' --input-format Values --output-format TSV --query "SELECT * FROM table" <<< '(1)' | wc -c
|
1
tests/queries/0_stateless/02426_pod_array_overflow_2.sql
Normal file
1
tests/queries/0_stateless/02426_pod_array_overflow_2.sql
Normal file
@ -0,0 +1 @@
|
||||
SELECT * FROM format(Native, 'k0\x23Array(Tuple(FixedString(1), Int64))\0\0\0\0\0\0\0<><30><EFBFBD><EFBFBD><EFBFBD>\0<><30><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>\0<30>\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0d\0\0\0\0\0\0\0\0\0\0\0\0\0<>1?Vi<11>%'); -- { serverError 128 }
|
1
tests/queries/0_stateless/02426_pod_array_overflow_3.sql
Normal file
1
tests/queries/0_stateless/02426_pod_array_overflow_3.sql
Normal file
@ -0,0 +1 @@
|
||||
SELECT * FROM format(Native, '\x01\x01\x01x\x0CArray(UInt8)\x01\x00\xBD\xEF\xBF\xBD\xEF\xBF\xBD\xEF'); -- { serverError 128 }
|
@ -0,0 +1,10 @@
|
||||
0 999999 999999
|
||||
0 999998 999998
|
||||
0 999997 999997
|
||||
0 999996 999996
|
||||
0 999995 999995
|
||||
0 999994 999994
|
||||
0 999993 999993
|
||||
0 999992 999992
|
||||
0 999991 999991
|
||||
0 999990 999990
|
@ -0,0 +1 @@
|
||||
SELECT * FROM (SELECT * FROM (SELECT 0 AS a, toNullable(number) AS b, toString(number) AS c FROM numbers(1000000.)) ORDER BY a DESC, b DESC, c ASC LIMIT 1500) LIMIT 10;
|
File diff suppressed because one or more lines are too long
@ -0,0 +1 @@
|
||||
SELECT arrayMap(x -> finalizeAggregation(x), state) FROM (SELECT groupArrayResample(9223372036854775806, 1048575, 65537)(number, number % 3), groupArrayStateResample(10, 2147483648, 65535)(number, number % 9223372036854775806) AS state FROM numbers(100));
|
Loading…
Reference in New Issue
Block a user