mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge branch 'master' into consistent_metadata4
This commit is contained in:
commit
bc9a3c0458
@ -220,7 +220,7 @@ if __name__ == '__main__':
|
||||
arg_parser.add_argument('--website-dir', default=website_dir)
|
||||
arg_parser.add_argument('--output-dir', default='build')
|
||||
arg_parser.add_argument('--enable-stable-releases', action='store_true')
|
||||
arg_parser.add_argument('--stable-releases-limit', type=int, default='4')
|
||||
arg_parser.add_argument('--stable-releases-limit', type=int, default='3')
|
||||
arg_parser.add_argument('--lts-releases-limit', type=int, default='2')
|
||||
arg_parser.add_argument('--nav-limit', type=int, default='0')
|
||||
arg_parser.add_argument('--version-prefix', type=str, default='')
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include <Common/ConcurrentBoundedQueue.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Common/clearPasswordFromCommandLine.h>
|
||||
#include <Core/Types.h>
|
||||
#include <IO/ReadBufferFromFileDescriptor.h>
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
@ -539,7 +540,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
|
||||
("password", value<std::string>()->default_value(""), "")
|
||||
("database", value<std::string>()->default_value("default"), "")
|
||||
("stacktrace", "print stack traces of exceptions")
|
||||
("confidence", value<size_t>()->default_value(5), "set the level of confidence for T-test [0=80%, 1=90%, 2=95%, 3=98%, 4=99%, 5=99.5%(default)")
|
||||
("confidence", value<size_t>()->default_value(5), "set the level of confidence for T-test [0=80%, 1=90%, 2=95%, 3=98%, 4=99%, 5=99.5%(default)")
|
||||
("query_id", value<std::string>()->default_value(""), "")
|
||||
;
|
||||
|
||||
@ -550,6 +551,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
|
||||
boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options);
|
||||
boost::program_options::notify(options);
|
||||
|
||||
clearPasswordFromCommandLine(argc, argv);
|
||||
|
||||
if (options.count("help"))
|
||||
{
|
||||
std::cout << "Usage: " << argv[0] << " [options] < queries.txt\n";
|
||||
|
@ -38,6 +38,7 @@
|
||||
#include <Common/Throttler.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/clearPasswordFromCommandLine.h>
|
||||
#include <Common/Config/ConfigProcessor.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
@ -2006,6 +2007,7 @@ public:
|
||||
|
||||
argsToConfig(common_arguments, config(), 100);
|
||||
|
||||
clearPasswordFromCommandLine(argc, argv);
|
||||
}
|
||||
};
|
||||
|
||||
|
7
programs/server/users.d/access_management.xml
Normal file
7
programs/server/users.d/access_management.xml
Normal file
@ -0,0 +1,7 @@
|
||||
<yandex>
|
||||
<users>
|
||||
<default>
|
||||
<access_management>1</access_management>
|
||||
</default>
|
||||
</users>
|
||||
</yandex>
|
@ -67,6 +67,12 @@ public:
|
||||
{
|
||||
data(place).count = new_count;
|
||||
}
|
||||
|
||||
/// The function returns non-Nullable type even when wrapped with Null combinator.
|
||||
bool returnDefaultWhenOnlyNull() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
@ -63,14 +63,15 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
|
||||
{
|
||||
auto type_without_low_cardinality = convertLowCardinalityTypesToNested(argument_types);
|
||||
|
||||
/// If one of types is Nullable, we apply aggregate function combinator "Null".
|
||||
/// If one of the types is Nullable, we apply aggregate function combinator "Null".
|
||||
|
||||
if (std::any_of(type_without_low_cardinality.begin(), type_without_low_cardinality.end(),
|
||||
[](const auto & type) { return type->isNullable(); }))
|
||||
{
|
||||
AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix("Null");
|
||||
if (!combinator)
|
||||
throw Exception("Logical error: cannot find aggregate function combinator to apply a function to Nullable arguments.", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Logical error: cannot find aggregate function combinator to apply a function to Nullable arguments.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
DataTypes nested_types = combinator->transformArguments(type_without_low_cardinality);
|
||||
Array nested_parameters = combinator->transformParameters(parameters);
|
||||
@ -132,9 +133,10 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl(
|
||||
|
||||
auto hints = this->getHints(name);
|
||||
if (!hints.empty())
|
||||
throw Exception("Unknown aggregate function " + name + ". Maybe you meant: " + toString(hints), ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION);
|
||||
throw Exception(fmt::format("Unknown aggregate function {}. Maybe you meant: {}", name, toString(hints)),
|
||||
ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION);
|
||||
else
|
||||
throw Exception("Unknown aggregate function " + name, ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION);
|
||||
throw Exception(fmt::format("Unknown aggregate function {}", name), ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION);
|
||||
}
|
||||
|
||||
|
||||
|
@ -33,6 +33,11 @@ public:
|
||||
AggregateFunctionPtr transformAggregateFunction(
|
||||
const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params) const override
|
||||
{
|
||||
/// Special case for 'count' function. It could be called with Nullable arguments
|
||||
/// - that means - count number of calls, when all arguments are not NULL.
|
||||
if (nested_function && nested_function->getName() == "count")
|
||||
return std::make_shared<AggregateFunctionCountNotNullUnary>(arguments[0], params);
|
||||
|
||||
bool has_nullable_types = false;
|
||||
bool has_null_types = false;
|
||||
for (const auto & arg_type : arguments)
|
||||
@ -49,35 +54,47 @@ public:
|
||||
}
|
||||
|
||||
if (!has_nullable_types)
|
||||
throw Exception("Aggregate function combinator 'Null' requires at least one argument to be Nullable", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (nested_function)
|
||||
if (auto adapter = nested_function->getOwnNullAdapter(nested_function, arguments, params))
|
||||
return adapter;
|
||||
|
||||
/// Special case for 'count' function. It could be called with Nullable arguments
|
||||
/// - that means - count number of calls, when all arguments are not NULL.
|
||||
if (nested_function && nested_function->getName() == "count")
|
||||
return std::make_shared<AggregateFunctionCountNotNullUnary>(arguments[0], params);
|
||||
throw Exception("Aggregate function combinator 'Null' requires at least one argument to be Nullable",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (has_null_types)
|
||||
return std::make_shared<AggregateFunctionNothing>(arguments, params);
|
||||
|
||||
bool return_type_is_nullable = nested_function->getReturnType()->canBeInsideNullable();
|
||||
assert(nested_function);
|
||||
|
||||
if (auto adapter = nested_function->getOwnNullAdapter(nested_function, arguments, params))
|
||||
return adapter;
|
||||
|
||||
bool return_type_is_nullable = !nested_function->returnDefaultWhenOnlyNull() && nested_function->getReturnType()->canBeInsideNullable();
|
||||
bool serialize_flag = return_type_is_nullable || nested_function->returnDefaultWhenOnlyNull();
|
||||
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if (return_type_is_nullable)
|
||||
return std::make_shared<AggregateFunctionNullUnary<true>>(nested_function, arguments, params);
|
||||
{
|
||||
return std::make_shared<AggregateFunctionNullUnary<true, true>>(nested_function, arguments, params);
|
||||
}
|
||||
else
|
||||
return std::make_shared<AggregateFunctionNullUnary<false>>(nested_function, arguments, params);
|
||||
{
|
||||
if (serialize_flag)
|
||||
return std::make_shared<AggregateFunctionNullUnary<false, true>>(nested_function, arguments, params);
|
||||
else
|
||||
return std::make_shared<AggregateFunctionNullUnary<false, false>>(nested_function, arguments, params);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
if (return_type_is_nullable)
|
||||
return std::make_shared<AggregateFunctionNullVariadic<true, true>>(nested_function, arguments, params);
|
||||
{
|
||||
return std::make_shared<AggregateFunctionNullVariadic<true, true, true>>(nested_function, arguments, params);
|
||||
}
|
||||
else
|
||||
return std::make_shared<AggregateFunctionNullVariadic<false, true>>(nested_function, arguments, params);
|
||||
{
|
||||
if (serialize_flag)
|
||||
return std::make_shared<AggregateFunctionNullVariadic<false, true, true>>(nested_function, arguments, params);
|
||||
else
|
||||
return std::make_shared<AggregateFunctionNullVariadic<false, true, false>>(nested_function, arguments, params);
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -28,7 +28,10 @@ namespace ErrorCodes
|
||||
/// If all rows had NULL, the behaviour is determined by "result_is_nullable" template parameter.
|
||||
/// true - return NULL; false - return value from empty aggregation state of nested function.
|
||||
|
||||
template <bool result_is_nullable, typename Derived>
|
||||
/// When serialize_flag is set to true, the flag about presense of values is serialized
|
||||
/// regardless to the "result_is_nullable" even if it's unneeded - for protocol compatibility.
|
||||
|
||||
template <bool result_is_nullable, bool serialize_flag, typename Derived>
|
||||
class AggregateFunctionNullBase : public IAggregateFunctionHelper<Derived>
|
||||
{
|
||||
protected:
|
||||
@ -129,7 +132,7 @@ public:
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
|
||||
{
|
||||
bool flag = getFlag(place);
|
||||
if constexpr (result_is_nullable)
|
||||
if constexpr (serialize_flag)
|
||||
writeBinary(flag, buf);
|
||||
if (flag)
|
||||
nested_function->serialize(nestedPlace(place), buf);
|
||||
@ -138,7 +141,7 @@ public:
|
||||
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override
|
||||
{
|
||||
bool flag = 1;
|
||||
if constexpr (result_is_nullable)
|
||||
if constexpr (serialize_flag)
|
||||
readBinary(flag, buf);
|
||||
if (flag)
|
||||
{
|
||||
@ -183,12 +186,15 @@ public:
|
||||
/** There are two cases: for single argument and variadic.
|
||||
* Code for single argument is much more efficient.
|
||||
*/
|
||||
template <bool result_is_nullable>
|
||||
class AggregateFunctionNullUnary final : public AggregateFunctionNullBase<result_is_nullable, AggregateFunctionNullUnary<result_is_nullable>>
|
||||
template <bool result_is_nullable, bool serialize_flag>
|
||||
class AggregateFunctionNullUnary final
|
||||
: public AggregateFunctionNullBase<result_is_nullable, serialize_flag,
|
||||
AggregateFunctionNullUnary<result_is_nullable, serialize_flag>>
|
||||
{
|
||||
public:
|
||||
AggregateFunctionNullUnary(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params)
|
||||
: AggregateFunctionNullBase<result_is_nullable, AggregateFunctionNullUnary<result_is_nullable>>(std::move(nested_function_), arguments, params)
|
||||
: AggregateFunctionNullBase<result_is_nullable, serialize_flag,
|
||||
AggregateFunctionNullUnary<result_is_nullable, serialize_flag>>(std::move(nested_function_), arguments, params)
|
||||
{
|
||||
}
|
||||
|
||||
@ -218,12 +224,15 @@ public:
|
||||
};
|
||||
|
||||
|
||||
template <bool result_is_nullable, bool null_is_skipped>
|
||||
class AggregateFunctionNullVariadic final : public AggregateFunctionNullBase<result_is_nullable, AggregateFunctionNullVariadic<result_is_nullable, null_is_skipped>>
|
||||
template <bool result_is_nullable, bool serialize_flag, bool null_is_skipped>
|
||||
class AggregateFunctionNullVariadic final
|
||||
: public AggregateFunctionNullBase<result_is_nullable, serialize_flag,
|
||||
AggregateFunctionNullVariadic<result_is_nullable, serialize_flag, null_is_skipped>>
|
||||
{
|
||||
public:
|
||||
AggregateFunctionNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params)
|
||||
: AggregateFunctionNullBase<result_is_nullable, AggregateFunctionNullVariadic<result_is_nullable, null_is_skipped>>(std::move(nested_function_), arguments, params),
|
||||
: AggregateFunctionNullBase<result_is_nullable, serialize_flag,
|
||||
AggregateFunctionNullVariadic<result_is_nullable, serialize_flag, null_is_skipped>>(std::move(nested_function_), arguments, params),
|
||||
number_of_arguments(arguments.size())
|
||||
{
|
||||
if (number_of_arguments == 1)
|
||||
@ -263,11 +272,6 @@ public:
|
||||
this->nested_function->add(this->nestedPlace(place), nested_columns, row_num, arena);
|
||||
}
|
||||
|
||||
bool allocatesMemoryInArena() const override
|
||||
{
|
||||
return this->nested_function->allocatesMemoryInArena();
|
||||
}
|
||||
|
||||
private:
|
||||
enum { MAX_ARGS = 8 };
|
||||
size_t number_of_arguments = 0;
|
||||
|
@ -244,6 +244,12 @@ public:
|
||||
{
|
||||
assert_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).set.size());
|
||||
}
|
||||
|
||||
/// The function returns non-Nullable type even when wrapped with Null combinator.
|
||||
bool returnDefaultWhenOnlyNull() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@ -298,6 +304,12 @@ public:
|
||||
{
|
||||
assert_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).set.size());
|
||||
}
|
||||
|
||||
/// The function returns non-Nullable type even when wrapped with Null combinator.
|
||||
bool returnDefaultWhenOnlyNull() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -240,9 +240,10 @@ public:
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
AggregateFunctionPtr getOwnNullAdapter(const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params) const override
|
||||
AggregateFunctionPtr getOwnNullAdapter(
|
||||
const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params) const override
|
||||
{
|
||||
return std::make_shared<AggregateFunctionNullVariadic<false, false>>(nested_function, arguments, params);
|
||||
return std::make_shared<AggregateFunctionNullVariadic<false, false, false>>(nested_function, arguments, params);
|
||||
}
|
||||
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override
|
||||
|
@ -171,6 +171,12 @@ public:
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
/** When the function is wrapped with Null combinator,
|
||||
* should we return Nullable type with NULL when no values were aggregated
|
||||
* or we should return non-Nullable type with default value (example: count, countDistinct).
|
||||
*/
|
||||
virtual bool returnDefaultWhenOnlyNull() const { return false; }
|
||||
|
||||
const DataTypes & getArgumentTypes() const { return argument_types; }
|
||||
const Array & getParameters() const { return parameters; }
|
||||
|
||||
|
@ -4,10 +4,10 @@
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#if __has_include(<sanitizer/asan_interface.h>)
|
||||
#include <Core/Defines.h>
|
||||
#if __has_include(<sanitizer/asan_interface.h>) && defined(ADDRESS_SANITIZER)
|
||||
# include <sanitizer/asan_interface.h>
|
||||
#endif
|
||||
#include <Core/Defines.h>
|
||||
#include <Common/memcpySmall.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/Allocator.h>
|
||||
|
@ -1,9 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#if __has_include(<sanitizer/asan_interface.h>)
|
||||
#include <Core/Defines.h>
|
||||
#if __has_include(<sanitizer/asan_interface.h>) && defined(ADDRESS_SANITIZER)
|
||||
# include <sanitizer/asan_interface.h>
|
||||
#endif
|
||||
#include <Core/Defines.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/BitHelpers.h>
|
||||
|
||||
|
@ -54,6 +54,18 @@ Elf::Elf(const std::string & path)
|
||||
throw Exception("The ELF is truncated (section names string table points after end of file)", ErrorCodes::CANNOT_PARSE_ELF);
|
||||
|
||||
section_names = reinterpret_cast<const char *>(mapped + section_names_offset);
|
||||
|
||||
/// Get program headers
|
||||
|
||||
ElfOff program_header_offset = header->e_phoff;
|
||||
uint16_t program_header_num_entries = header->e_phnum;
|
||||
|
||||
if (!program_header_offset
|
||||
|| !program_header_num_entries
|
||||
|| program_header_offset + program_header_num_entries * sizeof(ElfPhdr) > elf_size)
|
||||
throw Exception("The ELF is truncated (program header points after end of file)", ErrorCodes::CANNOT_PARSE_ELF);
|
||||
|
||||
program_headers = reinterpret_cast<const ElfPhdr *>(mapped + program_header_offset);
|
||||
}
|
||||
|
||||
|
||||
@ -104,6 +116,40 @@ std::optional<Elf::Section> Elf::findSectionByName(const char * name) const
|
||||
}
|
||||
|
||||
|
||||
String Elf::getBuildID() const
|
||||
{
|
||||
for (size_t idx = 0; idx < header->e_phnum; ++idx)
|
||||
{
|
||||
const ElfPhdr & phdr = program_headers[idx];
|
||||
|
||||
if (phdr.p_type == PT_NOTE)
|
||||
return getBuildID(mapped + phdr.p_offset, phdr.p_filesz);
|
||||
}
|
||||
return {};
|
||||
}
|
||||
|
||||
|
||||
String Elf::getBuildID(const char * nhdr_pos, size_t size)
|
||||
{
|
||||
const char * nhdr_end = nhdr_pos + size;
|
||||
|
||||
while (nhdr_pos < nhdr_end)
|
||||
{
|
||||
const ElfNhdr & nhdr = *reinterpret_cast<const ElfNhdr *>(nhdr_pos);
|
||||
|
||||
nhdr_pos += sizeof(ElfNhdr) + nhdr.n_namesz;
|
||||
if (nhdr.n_type == NT_GNU_BUILD_ID)
|
||||
{
|
||||
const char * build_id = nhdr_pos;
|
||||
return {build_id, nhdr.n_descsz};
|
||||
}
|
||||
nhdr_pos += nhdr.n_descsz;
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
|
||||
const char * Elf::Section::name() const
|
||||
{
|
||||
if (!elf.section_names)
|
||||
|
@ -17,6 +17,7 @@ using ElfEhdr = ElfW(Ehdr);
|
||||
using ElfOff = ElfW(Off);
|
||||
using ElfPhdr = ElfW(Phdr);
|
||||
using ElfShdr = ElfW(Shdr);
|
||||
using ElfNhdr = ElfW(Nhdr);
|
||||
using ElfSym = ElfW(Sym);
|
||||
|
||||
|
||||
@ -53,12 +54,18 @@ public:
|
||||
const char * end() const { return mapped + elf_size; }
|
||||
size_t size() const { return elf_size; }
|
||||
|
||||
/// Obtain build id from PT_NOTES section of program headers. Return empty string if does not exist.
|
||||
/// The string is returned in binary. Note that "readelf -n ./clickhouse-server" prints it in hex.
|
||||
String getBuildID() const;
|
||||
static String getBuildID(const char * nhdr_pos, size_t size);
|
||||
|
||||
private:
|
||||
MMapReadBufferFromFile in;
|
||||
size_t elf_size;
|
||||
const char * mapped;
|
||||
const ElfEhdr * header;
|
||||
const ElfShdr * section_headers;
|
||||
const ElfPhdr * program_headers;
|
||||
const char * section_names = nullptr;
|
||||
};
|
||||
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Core/AccurateComparison.h>
|
||||
#include <common/demangle.h>
|
||||
|
||||
|
||||
@ -14,7 +13,6 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_CONVERT_TYPE;
|
||||
extern const int BAD_TYPE_OF_FIELD;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
@ -177,243 +175,6 @@ template <> constexpr bool isDecimalField<DecimalField<Decimal64>>() { return tr
|
||||
template <> constexpr bool isDecimalField<DecimalField<Decimal128>>() { return true; }
|
||||
|
||||
|
||||
/** More precise comparison, used for index.
|
||||
* Differs from Field::operator< and Field::operator== in that it also compares values of different types.
|
||||
* Comparison rules are same as in FunctionsComparison (to be consistent with expression evaluation in query).
|
||||
*/
|
||||
class FieldVisitorAccurateEquals : public StaticVisitor<bool>
|
||||
{
|
||||
public:
|
||||
bool operator() (const UInt64 &, const Null &) const { return false; }
|
||||
bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; }
|
||||
bool operator() (const UInt64 & l, const UInt128 & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const UInt64 & l, const Int64 & r) const { return accurate::equalsOp(l, r); }
|
||||
bool operator() (const UInt64 & l, const Float64 & r) const { return accurate::equalsOp(l, r); }
|
||||
bool operator() (const UInt64 & l, const String & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const UInt64 & l, const Array & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const UInt64 & l, const Tuple & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const UInt64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); }
|
||||
|
||||
bool operator() (const Int64 &, const Null &) const { return false; }
|
||||
bool operator() (const Int64 & l, const UInt64 & r) const { return accurate::equalsOp(l, r); }
|
||||
bool operator() (const Int64 & l, const UInt128 & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const Int64 & l, const Int64 & r) const { return l == r; }
|
||||
bool operator() (const Int64 & l, const Float64 & r) const { return accurate::equalsOp(l, r); }
|
||||
bool operator() (const Int64 & l, const String & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const Int64 & l, const Array & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const Int64 & l, const Tuple & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const Int64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); }
|
||||
|
||||
bool operator() (const Float64 &, const Null &) const { return false; }
|
||||
bool operator() (const Float64 & l, const UInt64 & r) const { return accurate::equalsOp(l, r); }
|
||||
bool operator() (const Float64 & l, const UInt128 & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const Float64 & l, const Int64 & r) const { return accurate::equalsOp(l, r); }
|
||||
bool operator() (const Float64 & l, const Float64 & r) const { return l == r; }
|
||||
bool operator() (const Float64 & l, const String & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const Float64 & l, const Array & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const Float64 & l, const Tuple & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const Float64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); }
|
||||
|
||||
template <typename T>
|
||||
bool operator() (const Null &, const T &) const
|
||||
{
|
||||
return std::is_same_v<T, Null>;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool operator() (const String & l, const T & r) const
|
||||
{
|
||||
if constexpr (std::is_same_v<T, String>)
|
||||
return l == r;
|
||||
if constexpr (std::is_same_v<T, UInt128>)
|
||||
return stringToUUID(l) == r;
|
||||
if constexpr (std::is_same_v<T, Null>)
|
||||
return false;
|
||||
return cantCompare(l, r);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool operator() (const UInt128 & l, const T & r) const
|
||||
{
|
||||
if constexpr (std::is_same_v<T, UInt128>)
|
||||
return l == r;
|
||||
if constexpr (std::is_same_v<T, String>)
|
||||
return l == stringToUUID(r);
|
||||
if constexpr (std::is_same_v<T, Null>)
|
||||
return false;
|
||||
return cantCompare(l, r);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool operator() (const Array & l, const T & r) const
|
||||
{
|
||||
if constexpr (std::is_same_v<T, Array>)
|
||||
return l == r;
|
||||
if constexpr (std::is_same_v<T, Null>)
|
||||
return false;
|
||||
return cantCompare(l, r);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool operator() (const Tuple & l, const T & r) const
|
||||
{
|
||||
if constexpr (std::is_same_v<T, Tuple>)
|
||||
return l == r;
|
||||
if constexpr (std::is_same_v<T, Null>)
|
||||
return false;
|
||||
return cantCompare(l, r);
|
||||
}
|
||||
|
||||
template <typename T, typename U>
|
||||
bool operator() (const DecimalField<T> & l, const U & r) const
|
||||
{
|
||||
if constexpr (isDecimalField<U>())
|
||||
return l == r;
|
||||
if constexpr (std::is_same_v<U, Int64> || std::is_same_v<U, UInt64>)
|
||||
return l == DecimalField<Decimal128>(r, 0);
|
||||
if constexpr (std::is_same_v<U, Null>)
|
||||
return false;
|
||||
return cantCompare(l, r);
|
||||
}
|
||||
|
||||
template <typename T> bool operator() (const UInt64 & l, const DecimalField<T> & r) const { return DecimalField<Decimal128>(l, 0) == r; }
|
||||
template <typename T> bool operator() (const Int64 & l, const DecimalField<T> & r) const { return DecimalField<Decimal128>(l, 0) == r; }
|
||||
template <typename T> bool operator() (const Float64 & l, const DecimalField<T> & r) const { return cantCompare(l, r); }
|
||||
|
||||
template <typename T>
|
||||
bool operator() (const AggregateFunctionStateData & l, const T & r) const
|
||||
{
|
||||
if constexpr (std::is_same_v<T, AggregateFunctionStateData>)
|
||||
return l == r;
|
||||
return cantCompare(l, r);
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T, typename U>
|
||||
bool cantCompare(const T &, const U &) const
|
||||
{
|
||||
if constexpr (std::is_same_v<U, Null>)
|
||||
return false;
|
||||
throw Exception("Cannot compare " + demangle(typeid(T).name()) + " with " + demangle(typeid(U).name()),
|
||||
ErrorCodes::BAD_TYPE_OF_FIELD);
|
||||
}
|
||||
};
|
||||
|
||||
class FieldVisitorAccurateLess : public StaticVisitor<bool>
|
||||
{
|
||||
public:
|
||||
bool operator() (const UInt64 &, const Null &) const { return false; }
|
||||
bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; }
|
||||
bool operator() (const UInt64 & l, const UInt128 & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const UInt64 & l, const Int64 & r) const { return accurate::lessOp(l, r); }
|
||||
bool operator() (const UInt64 & l, const Float64 & r) const { return accurate::lessOp(l, r); }
|
||||
bool operator() (const UInt64 & l, const String & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const UInt64 & l, const Array & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const UInt64 & l, const Tuple & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const UInt64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); }
|
||||
|
||||
bool operator() (const Int64 &, const Null &) const { return false; }
|
||||
bool operator() (const Int64 & l, const UInt64 & r) const { return accurate::lessOp(l, r); }
|
||||
bool operator() (const Int64 & l, const UInt128 & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const Int64 & l, const Int64 & r) const { return l < r; }
|
||||
bool operator() (const Int64 & l, const Float64 & r) const { return accurate::lessOp(l, r); }
|
||||
bool operator() (const Int64 & l, const String & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const Int64 & l, const Array & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const Int64 & l, const Tuple & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const Int64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); }
|
||||
|
||||
bool operator() (const Float64 &, const Null &) const { return false; }
|
||||
bool operator() (const Float64 & l, const UInt64 & r) const { return accurate::lessOp(l, r); }
|
||||
bool operator() (const Float64 & l, const UInt128 & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const Float64 & l, const Int64 & r) const { return accurate::lessOp(l, r); }
|
||||
bool operator() (const Float64 & l, const Float64 & r) const { return l < r; }
|
||||
bool operator() (const Float64 & l, const String & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const Float64 & l, const Array & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const Float64 & l, const Tuple & r) const { return cantCompare(l, r); }
|
||||
bool operator() (const Float64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); }
|
||||
|
||||
template <typename T>
|
||||
bool operator() (const Null &, const T &) const
|
||||
{
|
||||
return !std::is_same_v<T, Null>;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool operator() (const String & l, const T & r) const
|
||||
{
|
||||
if constexpr (std::is_same_v<T, String>)
|
||||
return l < r;
|
||||
if constexpr (std::is_same_v<T, UInt128>)
|
||||
return stringToUUID(l) < r;
|
||||
if constexpr (std::is_same_v<T, Null>)
|
||||
return false;
|
||||
return cantCompare(l, r);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool operator() (const UInt128 & l, const T & r) const
|
||||
{
|
||||
if constexpr (std::is_same_v<T, UInt128>)
|
||||
return l < r;
|
||||
if constexpr (std::is_same_v<T, String>)
|
||||
return l < stringToUUID(r);
|
||||
if constexpr (std::is_same_v<T, Null>)
|
||||
return false;
|
||||
return cantCompare(l, r);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool operator() (const Array & l, const T & r) const
|
||||
{
|
||||
if constexpr (std::is_same_v<T, Array>)
|
||||
return l < r;
|
||||
if constexpr (std::is_same_v<T, Null>)
|
||||
return false;
|
||||
return cantCompare(l, r);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool operator() (const Tuple & l, const T & r) const
|
||||
{
|
||||
if constexpr (std::is_same_v<T, Tuple>)
|
||||
return l < r;
|
||||
if constexpr (std::is_same_v<T, Null>)
|
||||
return false;
|
||||
return cantCompare(l, r);
|
||||
}
|
||||
|
||||
template <typename T, typename U>
|
||||
bool operator() (const DecimalField<T> & l, const U & r) const
|
||||
{
|
||||
if constexpr (isDecimalField<U>())
|
||||
return l < r;
|
||||
if constexpr (std::is_same_v<U, Int64> || std::is_same_v<U, UInt64>)
|
||||
return l < DecimalField<Decimal128>(r, 0);
|
||||
if constexpr (std::is_same_v<U, Null>)
|
||||
return false;
|
||||
return cantCompare(l, r);
|
||||
}
|
||||
|
||||
template <typename T> bool operator() (const UInt64 & l, const DecimalField<T> & r) const { return DecimalField<Decimal128>(l, 0) < r; }
|
||||
template <typename T> bool operator() (const Int64 & l, const DecimalField<T> & r) const { return DecimalField<Decimal128>(l, 0) < r; }
|
||||
template <typename T> bool operator() (const Float64 &, const DecimalField<T> &) const { return false; }
|
||||
|
||||
template <typename T>
|
||||
bool operator() (const AggregateFunctionStateData & l, const T & r) const
|
||||
{
|
||||
return cantCompare(l, r);
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T, typename U>
|
||||
bool cantCompare(const T &, const U &) const
|
||||
{
|
||||
throw Exception("Cannot compare " + demangle(typeid(T).name()) + " with " + demangle(typeid(U).name()),
|
||||
ErrorCodes::BAD_TYPE_OF_FIELD);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Implements `+=` operation.
|
||||
* Returns false if the result is zero.
|
||||
*/
|
||||
|
142
src/Common/FieldVisitorsAccurateComparison.h
Normal file
142
src/Common/FieldVisitorsAccurateComparison.h
Normal file
@ -0,0 +1,142 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Core/AccurateComparison.h>
|
||||
#include <common/demangle.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_TYPE_OF_FIELD;
|
||||
}
|
||||
|
||||
/** More precise comparison, used for index.
|
||||
* Differs from Field::operator< and Field::operator== in that it also compares values of different types.
|
||||
* Comparison rules are same as in FunctionsComparison (to be consistent with expression evaluation in query).
|
||||
*/
|
||||
class FieldVisitorAccurateEquals : public StaticVisitor<bool>
|
||||
{
|
||||
public:
|
||||
template <typename T, typename U>
|
||||
bool operator() (const T & l, const U & r) const
|
||||
{
|
||||
if constexpr (std::is_same_v<T, Null> || std::is_same_v<U, Null>)
|
||||
return std::is_same_v<T, U>;
|
||||
else
|
||||
{
|
||||
if constexpr (std::is_same_v<T, U>)
|
||||
return l == r;
|
||||
|
||||
if constexpr (std::is_arithmetic_v<T> && std::is_arithmetic_v<U>)
|
||||
return accurate::equalsOp(l, r);
|
||||
|
||||
if constexpr (isDecimalField<T>() && isDecimalField<U>())
|
||||
return l == r;
|
||||
|
||||
if constexpr (isDecimalField<T>() && std::is_arithmetic_v<U>)
|
||||
return l == DecimalField<Decimal128>(r, 0);
|
||||
|
||||
if constexpr (std::is_arithmetic_v<T> && isDecimalField<U>())
|
||||
return DecimalField<Decimal128>(l, 0) == r;
|
||||
|
||||
if constexpr (std::is_same_v<T, String>)
|
||||
{
|
||||
if constexpr (std::is_same_v<U, UInt128>)
|
||||
return stringToUUID(l) == r;
|
||||
|
||||
if constexpr (std::is_arithmetic_v<U>)
|
||||
{
|
||||
ReadBufferFromString in(l);
|
||||
T parsed;
|
||||
readText(parsed, in);
|
||||
return operator()(parsed, r);
|
||||
}
|
||||
}
|
||||
|
||||
if constexpr (std::is_same_v<U, String>)
|
||||
{
|
||||
if constexpr (std::is_same_v<T, UInt128>)
|
||||
return l == stringToUUID(r);
|
||||
|
||||
if constexpr (std::is_arithmetic_v<T>)
|
||||
{
|
||||
ReadBufferFromString in(r);
|
||||
T parsed;
|
||||
readText(parsed, in);
|
||||
return operator()(l, parsed);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception("Cannot compare " + demangle(typeid(T).name()) + " with " + demangle(typeid(U).name()),
|
||||
ErrorCodes::BAD_TYPE_OF_FIELD);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FieldVisitorAccurateLess : public StaticVisitor<bool>
|
||||
{
|
||||
public:
|
||||
template <typename T, typename U>
|
||||
bool operator() (const T & l, const U & r) const
|
||||
{
|
||||
if constexpr (std::is_same_v<T, Null> || std::is_same_v<U, Null>)
|
||||
return false;
|
||||
else
|
||||
{
|
||||
if constexpr (std::is_same_v<T, U>)
|
||||
return l < r;
|
||||
|
||||
if constexpr (std::is_arithmetic_v<T> && std::is_arithmetic_v<U>)
|
||||
return accurate::lessOp(l, r);
|
||||
|
||||
if constexpr (isDecimalField<T>() && isDecimalField<U>())
|
||||
return l < r;
|
||||
|
||||
if constexpr (isDecimalField<T>() && std::is_arithmetic_v<U>)
|
||||
return l < DecimalField<Decimal128>(r, 0);
|
||||
|
||||
if constexpr (std::is_arithmetic_v<T> && isDecimalField<U>())
|
||||
return DecimalField<Decimal128>(l, 0) < r;
|
||||
|
||||
if constexpr (std::is_same_v<T, String>)
|
||||
{
|
||||
if constexpr (std::is_same_v<U, UInt128>)
|
||||
return stringToUUID(l) < r;
|
||||
|
||||
if constexpr (std::is_arithmetic_v<U>)
|
||||
{
|
||||
ReadBufferFromString in(l);
|
||||
T parsed;
|
||||
readText(parsed, in);
|
||||
return operator()(parsed, r);
|
||||
}
|
||||
}
|
||||
|
||||
if constexpr (std::is_same_v<U, String>)
|
||||
{
|
||||
if constexpr (std::is_same_v<T, UInt128>)
|
||||
return l < stringToUUID(r);
|
||||
|
||||
if constexpr (std::is_arithmetic_v<T>)
|
||||
{
|
||||
ReadBufferFromString in(r);
|
||||
T parsed;
|
||||
readText(parsed, in);
|
||||
return operator()(l, parsed);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception("Cannot compare " + demangle(typeid(T).name()) + " with " + demangle(typeid(U).name()),
|
||||
ErrorCodes::BAD_TYPE_OF_FIELD);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -38,6 +38,7 @@ void OptimizedRegularExpressionImpl<thread_safe>::analyze(
|
||||
required_substring_is_prefix = false;
|
||||
required_substring.clear();
|
||||
bool has_alternative_on_depth_0 = false;
|
||||
bool has_case_insensitive_flag = false;
|
||||
|
||||
/// Substring with a position.
|
||||
using Substring = std::pair<std::string, size_t>;
|
||||
@ -65,7 +66,17 @@ void OptimizedRegularExpressionImpl<thread_safe>::analyze(
|
||||
|
||||
switch (*pos)
|
||||
{
|
||||
case '|': case '(': case ')': case '^': case '$': case '.': case '[': case '?': case '*': case '+': case '{':
|
||||
case '|':
|
||||
case '(':
|
||||
case ')':
|
||||
case '^':
|
||||
case '$':
|
||||
case '.':
|
||||
case '[':
|
||||
case '?':
|
||||
case '*':
|
||||
case '+':
|
||||
case '{':
|
||||
if (depth == 0 && !in_curly_braces && !in_square_braces)
|
||||
{
|
||||
if (last_substring->first.empty())
|
||||
@ -110,6 +121,28 @@ void OptimizedRegularExpressionImpl<thread_safe>::analyze(
|
||||
trivial_substrings.resize(trivial_substrings.size() + 1);
|
||||
last_substring = &trivial_substrings.back();
|
||||
}
|
||||
|
||||
/// Check for case-insensitive flag.
|
||||
if (pos + 1 < end && pos[1] == '?')
|
||||
{
|
||||
for (size_t offset = 2; pos + offset < end; ++offset)
|
||||
{
|
||||
if (pos[offset] == '-' /// it means flag negation
|
||||
/// various possible flags, actually only imsU are supported by re2
|
||||
|| (pos[offset] >= 'a' && pos[offset] <= 'z')
|
||||
|| (pos[offset] >= 'A' && pos[offset] <= 'Z'))
|
||||
{
|
||||
if (pos[offset] == 'i')
|
||||
{
|
||||
/// Actually it can be negated case-insensitive flag. But we don't care.
|
||||
has_case_insensitive_flag = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
else
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
++pos;
|
||||
break;
|
||||
@ -209,7 +242,7 @@ void OptimizedRegularExpressionImpl<thread_safe>::analyze(
|
||||
|
||||
if (!is_trivial)
|
||||
{
|
||||
if (!has_alternative_on_depth_0)
|
||||
if (!has_alternative_on_depth_0 && !has_case_insensitive_flag)
|
||||
{
|
||||
/// We choose the non-alternative substring of the maximum length for first search.
|
||||
|
||||
|
@ -196,6 +196,20 @@ void collectSymbolsFromProgramHeaders(dl_phdr_info * info,
|
||||
}
|
||||
|
||||
|
||||
String getBuildIDFromProgramHeaders(dl_phdr_info * info)
|
||||
{
|
||||
for (size_t header_index = 0; header_index < info->dlpi_phnum; ++header_index)
|
||||
{
|
||||
const ElfPhdr & phdr = info->dlpi_phdr[header_index];
|
||||
if (phdr.p_type != PT_NOTE)
|
||||
continue;
|
||||
|
||||
return Elf::getBuildID(reinterpret_cast<const char *>(info->dlpi_addr + phdr.p_vaddr), phdr.p_memsz);
|
||||
}
|
||||
return {};
|
||||
}
|
||||
|
||||
|
||||
void collectSymbolsFromELFSymbolTable(
|
||||
dl_phdr_info * info,
|
||||
const Elf & elf,
|
||||
@ -283,8 +297,31 @@ void collectSymbolsFromELF(dl_phdr_info * info,
|
||||
|
||||
object_name = std::filesystem::exists(debug_info_path) ? debug_info_path : canonical_path;
|
||||
|
||||
/// But we have to compare Build ID to check that debug info corresponds to the same executable.
|
||||
String our_build_id = getBuildIDFromProgramHeaders(info);
|
||||
|
||||
SymbolIndex::Object object;
|
||||
object.elf = std::make_unique<Elf>(object_name);
|
||||
|
||||
String file_build_id = object.elf->getBuildID();
|
||||
|
||||
if (our_build_id != file_build_id)
|
||||
{
|
||||
/// If debug info doesn't correspond to our binary, fallback to the info in our binary.
|
||||
if (object_name != canonical_path)
|
||||
{
|
||||
object_name = canonical_path;
|
||||
object.elf = std::make_unique<Elf>(object_name);
|
||||
|
||||
/// But it can still be outdated, for example, if executable file was deleted from filesystem and replaced by another file.
|
||||
file_build_id = object.elf->getBuildID();
|
||||
if (our_build_id != file_build_id)
|
||||
return;
|
||||
}
|
||||
else
|
||||
return;
|
||||
}
|
||||
|
||||
object.address_begin = reinterpret_cast<const void *>(info->dlpi_addr);
|
||||
object.address_end = reinterpret_cast<const void *>(info->dlpi_addr + object.elf->size());
|
||||
object.name = object_name;
|
||||
|
18
src/Common/clearPasswordFromCommandLine.cpp
Normal file
18
src/Common/clearPasswordFromCommandLine.cpp
Normal file
@ -0,0 +1,18 @@
|
||||
#include <string.h>
|
||||
#include "clearPasswordFromCommandLine.h"
|
||||
|
||||
void clearPasswordFromCommandLine(int argc, char ** argv)
|
||||
{
|
||||
for (int arg = 1; arg < argc; ++arg)
|
||||
{
|
||||
if (arg + 1 < argc && 0 == strcmp(argv[arg], "--password"))
|
||||
{
|
||||
++arg;
|
||||
memset(argv[arg], 0, strlen(argv[arg]));
|
||||
}
|
||||
else if (0 == strncmp(argv[arg], "--password=", strlen("--password=")))
|
||||
{
|
||||
memset(argv[arg] + strlen("--password="), 0, strlen(argv[arg]) - strlen("--password="));
|
||||
}
|
||||
}
|
||||
}
|
6
src/Common/clearPasswordFromCommandLine.h
Normal file
6
src/Common/clearPasswordFromCommandLine.h
Normal file
@ -0,0 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
/** If there are --password=... or --password ... arguments in command line, replace their values with zero bytes.
|
||||
* This is needed to prevent password exposure in 'ps' and similar tools.
|
||||
*/
|
||||
void clearPasswordFromCommandLine(int argc, char ** argv);
|
@ -30,6 +30,7 @@ SRCS(
|
||||
Config/configReadClient.cpp
|
||||
Config/ConfigReloader.cpp
|
||||
createHardLink.cpp
|
||||
clearPasswordFromCommandLine.cpp
|
||||
CurrentMetrics.cpp
|
||||
CurrentThread.cpp
|
||||
DNSResolver.cpp
|
||||
|
@ -87,7 +87,7 @@
|
||||
#define DBMS_DISTRIBUTED_SIGNATURE_HEADER 0xCAFEDACEull
|
||||
#define DBMS_DISTRIBUTED_SIGNATURE_HEADER_OLD_FORMAT 0xCAFECABEull
|
||||
|
||||
#if !__has_include(<sanitizer/asan_interface.h>)
|
||||
#if !__has_include(<sanitizer/asan_interface.h>) || !defined(ADDRESS_SANITIZER)
|
||||
# define ASAN_UNPOISON_MEMORY_REGION(a, b)
|
||||
# define ASAN_POISON_MEMORY_REGION(a, b)
|
||||
#endif
|
||||
|
@ -7,6 +7,8 @@
|
||||
*/
|
||||
|
||||
#include <Common/SettingsChanges.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -91,7 +93,16 @@ Field SettingsCollection<Derived>::const_reference::getValue() const
|
||||
template <class Derived>
|
||||
Field SettingsCollection<Derived>::valueToCorrespondingType(size_t index, const Field & value)
|
||||
{
|
||||
return members()[index].value_to_corresponding_type(value);
|
||||
try
|
||||
{
|
||||
return members()[index].value_to_corresponding_type(value);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage(fmt::format("in attempt to set the value of setting to {}",
|
||||
applyVisitor(FieldVisitorToString(), value)));
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
@ -45,6 +45,8 @@ public:
|
||||
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
bool canBePromoted() const override { return false; }
|
||||
};
|
||||
|
||||
/** Tansform-type wrapper for DateTime64, applies given Transform to DateTime64 value or only to a whole part of it.
|
||||
|
@ -16,13 +16,18 @@ void DataTypeUUID::serializeText(const IColumn & column, size_t row_num, WriteBu
|
||||
writeText(UUID(assert_cast<const ColumnUInt128 &>(column).getData()[row_num]), ostr);
|
||||
}
|
||||
|
||||
void DataTypeUUID::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
void DataTypeUUID::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
UUID x;
|
||||
readText(x, istr);
|
||||
assert_cast<ColumnUInt128 &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
void DataTypeUUID::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
void DataTypeUUID::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
|
@ -17,6 +17,7 @@ public:
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
@ -30,6 +31,8 @@ public:
|
||||
|
||||
bool canBeUsedInBitOperations() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
|
||||
bool canBePromoted() const override { return false; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -22,6 +22,7 @@
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
|
||||
#include <Functions/IFunctionAdaptors.h>
|
||||
@ -51,7 +52,6 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TOO_LARGE_STRING_SIZE;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int LOGICAL_ERROR;
|
||||
@ -812,94 +812,51 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
bool executeDateOrDateTimeOrEnumOrUUIDWithConstString(
|
||||
bool executeWithConstString(
|
||||
Block & block, size_t result, const IColumn * col_left_untyped, const IColumn * col_right_untyped,
|
||||
const DataTypePtr & left_type, const DataTypePtr & right_type, bool left_is_num, size_t input_rows_count)
|
||||
const DataTypePtr & left_type, const DataTypePtr & right_type, size_t input_rows_count)
|
||||
{
|
||||
/// This is no longer very special case - comparing dates, datetimes, and enumerations with a string constant.
|
||||
const IColumn * column_string_untyped = !left_is_num ? col_left_untyped : col_right_untyped;
|
||||
const IColumn * column_number = left_is_num ? col_left_untyped : col_right_untyped;
|
||||
const IDataType * number_type = left_is_num ? left_type.get() : right_type.get();
|
||||
/// To compare something with const string, we cast constant to appropriate type and compare as usual.
|
||||
/// It is ok to throw exception if value is not convertible.
|
||||
/// We should deal with possible overflows, e.g. toUInt8(1) = '257' should return false.
|
||||
|
||||
WhichDataType which(number_type);
|
||||
const ColumnConst * left_const = checkAndGetColumnConstStringOrFixedString(col_left_untyped);
|
||||
const ColumnConst * right_const = checkAndGetColumnConstStringOrFixedString(col_right_untyped);
|
||||
|
||||
const bool legal_types = which.isDateOrDateTime() || which.isEnum() || which.isUUID();
|
||||
|
||||
const auto column_string = checkAndGetColumnConst<ColumnString>(column_string_untyped);
|
||||
if (!column_string || !legal_types)
|
||||
if (!left_const && !right_const)
|
||||
return false;
|
||||
|
||||
StringRef string_value = column_string->getDataAt(0);
|
||||
const IDataType * type_string = left_const ? left_type.get() : right_type.get();
|
||||
const DataTypePtr & type_to_compare = !left_const ? left_type : right_type;
|
||||
|
||||
if (which.isDate())
|
||||
Field string_value = left_const ? left_const->getField() : right_const->getField();
|
||||
Field converted = convertFieldToType(string_value, *type_to_compare, type_string);
|
||||
|
||||
/// If not possible to convert, comparison with =, <, >, <=, >= yields to false and comparison with != yields to true.
|
||||
if (converted.isNull())
|
||||
{
|
||||
DayNum date;
|
||||
ReadBufferFromMemory in(string_value.data, string_value.size);
|
||||
readDateText(date, in);
|
||||
if (!in.eof())
|
||||
throw Exception("String is too long for Date: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||
|
||||
ColumnPtr parsed_const_date_holder = DataTypeDate().createColumnConst(input_rows_count, date);
|
||||
const ColumnConst * parsed_const_date = assert_cast<const ColumnConst *>(parsed_const_date_holder.get());
|
||||
executeNumLeftType<DataTypeDate::FieldType>(block, result,
|
||||
left_is_num ? col_left_untyped : parsed_const_date,
|
||||
left_is_num ? parsed_const_date : col_right_untyped);
|
||||
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count,
|
||||
std::is_same_v<Op<int, int>, NotEqualsOp<int, int>>);
|
||||
}
|
||||
else if (which.isDateTime())
|
||||
else
|
||||
{
|
||||
time_t date_time;
|
||||
ReadBufferFromMemory in(string_value.data, string_value.size);
|
||||
readDateTimeText(date_time, in, dynamic_cast<const DataTypeDateTime &>(*number_type).getTimeZone());
|
||||
if (!in.eof())
|
||||
throw Exception("String is too long for DateTime: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||
auto column_converted = type_to_compare->createColumnConst(input_rows_count, converted);
|
||||
|
||||
ColumnPtr parsed_const_date_time_holder = DataTypeDateTime().createColumnConst(input_rows_count, UInt64(date_time));
|
||||
const ColumnConst * parsed_const_date_time = assert_cast<const ColumnConst *>(parsed_const_date_time_holder.get());
|
||||
executeNumLeftType<DataTypeDateTime::FieldType>(block, result,
|
||||
left_is_num ? col_left_untyped : parsed_const_date_time,
|
||||
left_is_num ? parsed_const_date_time : col_right_untyped);
|
||||
Block tmp_block
|
||||
{
|
||||
{ left_const ? column_converted : col_left_untyped->getPtr(), type_to_compare, "" },
|
||||
{ !left_const ? column_converted : col_right_untyped->getPtr(), type_to_compare, "" },
|
||||
block.getByPosition(result)
|
||||
};
|
||||
|
||||
executeImpl(tmp_block, {0, 1}, 2, input_rows_count);
|
||||
|
||||
block.getByPosition(result).column = std::move(tmp_block.getByPosition(2).column);
|
||||
}
|
||||
else if (which.isUUID())
|
||||
{
|
||||
UUID uuid;
|
||||
ReadBufferFromMemory in(string_value.data, string_value.size);
|
||||
readText(uuid, in);
|
||||
if (!in.eof())
|
||||
throw Exception("String is too long for UUID: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||
|
||||
ColumnPtr parsed_const_uuid_holder = DataTypeUUID().createColumnConst(input_rows_count, uuid);
|
||||
const ColumnConst * parsed_const_uuid = assert_cast<const ColumnConst *>(parsed_const_uuid_holder.get());
|
||||
executeNumLeftType<DataTypeUUID::FieldType>(block, result,
|
||||
left_is_num ? col_left_untyped : parsed_const_uuid,
|
||||
left_is_num ? parsed_const_uuid : col_right_untyped);
|
||||
}
|
||||
|
||||
else if (which.isEnum8())
|
||||
executeEnumWithConstString<DataTypeEnum8>(block, result, column_number, column_string,
|
||||
number_type, left_is_num, input_rows_count);
|
||||
else if (which.isEnum16())
|
||||
executeEnumWithConstString<DataTypeEnum16>(block, result, column_number, column_string,
|
||||
number_type, left_is_num, input_rows_count);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
/// Comparison between DataTypeEnum<T> and string constant containing the name of an enum element
|
||||
template <typename EnumType>
|
||||
void executeEnumWithConstString(
|
||||
Block & block, const size_t result, const IColumn * column_number, const ColumnConst * column_string,
|
||||
const IDataType * type_untyped, const bool left_is_num, size_t input_rows_count)
|
||||
{
|
||||
const auto type = static_cast<const EnumType *>(type_untyped);
|
||||
|
||||
const Field x = castToNearestFieldType(type->getValue(column_string->getValue<String>()));
|
||||
const auto enum_col = type->createColumnConst(input_rows_count, x);
|
||||
|
||||
executeNumLeftType<typename EnumType::FieldType>(block, result,
|
||||
left_is_num ? column_number : enum_col.get(),
|
||||
left_is_num ? enum_col.get() : column_number);
|
||||
}
|
||||
|
||||
void executeTuple(Block & block, size_t result, const ColumnWithTypeAndName & c0, const ColumnWithTypeAndName & c1,
|
||||
size_t input_rows_count)
|
||||
{
|
||||
@ -1124,17 +1081,11 @@ public:
|
||||
bool has_date = left.isDate() || right.isDate();
|
||||
|
||||
if (!((both_represented_by_number && !has_date) /// Do not allow compare date and number.
|
||||
|| (left.isStringOrFixedString() && right.isStringOrFixedString())
|
||||
|| (left.isStringOrFixedString() || right.isStringOrFixedString()) /// Everything can be compared with string by conversion.
|
||||
/// You can compare the date, datetime, or datatime64 and an enumeration with a constant string.
|
||||
|| (left.isString() && right.isDateOrDateTime())
|
||||
|| (left.isDateOrDateTime() && right.isString())
|
||||
|| (left.isDateOrDateTime() && right.isDateOrDateTime() && left.idx == right.idx) /// only date vs date, or datetime vs datetime
|
||||
|| (left.isUUID() && right.isUUID())
|
||||
|| (left.isUUID() && right.isString())
|
||||
|| (left.isString() && right.isUUID())
|
||||
|| (left.isEnum() && right.isEnum() && arguments[0]->getName() == arguments[1]->getName()) /// only equivalent enum type values can be compared against
|
||||
|| (left.isEnum() && right.isString())
|
||||
|| (left.isString() && right.isEnum())
|
||||
|| (left_tuple && right_tuple && left_tuple->getElements().size() == right_tuple->getElements().size())
|
||||
|| (arguments[0]->equals(*arguments[1]))))
|
||||
{
|
||||
@ -1151,7 +1102,8 @@ public:
|
||||
|
||||
if (left_tuple && right_tuple)
|
||||
{
|
||||
auto adaptor = FunctionOverloadResolverAdaptor(std::make_unique<DefaultOverloadResolver>(FunctionComparison<Op, Name>::create(context)));
|
||||
auto adaptor = FunctionOverloadResolverAdaptor(std::make_unique<DefaultOverloadResolver>(
|
||||
FunctionComparison<Op, Name>::create(context)));
|
||||
|
||||
size_t size = left_tuple->getElements().size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
@ -1201,6 +1153,9 @@ public:
|
||||
const bool left_is_num = col_left_untyped->isNumeric();
|
||||
const bool right_is_num = col_right_untyped->isNumeric();
|
||||
|
||||
const bool left_is_string = isStringOrFixedString(which_left);
|
||||
const bool right_is_string = isStringOrFixedString(which_right);
|
||||
|
||||
bool date_and_datetime = (left_type != right_type) &&
|
||||
which_left.isDateOrDateTime() && which_right.isDateOrDateTime();
|
||||
|
||||
@ -1226,64 +1181,14 @@ public:
|
||||
{
|
||||
executeTuple(block, result, col_with_type_and_name_left, col_with_type_and_name_right, input_rows_count);
|
||||
}
|
||||
else if (which_left.idx != which_right.idx
|
||||
&& (which_left.isDateTime64() || which_right.isDateTime64())
|
||||
&& (which_left.isStringOrFixedString() || which_right.isStringOrFixedString()))
|
||||
else if (left_is_string && right_is_string && executeString(block, result, col_left_untyped, col_right_untyped))
|
||||
{
|
||||
}
|
||||
else if (executeWithConstString(
|
||||
block, result, col_left_untyped, col_right_untyped,
|
||||
left_type, right_type,
|
||||
input_rows_count))
|
||||
{
|
||||
/** Special case of comparing DateTime64 against a string.
|
||||
*
|
||||
* Can't be moved to executeDateOrDateTimeOrEnumOrUUIDWithConstString()
|
||||
* since DateTime64 is basically a Decimal, but we do similar things, except type inference.
|
||||
* Outline:
|
||||
* - Extract string content
|
||||
* - Parse it as a ColumnDateTime64 value (same type as DateTime64, means same precision)
|
||||
* - Fabricate a column with type and name
|
||||
* - Compare left and right comlumns as DateTime64 columns.
|
||||
*/
|
||||
|
||||
const size_t datetime64_col_index = which_left.isDateTime64() ? 0 : 1;
|
||||
const size_t string_col_index = which_left.isStringOrFixedString() ? 0 : 1;
|
||||
|
||||
const auto & datetime64_col_with_type_and_name = block.getByPosition(arguments[datetime64_col_index]);
|
||||
const auto & string_col_with_type_and_name = block.getByPosition(arguments[string_col_index]);
|
||||
|
||||
if (!isColumnConst(*string_col_with_type_and_name.column))
|
||||
throw Exception(getName() + ", illegal column type of argument #" + std::to_string(string_col_index)
|
||||
+ " '" + string_col_with_type_and_name.name + "'"
|
||||
" expected const String or const FixedString,"
|
||||
" got " + string_col_with_type_and_name.type->getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (datetime64_col_with_type_and_name.column->size() == 0 || string_col_with_type_and_name.column->size() == 0)
|
||||
{
|
||||
// For some reason, when both left and right columns are empty (dry run while building a header block)
|
||||
// executeDecimal() fills result column with bogus value.
|
||||
block.getByPosition(result).column = ColumnUInt8::create();
|
||||
return;
|
||||
}
|
||||
|
||||
auto parsed_tmp_column_holder = datetime64_col_with_type_and_name.type->createColumn();
|
||||
|
||||
{
|
||||
const StringRef string_value = string_col_with_type_and_name.column->getDataAt(0);
|
||||
ReadBufferFromMemory in(string_value.data, string_value.size);
|
||||
datetime64_col_with_type_and_name.type->deserializeAsWholeText(*parsed_tmp_column_holder, in, FormatSettings{});
|
||||
|
||||
if (!in.eof())
|
||||
throw Exception(getName() + ": String is too long for " + datetime64_col_with_type_and_name.type->getName() + " : " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||
}
|
||||
|
||||
// It is necessary to wrap tmp column in ColumnConst to avoid overflow when comparing.
|
||||
// (non-const columns are expected to have same number of rows as every other column in block).
|
||||
const ColumnWithTypeAndName parsed_tmp_col_with_type_and_name{
|
||||
ColumnConst::create(std::move(parsed_tmp_column_holder), 1),
|
||||
datetime64_col_with_type_and_name.type,
|
||||
string_col_with_type_and_name.name};
|
||||
|
||||
executeDecimal(block, result,
|
||||
which_left.isDateTime64() ? datetime64_col_with_type_and_name : parsed_tmp_col_with_type_and_name,
|
||||
which_right.isDateTime64() ? datetime64_col_with_type_and_name : parsed_tmp_col_with_type_and_name);
|
||||
|
||||
}
|
||||
else if (isColumnedAsDecimal(left_type) || isColumnedAsDecimal(right_type))
|
||||
{
|
||||
@ -1294,19 +1199,10 @@ public:
|
||||
|
||||
executeDecimal(block, result, col_with_type_and_name_left, col_with_type_and_name_right);
|
||||
}
|
||||
else if (!left_is_num && !right_is_num && executeString(block, result, col_left_untyped, col_right_untyped))
|
||||
{
|
||||
}
|
||||
else if (left_type->equals(*right_type))
|
||||
{
|
||||
executeGenericIdenticalTypes(block, result, col_left_untyped, col_right_untyped);
|
||||
}
|
||||
else if (executeDateOrDateTimeOrEnumOrUUIDWithConstString(
|
||||
block, result, col_left_untyped, col_right_untyped,
|
||||
left_type, right_type,
|
||||
left_is_num, input_rows_count))
|
||||
{
|
||||
}
|
||||
else
|
||||
{
|
||||
executeGeneric(block, result, col_with_type_and_name_left, col_with_type_and_name_right);
|
||||
|
@ -722,10 +722,10 @@ struct ConvertThroughParsing
|
||||
parsed = ToDataType::tryReadText(vec_to[i], read_buffer, ToDataType::maxPrecision(), vec_to.getScale());
|
||||
else
|
||||
parsed = tryParseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone);
|
||||
|
||||
parsed = parsed && isAllRead(read_buffer);
|
||||
}
|
||||
|
||||
parsed = parsed && isAllRead(read_buffer);
|
||||
|
||||
if (!parsed)
|
||||
vec_to[i] = 0;
|
||||
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <Common/memcmpSmall.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
|
@ -54,7 +54,7 @@ public:
|
||||
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return false; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
@ -227,7 +227,6 @@ public:
|
||||
row_offset = next_row_offset;
|
||||
}
|
||||
}
|
||||
DUMP(Kind, needle, column_haystack, root_offsets_col, nested_offsets_col);
|
||||
|
||||
ColumnArray::MutablePtr nested_array_col = ColumnArray::create(std::move(data_col), std::move(nested_offsets_col));
|
||||
ColumnArray::MutablePtr root_array_col = ColumnArray::create(std::move(nested_array_col), std::move(root_offsets_col));
|
||||
|
@ -982,7 +982,16 @@ void Context::setSetting(const StringRef & name, const Field & value)
|
||||
|
||||
void Context::applySettingChange(const SettingChange & change)
|
||||
{
|
||||
setSetting(change.name, change.value);
|
||||
try
|
||||
{
|
||||
setSetting(change.name, change.value);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage(fmt::format("in attempt to set the value of setting '{}' to {}",
|
||||
change.name, applyVisitor(FieldVisitorToString(), change.value)));
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
@ -208,7 +208,9 @@ struct ExpressionAnalysisResult
|
||||
const FilterInfoPtr & filter_info,
|
||||
const Block & source_header);
|
||||
|
||||
/// Filter for row-level security.
|
||||
bool hasFilter() const { return filter_info.get(); }
|
||||
|
||||
bool hasJoin() const { return before_join.get(); }
|
||||
bool hasPrewhere() const { return prewhere_info.get(); }
|
||||
bool hasWhere() const { return before_where.get(); }
|
||||
|
@ -1,4 +1,6 @@
|
||||
#include <Interpreters/FillingRow.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
#include <Core/SortDescription.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -37,7 +37,7 @@
|
||||
#include <Core/Field.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Columns/Collator.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <ext/map.h>
|
||||
@ -94,7 +94,8 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
/// Assumes `storage` is set and the table filter (row-level security) is not empty.
|
||||
String InterpreterSelectQuery::generateFilterActions(ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns) const
|
||||
String InterpreterSelectQuery::generateFilterActions(
|
||||
ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns) const
|
||||
{
|
||||
const auto & db_name = table_id.getDatabaseName();
|
||||
const auto & table_name = table_id.getTableName();
|
||||
@ -474,8 +475,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
|
||||
second_stage,
|
||||
options.only_analyze,
|
||||
filter_info,
|
||||
source_header
|
||||
);
|
||||
source_header);
|
||||
|
||||
if (options.to_stage == QueryProcessingStage::Enum::FetchColumns)
|
||||
{
|
||||
@ -979,10 +979,13 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
|
||||
/// Optimization for trivial query like SELECT count() FROM table.
|
||||
bool optimize_trivial_count =
|
||||
syntax_analyzer_result->optimize_trivial_count && storage &&
|
||||
processing_stage == QueryProcessingStage::FetchColumns &&
|
||||
query_analyzer->hasAggregation() && (query_analyzer->aggregates().size() == 1) &&
|
||||
typeid_cast<AggregateFunctionCount *>(query_analyzer->aggregates()[0].function.get());
|
||||
syntax_analyzer_result->optimize_trivial_count
|
||||
&& storage
|
||||
&& !filter_info
|
||||
&& processing_stage == QueryProcessingStage::FetchColumns
|
||||
&& query_analyzer->hasAggregation()
|
||||
&& (query_analyzer->aggregates().size() == 1)
|
||||
&& typeid_cast<AggregateFunctionCount *>(query_analyzer->aggregates()[0].function.get());
|
||||
|
||||
if (optimize_trivial_count)
|
||||
{
|
||||
|
@ -132,7 +132,8 @@ private:
|
||||
void executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, const std::unordered_map<String, SubqueryForSet> & subqueries_for_sets);
|
||||
void executeMergeSorted(QueryPipeline & pipeline, const SortDescription & sort_description, UInt64 limit);
|
||||
|
||||
String generateFilterActions(ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns = {}) const;
|
||||
String generateFilterActions(
|
||||
ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns = {}) const;
|
||||
|
||||
enum class Modificator
|
||||
{
|
||||
@ -159,6 +160,7 @@ private:
|
||||
|
||||
/// Is calculated in getSampleBlock. Is used later in readImpl.
|
||||
ExpressionAnalysisResult analysis_result;
|
||||
/// For row-level security.
|
||||
FilterInfoPtr filter_info;
|
||||
|
||||
QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns;
|
||||
|
@ -33,8 +33,6 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
extern const int TYPE_MISMATCH;
|
||||
extern const int TOO_LARGE_STRING_SIZE;
|
||||
extern const int CANNOT_CONVERT_TYPE;
|
||||
}
|
||||
|
||||
|
||||
@ -124,42 +122,6 @@ static Field convertDecimalType(const Field & from, const To & type)
|
||||
}
|
||||
|
||||
|
||||
DayNum stringToDate(const String & s)
|
||||
{
|
||||
ReadBufferFromString in(s);
|
||||
DayNum date{};
|
||||
|
||||
readDateText(date, in);
|
||||
if (!in.eof())
|
||||
throw Exception("String is too long for Date: " + s, ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||
|
||||
return date;
|
||||
}
|
||||
|
||||
UInt64 stringToDateTime(const String & s)
|
||||
{
|
||||
ReadBufferFromString in(s);
|
||||
time_t date_time{};
|
||||
|
||||
readDateTimeText(date_time, in);
|
||||
if (!in.eof())
|
||||
throw Exception("String is too long for DateTime: " + s, ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||
|
||||
return UInt64(date_time);
|
||||
}
|
||||
|
||||
DateTime64::NativeType stringToDateTime64(const String & s, UInt32 scale)
|
||||
{
|
||||
ReadBufferFromString in(s);
|
||||
DateTime64 datetime64 {0};
|
||||
|
||||
readDateTime64Text(datetime64, scale, in);
|
||||
if (!in.eof())
|
||||
throw Exception("String is too long for DateTime64: " + s, ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||
|
||||
return datetime64.value;
|
||||
}
|
||||
|
||||
Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const IDataType * from_type_hint)
|
||||
{
|
||||
WhichDataType which_type(type);
|
||||
@ -184,7 +146,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
|
||||
{
|
||||
return static_cast<const DataTypeDateTime &>(type).getTimeZone().fromDayNum(DayNum(src.get<UInt64>()));
|
||||
}
|
||||
else if (type.isValueRepresentedByNumber())
|
||||
else if (type.isValueRepresentedByNumber() && src.getType() != Field::Types::String)
|
||||
{
|
||||
if (which_type.isUInt8()) return convertNumericType<UInt8>(src, type);
|
||||
if (which_type.isUInt16()) return convertNumericType<UInt16>(src, type);
|
||||
@ -200,9 +162,6 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
|
||||
if (const auto * ptype = typeid_cast<const DataTypeDecimal<Decimal64> *>(&type)) return convertDecimalType(src, *ptype);
|
||||
if (const auto * ptype = typeid_cast<const DataTypeDecimal<Decimal128> *>(&type)) return convertDecimalType(src, *ptype);
|
||||
|
||||
if (!which_type.isDateOrDateTime() && !which_type.isUUID() && !which_type.isEnum())
|
||||
throw Exception{"Cannot convert field to type " + type.getName(), ErrorCodes::CANNOT_CONVERT_TYPE};
|
||||
|
||||
if (which_type.isEnum() && (src.getType() == Field::Types::UInt64 || src.getType() == Field::Types::Int64))
|
||||
{
|
||||
/// Convert UInt64 or Int64 to Enum's value
|
||||
@ -214,36 +173,20 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
|
||||
/// We don't need any conversion UInt64 is under type of Date and DateTime
|
||||
return src;
|
||||
}
|
||||
// TODO (vnemkov): extra cases for DateTime64: converting from integer, converting from Decimal
|
||||
|
||||
if (src.getType() == Field::Types::String)
|
||||
if (which_type.isUUID() && src.getType() == Field::Types::UInt128)
|
||||
{
|
||||
if (which_type.isDate())
|
||||
{
|
||||
/// Convert 'YYYY-MM-DD' Strings to Date
|
||||
return stringToDate(src.get<const String &>());
|
||||
}
|
||||
else if (which_type.isDateTime())
|
||||
{
|
||||
/// Convert 'YYYY-MM-DD hh:mm:ss' Strings to DateTime
|
||||
return stringToDateTime(src.get<const String &>());
|
||||
}
|
||||
else if (which_type.isDateTime64())
|
||||
{
|
||||
const auto * date_time64 = typeid_cast<const DataTypeDateTime64 *>(&type);
|
||||
/// Convert 'YYYY-MM-DD hh:mm:ss.NNNNNNNNN' Strings to DateTime
|
||||
return stringToDateTime64(src.get<const String &>(), date_time64->getScale());
|
||||
}
|
||||
else if (which_type.isUUID())
|
||||
{
|
||||
return stringToUUID(src.get<const String &>());
|
||||
}
|
||||
else if (which_type.isEnum())
|
||||
{
|
||||
/// Convert String to Enum's value
|
||||
return dynamic_cast<const IDataTypeEnum &>(type).castToValue(src);
|
||||
}
|
||||
/// Already in needed type.
|
||||
return src;
|
||||
}
|
||||
|
||||
if (which_type.isDateTime64() && src.getType() == Field::Types::Decimal64)
|
||||
{
|
||||
/// Already in needed type.
|
||||
return src;
|
||||
}
|
||||
|
||||
/// TODO Conversion from integers to DateTime64
|
||||
}
|
||||
else if (which_type.isStringOrFixedString())
|
||||
{
|
||||
@ -328,17 +271,37 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
|
||||
return src;
|
||||
}
|
||||
|
||||
/// Conversion from string by parsing.
|
||||
if (src.getType() == Field::Types::String)
|
||||
{
|
||||
const auto col = type.createColumn();
|
||||
ReadBufferFromString buffer(src.get<String>());
|
||||
type.deserializeAsTextEscaped(*col, buffer, FormatSettings{});
|
||||
/// Promote data type to avoid overflows. Note that overflows in the largest data type are still possible.
|
||||
const IDataType * type_to_parse = &type;
|
||||
DataTypePtr holder;
|
||||
|
||||
return (*col)[0];
|
||||
if (type.canBePromoted())
|
||||
{
|
||||
holder = type.promoteNumericType();
|
||||
type_to_parse = holder.get();
|
||||
}
|
||||
|
||||
const auto col = type_to_parse->createColumn();
|
||||
ReadBufferFromString in_buffer(src.get<String>());
|
||||
try
|
||||
{
|
||||
type_to_parse->deserializeAsWholeText(*col, in_buffer, FormatSettings{});
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage(fmt::format("while converting '{}' to {}", src.get<String>(), type.getName()));
|
||||
throw;
|
||||
}
|
||||
if (!in_buffer.eof())
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert string {} to type {}", src.get<String>(), type.getName());
|
||||
|
||||
Field parsed = (*col)[0];
|
||||
return convertFieldToType(parsed, type, from_type_hint);
|
||||
}
|
||||
|
||||
|
||||
// TODO (nemkov): should we attempt to parse value using or `type.deserializeAsTextEscaped()` type.deserializeAsTextEscaped() ?
|
||||
throw Exception("Type mismatch in IN or VALUES section. Expected: " + type.getName() + ". Got: "
|
||||
+ Field::Types::toString(src.getType()), ErrorCodes::TYPE_MISMATCH);
|
||||
}
|
||||
|
@ -65,7 +65,8 @@ namespace
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " IDENTIFIED WITH " << authentication_type_name
|
||||
<< (settings.hilite ? IAST::hilite_none : "");
|
||||
if (password)
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " BY " << quoteString(*password);
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " BY " << (settings.hilite ? IAST::hilite_none : "")
|
||||
<< quoteString(*password);
|
||||
}
|
||||
|
||||
|
||||
|
@ -100,9 +100,9 @@ void AggregatingInOrderTransform::consume(Chunk chunk)
|
||||
params->aggregator.createStatesAndFillKeyColumnsWithSingleKey(variants, key_columns, key_begin, res_key_columns);
|
||||
++cur_block_size;
|
||||
}
|
||||
size_t mid = 0;
|
||||
size_t high = 0;
|
||||
size_t low = -1;
|
||||
ssize_t mid = 0;
|
||||
ssize_t high = 0;
|
||||
ssize_t low = -1;
|
||||
/// Will split block into segments with the same key
|
||||
while (key_end != rows)
|
||||
{
|
||||
|
@ -430,6 +430,10 @@ void HTTPHandler::processQuery(
|
||||
|
||||
auto param_could_be_skipped = [&] (const String & name)
|
||||
{
|
||||
/// Empty parameter appears when URL like ?&a=b or a=b&&c=d. Just skip them for user's convenience.
|
||||
if (name.empty())
|
||||
return true;
|
||||
|
||||
if (reserved_param_names.count(name))
|
||||
return true;
|
||||
|
||||
|
@ -1,9 +1,6 @@
|
||||
#include "HTTPHandlerFactory.h"
|
||||
|
||||
#include <re2/re2.h>
|
||||
#include <re2/stringpiece.h>
|
||||
#include <common/find_symbols.h>
|
||||
#include <Poco/StringTokenizer.h>
|
||||
#include <Poco/Util/LayeredConfiguration.h>
|
||||
|
||||
#include "HTTPHandler.h"
|
||||
|
@ -7,7 +7,7 @@
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Interpreters/Set.h>
|
||||
@ -826,8 +826,8 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, const Context & cont
|
||||
}
|
||||
|
||||
bool cast_not_needed =
|
||||
is_set_const /// Set args are already casted inside Set::createFromAST
|
||||
|| (isNativeNumber(key_expr_type) && isNativeNumber(const_type)); /// Numbers are accurately compared without cast.
|
||||
is_set_const /// Set args are already casted inside Set::createFromAST
|
||||
|| (isNativeNumber(key_expr_type) && isNativeNumber(const_type)); /// Numbers are accurately compared without cast.
|
||||
|
||||
if (!cast_not_needed)
|
||||
castValueToType(key_expr_type, const_value, const_type, node);
|
||||
|
@ -905,7 +905,9 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r
|
||||
if (prewhere->remove_prewhere_column)
|
||||
result.columns.erase(result.columns.begin() + prewhere_column_pos);
|
||||
else
|
||||
result.columns[prewhere_column_pos] = DataTypeUInt8().createColumnConst(result.num_rows, 1u)->convertToFullColumnIfConst();
|
||||
result.columns[prewhere_column_pos] =
|
||||
getSampleBlock().getByName(prewhere->prewhere_column_name).type->
|
||||
createColumnConst(result.num_rows, 1u)->convertToFullColumnIfConst();
|
||||
}
|
||||
}
|
||||
/// Filter in WHERE instead
|
||||
|
@ -198,7 +198,8 @@ void ReplicatedMergeTreeBlockOutputStream::writeExistingPart(MergeTreeData::Muta
|
||||
}
|
||||
|
||||
|
||||
void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id)
|
||||
void ReplicatedMergeTreeBlockOutputStream::commitPart(
|
||||
zkutil::ZooKeeperPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id)
|
||||
{
|
||||
storage.check(part->getColumns());
|
||||
assertSessionIsNotExpired(zookeeper);
|
||||
|
@ -244,6 +244,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
|
||||
{
|
||||
LOG_WARNING(log, "No metadata in ZooKeeper: table will be in readonly mode.");
|
||||
is_readonly = true;
|
||||
has_metadata_in_zookeeper = false;
|
||||
return;
|
||||
}
|
||||
|
||||
@ -620,9 +621,14 @@ void StorageReplicatedMergeTree::createReplica()
|
||||
|
||||
void StorageReplicatedMergeTree::drop()
|
||||
{
|
||||
/// There is also the case when user has configured ClickHouse to wrong ZooKeeper cluster,
|
||||
/// in this case, has_metadata_in_zookeeper = false, and we also permit to drop the table.
|
||||
|
||||
if (has_metadata_in_zookeeper)
|
||||
{
|
||||
auto zookeeper = tryGetZooKeeper();
|
||||
|
||||
/// If probably there is metadata in ZooKeeper, we don't allow to drop the table.
|
||||
if (is_readonly || !zookeeper)
|
||||
throw Exception("Can't drop readonly replicated table (need to drop data in ZooKeeper as well)", ErrorCodes::TABLE_IS_READ_ONLY);
|
||||
|
||||
@ -4049,8 +4055,20 @@ void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, c
|
||||
MergeTreeData::rename(new_path_to_table_data, new_table_id);
|
||||
|
||||
/// Update table name in zookeeper
|
||||
auto zookeeper = getZooKeeper();
|
||||
zookeeper->set(replica_path + "/host", getReplicatedMergeTreeAddress().toString());
|
||||
if (!is_readonly)
|
||||
{
|
||||
/// We don't do it for readonly tables, because it will be updated on next table startup.
|
||||
/// It is also Ok to skip ZK error for the same reason.
|
||||
try
|
||||
{
|
||||
auto zookeeper = getZooKeeper();
|
||||
zookeeper->set(replica_path + "/host", getReplicatedMergeTreeAddress().toString());
|
||||
}
|
||||
catch (Coordination::Exception & e)
|
||||
{
|
||||
LOG_WARNING(log, "Cannot update the value of 'host' node (replica address) in ZooKeeper: {}", e.displayText());
|
||||
}
|
||||
}
|
||||
|
||||
/// TODO: You can update names of loggers.
|
||||
}
|
||||
|
@ -210,6 +210,8 @@ private:
|
||||
|
||||
/// If true, the table is offline and can not be written to it.
|
||||
std::atomic_bool is_readonly {false};
|
||||
/// If false - ZooKeeper is available, but there is no table metadata. It's safe to drop table in this case.
|
||||
bool has_metadata_in_zookeeper = true;
|
||||
|
||||
String zookeeper_path;
|
||||
String replica_name;
|
||||
|
@ -32,6 +32,7 @@ StorageSystemColumns::StorageSystemColumns(const std::string & name_)
|
||||
{ "table", std::make_shared<DataTypeString>() },
|
||||
{ "name", std::make_shared<DataTypeString>() },
|
||||
{ "type", std::make_shared<DataTypeString>() },
|
||||
{ "position", std::make_shared<DataTypeUInt64>() },
|
||||
{ "default_kind", std::make_shared<DataTypeString>() },
|
||||
{ "default_expression", std::make_shared<DataTypeString>() },
|
||||
{ "data_compressed_bytes", std::make_shared<DataTypeUInt64>() },
|
||||
@ -131,8 +132,10 @@ protected:
|
||||
|
||||
bool check_access_for_columns = check_access_for_tables && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name);
|
||||
|
||||
size_t position = 0;
|
||||
for (const auto & column : columns)
|
||||
{
|
||||
++position;
|
||||
if (check_access_for_columns && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name, column.name))
|
||||
continue;
|
||||
|
||||
@ -147,6 +150,8 @@ protected:
|
||||
res_columns[res_index++]->insert(column.name);
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(column.type->getName());
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(position);
|
||||
|
||||
if (column.default_desc.expression)
|
||||
{
|
||||
|
@ -49,6 +49,7 @@ StorageSystemPartsColumns::StorageSystemPartsColumns(const std::string & name_)
|
||||
|
||||
{"column", std::make_shared<DataTypeString>()},
|
||||
{"type", std::make_shared<DataTypeString>()},
|
||||
{"column_position", std::make_shared<DataTypeUInt64>()},
|
||||
{"default_kind", std::make_shared<DataTypeString>()},
|
||||
{"default_expression", std::make_shared<DataTypeString>()},
|
||||
{"column_bytes_on_disk", std::make_shared<DataTypeUInt64>()},
|
||||
@ -101,9 +102,10 @@ void StorageSystemPartsColumns::processNextStorage(MutableColumns & columns_, co
|
||||
|
||||
using State = IMergeTreeDataPart::State;
|
||||
|
||||
size_t column_position = 0;
|
||||
for (const auto & column : part->getColumns())
|
||||
|
||||
{
|
||||
++column_position;
|
||||
size_t j = 0;
|
||||
{
|
||||
WriteBufferFromOwnString out;
|
||||
@ -143,6 +145,7 @@ void StorageSystemPartsColumns::processNextStorage(MutableColumns & columns_, co
|
||||
|
||||
columns_[j++]->insert(column.name);
|
||||
columns_[j++]->insert(column.type->getName());
|
||||
columns_[j++]->insert(column_position);
|
||||
|
||||
auto column_info_it = columns_info.find(column.name);
|
||||
if (column_info_it != columns_info.end())
|
||||
|
@ -13,7 +13,7 @@
|
||||
11 Feb 2018 06:40:50 +0300 2018-02-11 03:40:50 2018-02-11 03:40:50
|
||||
17 Apr 2 1:2:3 2000-04-17 01:02:03 2000-04-17 01:02:03
|
||||
19700102 01:00:00 1970-01-02 01:00:00 1970-01-02 01:00:00
|
||||
1970010201:00:00 2032-06-06 02:03:21 2032-06-06 02:03:21
|
||||
1970010201:00:00 ᴺᵁᴸᴸ 0000-00-00 00:00:00
|
||||
19700102010203 1970-01-02 01:02:03 1970-01-02 01:02:03
|
||||
19700102010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03
|
||||
1970/01/02 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03
|
||||
@ -61,7 +61,7 @@
|
||||
2017/01/32 0000-00-00 00:00:00 0000-00-00 00:00:00
|
||||
2017-01 MSD Jun 2017-05-31 20:00:00 2017-05-31 20:00:00
|
||||
201701 MSD Jun 2017-05-31 20:00:00 2017-05-31 20:00:00
|
||||
2017 25 1:2:3 0000-00-00 00:00:00 0000-00-00 00:00:00
|
||||
2017 25 1:2:3 ᴺᵁᴸᴸ 0000-00-00 00:00:00
|
||||
2017 25 Apr 1:2:3 2017-04-01 01:02:03 2017-04-01 01:02:03
|
||||
2017 Apr 01 11:22:33 2017-04-01 11:22:33 2017-04-01 11:22:33
|
||||
2017 Apr 02 01/02/03 UTC+0300 ᴺᵁᴸᴸ 0000-00-00 00:00:00
|
||||
@ -74,8 +74,8 @@
|
||||
2017 Apr 02 1:2:3 2017-04-02 01:02:03 2017-04-02 01:02:03
|
||||
2017 Apr 02 1:2:33 2017-04-02 01:02:33 2017-04-02 01:02:33
|
||||
2017 Apr 02 1:2:3 MSK 2017-04-01 22:02:03 2017-04-01 22:02:03
|
||||
2017 Apr 02 1:2:3 MSK 2017 2017-04-01 22:02:03 2017-04-01 22:02:03
|
||||
2017 Apr 02 1:2:3 MSK 2018 2017-04-01 22:02:03 2017-04-01 22:02:03
|
||||
2017 Apr 02 1:2:3 MSK 2017 ᴺᵁᴸᴸ 0000-00-00 00:00:00
|
||||
2017 Apr 02 1:2:3 MSK 2018 ᴺᵁᴸᴸ 0000-00-00 00:00:00
|
||||
2017 Apr 02 1:2:3 UTC+0000 2017-04-02 01:02:03 2017-04-02 01:02:03
|
||||
2017 Apr 02 1:2:3 UTC+0300 2017-04-01 22:02:03 2017-04-01 22:02:03
|
||||
2017 Apr 02 1:2:3 UTC+0400 2017-04-01 21:02:03 2017-04-01 21:02:03
|
||||
@ -101,6 +101,6 @@
|
||||
25 Jan 2017 1:2:3 Z PM 2017-01-25 13:02:03 2017-01-25 13:02:03
|
||||
25 Jan 2017 1:2:3Z PM 2017-01-25 13:02:03 2017-01-25 13:02:03
|
||||
25 Jan 2017 1:2:3 Z PM +03:00 2017-01-25 10:02:03 2017-01-25 10:02:03
|
||||
Jun, 11 Feb 2018 06:40:50 +0300 2000-06-01 00:00:00 2000-06-01 00:00:00
|
||||
Jun, 11 Feb 2018 06:40:50 +0300 ᴺᵁᴸᴸ 0000-00-00 00:00:00
|
||||
Sun 11 Feb 2018 06:40:50 +0300 2018-02-11 03:40:50 2018-02-11 03:40:50
|
||||
Sun, 11 Feb 2018 06:40:50 +0300 2018-02-11 03:40:50 2018-02-11 03:40:50
|
||||
|
@ -4,7 +4,7 @@ DROP TABLE IF EXISTS numbers2;
|
||||
CREATE TABLE numbers1 ENGINE = Memory AS SELECT number as _table FROM numbers(1000);
|
||||
CREATE TABLE numbers2 ENGINE = Memory AS SELECT number as _table FROM numbers(1000);
|
||||
|
||||
SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table='numbers1'; -- { serverError 43 }
|
||||
SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table='numbers1'; -- { serverError 53 }
|
||||
SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table=1;
|
||||
|
||||
DROP TABLE numbers1;
|
||||
|
@ -1,3 +1,5 @@
|
||||
0
|
||||
1
|
||||
-42 -42 1 0 0 0 1 1
|
||||
42 42 1 0 0 0 1 1
|
||||
-42 -42.42000 0 0 1 1 0 1
|
||||
|
@ -19,7 +19,7 @@ INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-42, -42, -42, -0.42,
|
||||
|
||||
SELECT a > toFloat64(0) FROM decimal; -- { serverError 43 }
|
||||
SELECT g > toFloat32(0) FROM decimal; -- { serverError 43 }
|
||||
SELECT a > '0.0' FROM decimal; -- { serverError 43 }
|
||||
SELECT a > '0.0' FROM decimal ORDER BY a;
|
||||
|
||||
SELECT a, b, a = b, a < b, a > b, a != b, a <= b, a >= b FROM decimal ORDER BY a;
|
||||
SELECT a, g, a = g, a < g, a > g, a != g, a <= g, a >= g FROM decimal ORDER BY a;
|
||||
|
@ -13,7 +13,7 @@ $CLICKHOUSE_CLIENT -q "select name from system.table_functions format TSV;" > $S
|
||||
# if you want long run use: env SQL_FUZZY_RUNS=100000 clickhouse-test sql_fuzzy
|
||||
|
||||
for SQL_FUZZY_RUN in $(seq ${SQL_FUZZY_RUNS:=10}); do
|
||||
env SQL_FUZZY_RUN=$SQL_FUZZY_RUN $CURDIR/00746_sql_fuzzy.pl | $CLICKHOUSE_CLIENT --max_execution_time 10 -n --ignore-error >/dev/null 2>&1
|
||||
env SQL_FUZZY_RUN=$SQL_FUZZY_RUN $CURDIR/00746_sql_fuzzy.pl | $CLICKHOUSE_CLIENT --format Null --max_execution_time 10 -n --ignore-error >/dev/null 2>&1
|
||||
if [[ `$CLICKHOUSE_CLIENT -q "SELECT 'Still alive'"` != 'Still alive' ]]; then
|
||||
break
|
||||
fi
|
||||
|
@ -1 +1 @@
|
||||
default merge_ab x UInt8 0 0 0 0 0 0 0
|
||||
x
|
||||
|
@ -6,7 +6,7 @@ CREATE TABLE merge_a (x UInt8) ENGINE = StripeLog;
|
||||
CREATE TABLE merge_b (x UInt8) ENGINE = StripeLog;
|
||||
CREATE TABLE merge_ab AS merge(currentDatabase(), '^merge_[ab]$');
|
||||
|
||||
SELECT * FROM system.columns WHERE database = currentDatabase() AND table = 'merge_ab';
|
||||
SELECT name FROM system.columns WHERE database = currentDatabase() AND table = 'merge_ab';
|
||||
|
||||
DROP TABLE merge_a;
|
||||
DROP TABLE merge_b;
|
||||
|
@ -1,12 +1,12 @@
|
||||
-- Error cases:
|
||||
-- non-const string column
|
||||
WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT DT64 = materialize(S); -- {serverError 44}
|
||||
WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT materialize(S) = toDateTime64(S, 3); -- {serverError 44}
|
||||
WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT * WHERE DT64 = materialize(S); -- {serverError 44}
|
||||
WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT * WHERE materialize(S) = DT64; -- {serverError 44}
|
||||
WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT DT64 = materialize(S); -- {serverError 43}
|
||||
WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT materialize(S) = toDateTime64(S, 3); -- {serverError 43}
|
||||
WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT * WHERE DT64 = materialize(S); -- {serverError 43}
|
||||
WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT * WHERE materialize(S) = DT64; -- {serverError 43}
|
||||
|
||||
SELECT * WHERE toDateTime64(123.345, 3) == 'ABCD'; -- {serverError 131} -- invalid DateTime64 string
|
||||
SELECT * WHERE toDateTime64(123.345, 3) == '2020-02-05 14:34:12.33333333333333333333333333333333333333333333333333333333'; -- {serverError 131} -- invalid string length
|
||||
SELECT * WHERE toDateTime64(123.345, 3) == 'ABCD'; -- {serverError 53} -- invalid DateTime64 string
|
||||
SELECT * WHERE toDateTime64(123.345, 3) == '2020-02-05 14:34:12.33333333333333333333333333333333333333333333333333333333'; -- {serverError 53} -- invalid string length
|
||||
|
||||
SELECT 'in SELECT';
|
||||
WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT DT64 = S;
|
||||
|
@ -1 +1 @@
|
||||
SELECT -2487, globalNullIn(toIntervalMinute(-88074), 'qEkek..'), [-27.537293]; -- { serverError 70 }
|
||||
SELECT -2487, globalNullIn(toIntervalMinute(-88074), 'qEkek..'), [-27.537293]; -- { serverError 53 }
|
||||
|
@ -0,0 +1 @@
|
||||
some_field_value 1
|
@ -0,0 +1,5 @@
|
||||
drop table if exists data;
|
||||
CREATE TABLE data (ts DateTime, field String, num_field Nullable(Float64)) ENGINE = MergeTree() PARTITION BY ts ORDER BY ts;
|
||||
insert into data values(toDateTime('2020-05-14 02:08:00'),'some_field_value',7.);
|
||||
SELECT field, countIf(num_field > 6.0) FROM data PREWHERE (num_field>6.0) GROUP BY field;
|
||||
drop table if exists data;
|
@ -1,2 +1,2 @@
|
||||
2000 1999000
|
||||
2000 1999000
|
||||
400 79800
|
||||
400 79800
|
||||
|
@ -22,8 +22,8 @@ function thread()
|
||||
}
|
||||
|
||||
|
||||
thread 0 1000 &
|
||||
thread 1 1000 &
|
||||
thread 0 200 &
|
||||
thread 1 200 &
|
||||
|
||||
wait
|
||||
|
||||
|
@ -0,0 +1,3 @@
|
||||
3
|
||||
2
|
||||
3
|
@ -0,0 +1,12 @@
|
||||
DROP TABLE IF EXISTS t;
|
||||
|
||||
CREATE TABLE t (x UInt8) ENGINE = MergeTree ORDER BY x;
|
||||
INSERT INTO t VALUES (1), (2), (3);
|
||||
|
||||
SELECT count() FROM t;
|
||||
CREATE ROW POLICY filter ON t USING (x % 2 = 1) TO ALL;
|
||||
SELECT count() FROM t;
|
||||
DROP ROW POLICY filter ON t;
|
||||
SELECT count() FROM t;
|
||||
|
||||
DROP TABLE t;
|
@ -0,0 +1,2 @@
|
||||
1
|
||||
0
|
6
tests/queries/0_stateless/01310_enum_comparison.sql
Normal file
6
tests/queries/0_stateless/01310_enum_comparison.sql
Normal file
@ -0,0 +1,6 @@
|
||||
CREATE TEMPORARY TABLE enum (x Enum('hello' = 1, 'world' = 2));
|
||||
INSERT INTO enum VALUES ('hello');
|
||||
|
||||
SELECT count() FROM enum WHERE x = 'hello';
|
||||
SELECT count() FROM enum WHERE x = 'world';
|
||||
SELECT count() FROM enum WHERE x = 'xyz'; -- { serverError 36 }
|
@ -0,0 +1,32 @@
|
||||
0
|
||||
1
|
||||
0
|
||||
---
|
||||
1
|
||||
0
|
||||
1
|
||||
---
|
||||
1
|
||||
0
|
||||
0
|
||||
---
|
||||
0
|
||||
---
|
||||
1
|
||||
0
|
||||
---
|
||||
---
|
||||
0
|
||||
---
|
||||
1
|
||||
---
|
||||
0
|
||||
---
|
||||
0
|
||||
---
|
||||
0
|
||||
---
|
||||
0
|
||||
---
|
||||
1
|
||||
---
|
@ -0,0 +1,33 @@
|
||||
SELECT number = '1' FROM numbers(3);
|
||||
SELECT '---';
|
||||
SELECT '1' != number FROM numbers(3);
|
||||
SELECT '---';
|
||||
SELECT '1' > number FROM numbers(3);
|
||||
SELECT '---';
|
||||
SELECT 1 = '257';
|
||||
SELECT '---';
|
||||
SELECT 1 IN (1.23, '1', 2);
|
||||
SELECT 1 IN (1.23, '2', 2);
|
||||
SELECT '---';
|
||||
|
||||
-- it should work but it doesn't.
|
||||
SELECT 1 = '1.0'; -- { serverError 53 }
|
||||
SELECT '---';
|
||||
|
||||
SELECT 1 = '257';
|
||||
SELECT '---';
|
||||
SELECT 1 != '257';
|
||||
SELECT '---';
|
||||
SELECT 1 < '257'; -- this is wrong for now
|
||||
SELECT '---';
|
||||
SELECT 1 > '257';
|
||||
SELECT '---';
|
||||
SELECT 1 <= '257'; -- this is wrong for now
|
||||
SELECT '---';
|
||||
SELECT 1 >= '257';
|
||||
SELECT '---';
|
||||
|
||||
SELECT toDateTime('2020-06-13 01:02:03') = '2020-06-13T01:02:03';
|
||||
SELECT '---';
|
||||
|
||||
SELECT 0 = ''; -- { serverError 32 }
|
@ -0,0 +1,8 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
@ -0,0 +1,8 @@
|
||||
SELECT match('Too late', 'Too late');
|
||||
select match('Too late', '(?i)Too late');
|
||||
select match('Too late', '(?i)too late');
|
||||
select match('Too late', '(?i:too late)');
|
||||
select match('Too late', '(?i)to{2} late');
|
||||
select match('Too late', '(?i)to(?)o late');
|
||||
select match('Too late', '(?i)to+ late');
|
||||
select match('Too late', '(?i)to(?:o|o) late');
|
@ -0,0 +1,12 @@
|
||||
1
|
||||
999999
|
||||
100000
|
||||
899999
|
||||
100001
|
||||
900000
|
||||
1
|
||||
999999
|
||||
100000
|
||||
899999
|
||||
100001
|
||||
900000
|
@ -0,0 +1,32 @@
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS index_granularity = 1000;
|
||||
INSERT INTO test SELECT * FROM numbers(1000000);
|
||||
OPTIMIZE TABLE test;
|
||||
|
||||
SET max_rows_to_read = 2000;
|
||||
SELECT count() FROM test WHERE x = 100000;
|
||||
SET max_rows_to_read = 1000000;
|
||||
SELECT count() FROM test WHERE x != 100000;
|
||||
SET max_rows_to_read = 101000;
|
||||
SELECT count() FROM test WHERE x < 100000;
|
||||
SET max_rows_to_read = 900000;
|
||||
SELECT count() FROM test WHERE x > 100000;
|
||||
SET max_rows_to_read = 101000;
|
||||
SELECT count() FROM test WHERE x <= 100000;
|
||||
SET max_rows_to_read = 901000;
|
||||
SELECT count() FROM test WHERE x >= 100000;
|
||||
|
||||
SET max_rows_to_read = 2000;
|
||||
SELECT count() FROM test WHERE x = '100000';
|
||||
SET max_rows_to_read = 1000000;
|
||||
SELECT count() FROM test WHERE x != '100000';
|
||||
SET max_rows_to_read = 101000;
|
||||
SELECT count() FROM test WHERE x < '100000';
|
||||
SET max_rows_to_read = 900000;
|
||||
SELECT count() FROM test WHERE x > '100000';
|
||||
SET max_rows_to_read = 101000;
|
||||
SELECT count() FROM test WHERE x <= '100000';
|
||||
SET max_rows_to_read = 901000;
|
||||
SELECT count() FROM test WHERE x >= '100000';
|
||||
|
||||
DROP TABLE test;
|
@ -0,0 +1,3 @@
|
||||
1
|
||||
1
|
||||
1
|
10
tests/queries/0_stateless/01312_skip_empty_params.sh
Executable file
10
tests/queries/0_stateless/01312_skip_empty_params.sh
Executable file
@ -0,0 +1,10 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
set -e
|
||||
|
||||
$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query=select%201&log_queries=1"
|
||||
$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&&query=select%201&log_queries=1"
|
||||
$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query=select%201&&&log_queries=1"
|
@ -0,0 +1,6 @@
|
||||
\N
|
||||
0000-00-00 00:00:00
|
||||
\N
|
||||
0000-00-00 00:00:00.000
|
||||
\N
|
||||
0000-00-00 00:00:00
|
@ -0,0 +1,12 @@
|
||||
SELECT parseDateTimeBestEffort('<Empty>'); -- { serverError 6 }
|
||||
SELECT parseDateTimeBestEffortOrNull('<Empty>');
|
||||
SELECT parseDateTimeBestEffortOrZero('<Empty>');
|
||||
|
||||
SELECT parseDateTime64BestEffort('<Empty>'); -- { serverError 6 }
|
||||
SELECT parseDateTime64BestEffortOrNull('<Empty>');
|
||||
SELECT parseDateTime64BestEffortOrZero('<Empty>');
|
||||
|
||||
SET date_time_input_format = 'best_effort';
|
||||
SELECT toDateTime('<Empty>'); -- { serverError 41 }
|
||||
SELECT toDateTimeOrNull('<Empty>');
|
||||
SELECT toDateTimeOrZero('<Empty>');
|
@ -0,0 +1,6 @@
|
||||
x UInt8 1
|
||||
y String 2
|
||||
z Array(String) 3
|
||||
x UInt8 1
|
||||
y String 2
|
||||
z Array(String) 3
|
@ -0,0 +1,8 @@
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (x UInt8, y String, z Array(String)) ENGINE = MergeTree ORDER BY tuple();
|
||||
INSERT INTO test (x) VALUES (1);
|
||||
|
||||
SELECT name, type, position FROM system.columns WHERE database = currentDatabase() AND table = 'test';
|
||||
SELECT column, type, column_position FROM system.parts_columns WHERE database = currentDatabase() AND table = 'test';
|
||||
|
||||
DROP TABLE test;
|
@ -0,0 +1,9 @@
|
||||
0
|
||||
0
|
||||
0
|
||||
5
|
||||
5
|
||||
5
|
||||
0
|
||||
\N
|
||||
\N
|
@ -0,0 +1,12 @@
|
||||
SELECT uniq(number >= 10 ? number : NULL) FROM numbers(10);
|
||||
SELECT uniqExact(number >= 10 ? number : NULL) FROM numbers(10);
|
||||
SELECT count(DISTINCT number >= 10 ? number : NULL) FROM numbers(10);
|
||||
|
||||
SELECT uniq(number >= 5 ? number : NULL) FROM numbers(10);
|
||||
SELECT uniqExact(number >= 5 ? number : NULL) FROM numbers(10);
|
||||
SELECT count(DISTINCT number >= 5 ? number : NULL) FROM numbers(10);
|
||||
|
||||
SELECT count(NULL);
|
||||
-- These two returns NULL for now, but we want to change them to return 0.
|
||||
SELECT uniq(NULL);
|
||||
SELECT count(DISTINCT NULL);
|
@ -0,0 +1 @@
|
||||
[1mCREATE USER[0m user[1m IDENTIFIED WITH plaintext_password[0m[1m BY [0m'hello'
|
8
tests/queries/0_stateless/01316_create_user_syntax_hilite.sh
Executable file
8
tests/queries/0_stateless/01316_create_user_syntax_hilite.sh
Executable file
@ -0,0 +1,8 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
set -e
|
||||
|
||||
$CLICKHOUSE_FORMAT --hilite <<< "CREATE USER user IDENTIFIED WITH PLAINTEXT_PASSWORD BY 'hello'"
|
@ -0,0 +1,2 @@
|
||||
0
|
||||
0
|
23
tests/queries/0_stateless/01317_no_password_in_command_line.sh
Executable file
23
tests/queries/0_stateless/01317_no_password_in_command_line.sh
Executable file
@ -0,0 +1,23 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
set -e
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP USER IF EXISTS user"
|
||||
$CLICKHOUSE_CLIENT --query "CREATE USER user IDENTIFIED WITH PLAINTEXT_PASSWORD BY 'hello'"
|
||||
|
||||
# False positive result due to race condition with sleeps is Ok.
|
||||
|
||||
$CLICKHOUSE_CLIENT --user user --password hello --query "SELECT sleep(1)" &
|
||||
sleep 0.1
|
||||
ps auxw | grep -F -- '--password' | grep -F hello ||:
|
||||
wait
|
||||
|
||||
$CLICKHOUSE_CLIENT --user user --password=hello --query "SELECT sleep(1)" &
|
||||
sleep 0.1
|
||||
ps auxw | grep -F -- '--password' | grep -F hello ||:
|
||||
wait
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP USER user"
|
@ -6,7 +6,7 @@
|
||||
<p class="lead">ClickHouse's performance <a href="benchmark/dbms/">exceeds</a> comparable column-oriented database management systems currently available
|
||||
on the market. It processes hundreds of millions to more than a billion rows and tens of gigabytes of data per single server per second.</p>
|
||||
|
||||
<a href="benchmark/dbms/" class="btn btn-lg btn-yellow mt-3 mx-auto" role="button">Detailed comparison</a>
|
||||
<a href="benchmark/dbms/" class="btn btn-lg btn-yellow mt-3 mx-auto" role="button" rel="nofollow">Detailed comparison</a>
|
||||
</div>
|
||||
</div>
|
||||
<div class="row mt-5">
|
||||
|
Loading…
Reference in New Issue
Block a user