提交 bc9a3c04 编写于 作者: A alesapin

Merge branch 'master' into consistent_metadata4

......@@ -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);
}
};
......
<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>
......
#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;
};
......
#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.
*/
......
#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;
......
#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="));
}
}
}
#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,92 +812,49 @@ 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())
{
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);
}
else if (which.isDateTime())
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())
{
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);
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.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count,
std::is_same_v<Op<int, int>, NotEqualsOp<int, int>>);
}
else if (which.isUUID())
else
{
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);
auto column_converted = type_to_compare->createColumnConst(input_rows_count, converted);
return true;
}
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)
};
/// 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);
executeImpl(tmp_block, {0, 1}, 2, input_rows_count);
const Field x = castToNearestFieldType(type->getValue(column_string->getValue<String>()));
const auto enum_col = type->createColumnConst(input_rows_count, x);
block.getByPosition(result).column = std::move(tmp_block.getByPosition(2).column);
}
executeNumLeftType<typename EnumType::FieldType>(block, result,
left_is_num ? column_number : enum_col.get(),
left_is_num ? enum_col.get() : column_number);
return true;
}
void executeTuple(Block & block, size_t result, const ColumnWithTypeAndName & c0, const ColumnWithTypeAndName & c1,
......@@ -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(); }
......
#include <Interpreters/FillingRow.h>
#include <Common/FieldVisitorsAccurateComparison.h>
namespace DB
{
......
#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;
......
#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;
......
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
......
......@@ -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;
......
-- 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;
......
SELECT -2487, globalNullIn(toIntervalMinute(-88074), 'qEkek..'), [-27.537293]; -- { serverError 70 }
SELECT -2487, globalNullIn(toIntervalMinute(-88074), 'qEkek..'), [-27.537293]; -- { serverError 53 }
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;
......@@ -22,8 +22,8 @@ function thread()
}
thread 0 1000 &
thread 1 1000 &
thread 0 200 &
thread 1 200 &
wait
......
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;
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
1
0
---
1
0
1
---
1
0
0
---
0
---
1
0
---
---
0
---
1
---
0
---
0
---
0
---
0
---
1
---
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 }
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');
1
999999
100000
899999
100001
900000
1
999999
100000
899999
100001
900000
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;
#!/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"
\N
0000-00-00 00:00:00
\N
0000-00-00 00:00:00.000
\N
0000-00-00 00:00:00
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>');
x UInt8 1
y String 2
z Array(String) 3
x UInt8 1
y String 2
z Array(String) 3
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;
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);
CREATE USER user IDENTIFIED WITH plaintext_password BY 'hello'
#!/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'"
#!/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">
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册