mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-20 14:42:02 +00:00
Merge branch 'master' into fix-merge-sample-by
This commit is contained in:
commit
95c1bb977d
@ -24,7 +24,7 @@ Returns a pseudo-random UInt64 number, evenly distributed among all UInt64-type
|
||||
|
||||
Uses a linear congruential generator.
|
||||
|
||||
## canonicalRand
|
||||
## randCanonical
|
||||
The function generates pseudo random results with independent and identically distributed uniformly distributed values in [0, 1).
|
||||
|
||||
Non-deterministic. Return type is Float64.
|
||||
|
@ -29,6 +29,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int TOO_LARGE_STRING_SIZE;
|
||||
}
|
||||
|
||||
/** Aggregate functions that store one of passed values.
|
||||
@ -521,7 +522,11 @@ public:
|
||||
{
|
||||
if (capacity < rhs_size)
|
||||
{
|
||||
capacity = static_cast<UInt32>(roundUpToPowerOfTwoOrZero(rhs_size));
|
||||
capacity = static_cast<Int32>(roundUpToPowerOfTwoOrZero(rhs_size));
|
||||
/// It might happen if the size was too big and the rounded value does not fit a size_t
|
||||
if (unlikely(capacity < rhs_size))
|
||||
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "String size is too big ({})", rhs_size);
|
||||
|
||||
/// Don't free large_data here.
|
||||
large_data = arena->alloc(capacity);
|
||||
}
|
||||
|
@ -202,7 +202,7 @@ public:
|
||||
auto & merged_maps = this->data(place).merged_maps;
|
||||
for (size_t col = 0, size = values_types.size(); col < size; ++col)
|
||||
{
|
||||
const auto & array_column = assert_cast<const ColumnArray&>(*columns[col + 1]);
|
||||
const auto & array_column = assert_cast<const ColumnArray &>(*columns[col + 1]);
|
||||
const IColumn & value_column = array_column.getData();
|
||||
const IColumn::Offsets & offsets = array_column.getOffsets();
|
||||
const size_t values_vec_offset = offsets[row_num - 1];
|
||||
@ -532,7 +532,12 @@ private:
|
||||
public:
|
||||
explicit FieldVisitorMax(const Field & rhs_) : rhs(rhs_) {}
|
||||
|
||||
bool operator() (Null &) const { throw Exception("Cannot compare Nulls", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (Null &) const
|
||||
{
|
||||
/// Do not update current value, skip nulls
|
||||
return false;
|
||||
}
|
||||
|
||||
bool operator() (AggregateFunctionStateData &) const { throw Exception("Cannot compare AggregateFunctionStates", ErrorCodes::LOGICAL_ERROR); }
|
||||
|
||||
bool operator() (Array & x) const { return compareImpl<Array>(x); }
|
||||
@ -567,7 +572,13 @@ private:
|
||||
public:
|
||||
explicit FieldVisitorMin(const Field & rhs_) : rhs(rhs_) {}
|
||||
|
||||
bool operator() (Null &) const { throw Exception("Cannot compare Nulls", ErrorCodes::LOGICAL_ERROR); }
|
||||
|
||||
bool operator() (Null &) const
|
||||
{
|
||||
/// Do not update current value, skip nulls
|
||||
return false;
|
||||
}
|
||||
|
||||
bool operator() (AggregateFunctionStateData &) const { throw Exception("Cannot sum AggregateFunctionStates", ErrorCodes::LOGICAL_ERROR); }
|
||||
|
||||
bool operator() (Array & x) const { return compareImpl<Array>(x); }
|
||||
|
@ -1517,6 +1517,7 @@ void QueryAnalyzer::collectScopeValidIdentifiersForTypoCorrection(
|
||||
{
|
||||
for (const auto & [name, expression] : scope.alias_name_to_expression_node)
|
||||
{
|
||||
assert(expression);
|
||||
auto expression_identifier = Identifier(name);
|
||||
valid_identifiers_result.insert(expression_identifier);
|
||||
|
||||
@ -2170,6 +2171,18 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier
|
||||
auto & alias_identifier_node = it->second->as<IdentifierNode &>();
|
||||
auto identifier = alias_identifier_node.getIdentifier();
|
||||
auto lookup_result = tryResolveIdentifier(IdentifierLookup{identifier, identifier_lookup.lookup_context}, scope, identifier_resolve_settings);
|
||||
if (!lookup_result.isResolved())
|
||||
{
|
||||
std::unordered_set<Identifier> valid_identifiers;
|
||||
collectScopeWithParentScopesValidIdentifiersForTypoCorrection(identifier, scope, true, false, false, valid_identifiers);
|
||||
|
||||
auto hints = collectIdentifierTypoHints(identifier, valid_identifiers);
|
||||
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown {} identifier '{}' in scope {}{}",
|
||||
toStringLowercase(IdentifierLookupContext::EXPRESSION),
|
||||
identifier.getFullName(),
|
||||
scope.scope_node->formatASTForErrorMessage(),
|
||||
getHintsErrorMessageSuffix(hints));
|
||||
}
|
||||
it->second = lookup_result.resolved_identifier;
|
||||
|
||||
/** During collection of aliases if node is identifier and has alias, we cannot say if it is
|
||||
|
@ -141,7 +141,7 @@ public:
|
||||
/// Get piece of memory, without alignment.
|
||||
char * alloc(size_t size)
|
||||
{
|
||||
if (unlikely(head->pos + size > head->end))
|
||||
if (unlikely(static_cast<std::ptrdiff_t>(size) > head->end - head->pos))
|
||||
addMemoryChunk(size);
|
||||
|
||||
char * res = head->pos;
|
||||
|
@ -21,7 +21,12 @@ bool FieldVisitorSum::operator() (UInt64 & x) const
|
||||
|
||||
bool FieldVisitorSum::operator() (Float64 & x) const { x += rhs.get<Float64>(); return x != 0; }
|
||||
|
||||
bool FieldVisitorSum::operator() (Null &) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (Null &) const
|
||||
{
|
||||
/// Do not add anything
|
||||
return false;
|
||||
}
|
||||
|
||||
bool FieldVisitorSum::operator() (String &) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (Array &) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (Tuple &) const { throw Exception("Cannot sum Tuples", ErrorCodes::LOGICAL_ERROR); }
|
||||
|
@ -20,17 +20,19 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
|
||||
#include <DataTypes/Serializations/SerializationDecimal.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/Serializations/SerializationDecimal.h>
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
@ -720,8 +722,16 @@ public:
|
||||
return false;
|
||||
}
|
||||
|
||||
auto & col_vec = assert_cast<ColumnVector<NumberType> &>(dest);
|
||||
col_vec.insertValue(value);
|
||||
if (dest.getDataType() == TypeIndex::LowCardinality)
|
||||
{
|
||||
ColumnLowCardinality & col_low = assert_cast<ColumnLowCardinality &>(dest);
|
||||
col_low.insertData(reinterpret_cast<const char *>(&value), sizeof(value));
|
||||
}
|
||||
else
|
||||
{
|
||||
auto & col_vec = assert_cast<ColumnVector<NumberType> &>(dest);
|
||||
col_vec.insertValue(value);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
};
|
||||
@ -825,8 +835,17 @@ public:
|
||||
return JSONExtractRawImpl<JSONParser>::insertResultToColumn(dest, element, {});
|
||||
|
||||
auto str = element.getString();
|
||||
ColumnString & col_str = assert_cast<ColumnString &>(dest);
|
||||
col_str.insertData(str.data(), str.size());
|
||||
|
||||
if (dest.getDataType() == TypeIndex::LowCardinality)
|
||||
{
|
||||
ColumnLowCardinality & col_low = assert_cast<ColumnLowCardinality &>(dest);
|
||||
col_low.insertData(str.data(), str.size());
|
||||
}
|
||||
else
|
||||
{
|
||||
ColumnString & col_str = assert_cast<ColumnString &>(dest);
|
||||
col_str.insertData(str.data(), str.size());
|
||||
}
|
||||
return true;
|
||||
}
|
||||
};
|
||||
@ -855,25 +874,41 @@ struct JSONExtractTree
|
||||
}
|
||||
};
|
||||
|
||||
class LowCardinalityNode : public Node
|
||||
class LowCardinalityFixedStringNode : public Node
|
||||
{
|
||||
public:
|
||||
LowCardinalityNode(DataTypePtr dictionary_type_, std::unique_ptr<Node> impl_)
|
||||
: dictionary_type(dictionary_type_), impl(std::move(impl_)) {}
|
||||
explicit LowCardinalityFixedStringNode(const size_t fixed_length_) : fixed_length(fixed_length_) { }
|
||||
bool insertResultToColumn(IColumn & dest, const Element & element) override
|
||||
{
|
||||
auto from_col = dictionary_type->createColumn();
|
||||
if (impl->insertResultToColumn(*from_col, element))
|
||||
// If element is an object we delegate the insertion to JSONExtractRawImpl
|
||||
if (element.isObject())
|
||||
return JSONExtractRawImpl<JSONParser>::insertResultToLowCardinalityFixedStringColumn(dest, element, fixed_length);
|
||||
else if (!element.isString())
|
||||
return false;
|
||||
|
||||
auto str = element.getString();
|
||||
if (str.size() > fixed_length)
|
||||
return false;
|
||||
|
||||
// For the non low cardinality case of FixedString, the padding is done in the FixedString Column implementation.
|
||||
// In order to avoid having to pass the data to a FixedString Column and read it back (which would slow down the execution)
|
||||
// the data is padded here and written directly to the Low Cardinality Column
|
||||
if (str.size() == fixed_length)
|
||||
{
|
||||
std::string_view value = from_col->getDataAt(0).toView();
|
||||
assert_cast<ColumnLowCardinality &>(dest).insertData(value.data(), value.size());
|
||||
return true;
|
||||
assert_cast<ColumnLowCardinality &>(dest).insertData(str.data(), str.size());
|
||||
}
|
||||
return false;
|
||||
else
|
||||
{
|
||||
String padded_str(str);
|
||||
padded_str.resize(fixed_length, '\0');
|
||||
|
||||
assert_cast<ColumnLowCardinality &>(dest).insertData(padded_str.data(), padded_str.size());
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private:
|
||||
DataTypePtr dictionary_type;
|
||||
std::unique_ptr<Node> impl;
|
||||
const size_t fixed_length;
|
||||
};
|
||||
|
||||
class UUIDNode : public Node
|
||||
@ -885,7 +920,15 @@ struct JSONExtractTree
|
||||
return false;
|
||||
|
||||
auto uuid = parseFromString<UUID>(element.getString());
|
||||
assert_cast<ColumnUUID &>(dest).insert(uuid);
|
||||
if (dest.getDataType() == TypeIndex::LowCardinality)
|
||||
{
|
||||
ColumnLowCardinality & col_low = assert_cast<ColumnLowCardinality &>(dest);
|
||||
col_low.insertData(reinterpret_cast<const char *>(&uuid), sizeof(uuid));
|
||||
}
|
||||
else
|
||||
{
|
||||
assert_cast<ColumnUUID &>(dest).insert(uuid);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
};
|
||||
@ -928,6 +971,7 @@ struct JSONExtractTree
|
||||
assert_cast<ColumnDecimal<DecimalType> &>(dest).insert(value);
|
||||
return true;
|
||||
}
|
||||
|
||||
private:
|
||||
DataTypePtr data_type;
|
||||
};
|
||||
@ -946,13 +990,18 @@ struct JSONExtractTree
|
||||
public:
|
||||
bool insertResultToColumn(IColumn & dest, const Element & element) override
|
||||
{
|
||||
if (!element.isString())
|
||||
if (element.isNull())
|
||||
return false;
|
||||
auto & col_str = assert_cast<ColumnFixedString &>(dest);
|
||||
|
||||
if (!element.isString())
|
||||
return JSONExtractRawImpl<JSONParser>::insertResultToFixedStringColumn(dest, element, {});
|
||||
|
||||
auto str = element.getString();
|
||||
auto & col_str = assert_cast<ColumnFixedString &>(dest);
|
||||
if (str.size() > col_str.getN())
|
||||
return false;
|
||||
col_str.insertData(str.data(), str.size());
|
||||
|
||||
return true;
|
||||
}
|
||||
};
|
||||
@ -1178,9 +1227,18 @@ struct JSONExtractTree
|
||||
case TypeIndex::UUID: return std::make_unique<UUIDNode>();
|
||||
case TypeIndex::LowCardinality:
|
||||
{
|
||||
// The low cardinality case is treated in two different ways:
|
||||
// For FixedString type, an especial class is implemented for inserting the data in the destination column,
|
||||
// as the string length must be passed in order to check and pad the incoming data.
|
||||
// For the rest of low cardinality types, the insertion is done in their corresponding class, adapting the data
|
||||
// as needed for the insertData function of the ColumnLowCardinality.
|
||||
auto dictionary_type = typeid_cast<const DataTypeLowCardinality *>(type.get())->getDictionaryType();
|
||||
auto impl = build(function_name, dictionary_type);
|
||||
return std::make_unique<LowCardinalityNode>(dictionary_type, std::move(impl));
|
||||
if ((*dictionary_type).getTypeId() == TypeIndex::FixedString)
|
||||
{
|
||||
auto fixed_length = typeid_cast<const DataTypeFixedString *>(dictionary_type.get())->getN();
|
||||
return std::make_unique<LowCardinalityFixedStringNode>(fixed_length);
|
||||
}
|
||||
return build(function_name, dictionary_type);
|
||||
}
|
||||
case TypeIndex::Decimal256: return std::make_unique<DecimalNode<Decimal256>>(type);
|
||||
case TypeIndex::Decimal128: return std::make_unique<DecimalNode<Decimal128>>(type);
|
||||
@ -1332,13 +1390,63 @@ public:
|
||||
|
||||
static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view)
|
||||
{
|
||||
ColumnString & col_str = assert_cast<ColumnString &>(dest);
|
||||
auto & chars = col_str.getChars();
|
||||
WriteBufferFromVector<ColumnString::Chars> buf(chars, AppendModeTag());
|
||||
if (dest.getDataType() == TypeIndex::LowCardinality)
|
||||
{
|
||||
ColumnString::Chars chars;
|
||||
WriteBufferFromVector<ColumnString::Chars> buf(chars, AppendModeTag());
|
||||
traverse(element, buf);
|
||||
buf.finalize();
|
||||
assert_cast<ColumnLowCardinality &>(dest).insertData(reinterpret_cast<const char *>(chars.data()), chars.size());
|
||||
}
|
||||
else
|
||||
{
|
||||
ColumnString & col_str = assert_cast<ColumnString &>(dest);
|
||||
auto & chars = col_str.getChars();
|
||||
WriteBufferFromVector<ColumnString::Chars> buf(chars, AppendModeTag());
|
||||
traverse(element, buf);
|
||||
buf.finalize();
|
||||
chars.push_back(0);
|
||||
col_str.getOffsets().push_back(chars.size());
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
// We use insertResultToFixedStringColumn in case we are inserting raw data in a FixedString column
|
||||
static bool insertResultToFixedStringColumn(IColumn & dest, const Element & element, std::string_view)
|
||||
{
|
||||
ColumnFixedString::Chars chars;
|
||||
WriteBufferFromVector<ColumnFixedString::Chars> buf(chars, AppendModeTag());
|
||||
traverse(element, buf);
|
||||
buf.finalize();
|
||||
chars.push_back(0);
|
||||
col_str.getOffsets().push_back(chars.size());
|
||||
|
||||
auto & col_str = assert_cast<ColumnFixedString &>(dest);
|
||||
|
||||
if (chars.size() > col_str.getN())
|
||||
return false;
|
||||
|
||||
chars.resize_fill(col_str.getN());
|
||||
col_str.insertData(reinterpret_cast<const char *>(chars.data()), chars.size());
|
||||
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
// We use insertResultToLowCardinalityFixedStringColumn in case we are inserting raw data in a Low Cardinality FixedString column
|
||||
static bool insertResultToLowCardinalityFixedStringColumn(IColumn & dest, const Element & element, size_t fixed_length)
|
||||
{
|
||||
if (element.getObject().size() > fixed_length)
|
||||
return false;
|
||||
|
||||
ColumnFixedString::Chars chars;
|
||||
WriteBufferFromVector<ColumnFixedString::Chars> buf(chars, AppendModeTag());
|
||||
traverse(element, buf);
|
||||
buf.finalize();
|
||||
|
||||
if (chars.size() > fixed_length)
|
||||
return false;
|
||||
chars.resize_fill(fixed_length);
|
||||
assert_cast<ColumnLowCardinality &>(dest).insertData(reinterpret_cast<const char *>(chars.data()), chars.size());
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -34,7 +34,7 @@ private:
|
||||
|
||||
struct NameCanonicalRand
|
||||
{
|
||||
static constexpr auto name = "canonicalRand";
|
||||
static constexpr auto name = "randCanonical";
|
||||
};
|
||||
|
||||
class FunctionCanonicalRand : public FunctionRandomImpl<CanonicalRandImpl, Float64, NameCanonicalRand>
|
||||
@ -52,7 +52,7 @@ REGISTER_FUNCTION(CanonicalRand)
|
||||
The function generates pseudo random results with independent and identically distributed uniformly distributed values in [0, 1).
|
||||
Non-deterministic. Return type is Float64.
|
||||
)",
|
||||
Documentation::Examples{{"canonicalRand", "SELECT canonicalRand()"}},
|
||||
Documentation::Examples{{"randCanonical", "SELECT randCanonical()"}},
|
||||
Documentation::Categories{"Mathematical"}});
|
||||
}
|
||||
|
||||
|
@ -12,7 +12,15 @@ try
|
||||
std::string input = std::string(reinterpret_cast<const char*>(data), size);
|
||||
|
||||
DB::ParserQueryWithOutput parser(input.data() + input.size());
|
||||
DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 1000);
|
||||
|
||||
const UInt64 max_parser_depth = 1000;
|
||||
DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, max_parser_depth);
|
||||
|
||||
const UInt64 max_ast_depth = 1000;
|
||||
ast->checkDepth(max_ast_depth);
|
||||
|
||||
const UInt64 max_ast_elements = 50000;
|
||||
ast->checkSize(max_ast_elements);
|
||||
|
||||
DB::WriteBufferFromOwnString wb;
|
||||
DB::formatAST(*ast, wb);
|
||||
|
73
tests/performance/low_cardinality_from_json.xml
Normal file
73
tests/performance/low_cardinality_from_json.xml
Normal file
@ -0,0 +1,73 @@
|
||||
<test>
|
||||
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>string_json</name>
|
||||
<values>
|
||||
<value>'{"a": "hi", "b": "hello", "c": "hola", "d": "see you, bye, bye"}'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
<substitution>
|
||||
<name>int_json</name>
|
||||
<values>
|
||||
<value>'{"a": 11, "b": 2222, "c": 33333333, "d": 4444444444444444}'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
<substitution>
|
||||
<name>uuid_json</name>
|
||||
<values>
|
||||
<value>'{"a": "2d49dc6e-ddce-4cd0-afb8-790956df54c4", "b": "2d49dc6e-ddce-4cd0-afb8-790956df54c3", "c": "2d49dc6e-ddce-4cd0-afb8-790956df54c1", "d": "2d49dc6e-ddce-4cd0-afb8-790956df54c1"}'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
<substitution>
|
||||
<name>low_cardinality_tuple_string</name>
|
||||
<values>
|
||||
<value>'Tuple(a LowCardinality(String), b LowCardinality(String), c LowCardinality(String), d LowCardinality(String) )'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
<substitution>
|
||||
<name>low_cardinality_tuple_fixed_string</name>
|
||||
<values>
|
||||
<value>'Tuple(a LowCardinality(FixedString(20)), b LowCardinality(FixedString(20)), c LowCardinality(FixedString(20)), d LowCardinality(FixedString(20)) )'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
<substitution>
|
||||
<name>low_cardinality_tuple_int8</name>
|
||||
<values>
|
||||
<value>'Tuple(a LowCardinality(Int8), b LowCardinality(Int8), c LowCardinality(Int8), d LowCardinality(Int8) )'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
<substitution>
|
||||
<name>low_cardinality_tuple_int16</name>
|
||||
<values>
|
||||
<value>'Tuple(a LowCardinality(Int16), b LowCardinality(Int16), c LowCardinality(Int16), d LowCardinality(Int16) )'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
<substitution>
|
||||
<name>low_cardinality_tuple_int32</name>
|
||||
<values>
|
||||
<value>'Tuple(a LowCardinality(Int32), b LowCardinality(Int32), c LowCardinality(Int32), d LowCardinality(Int32) )'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
<substitution>
|
||||
<name>low_cardinality_tuple_int64</name>
|
||||
<values>
|
||||
<value>'Tuple(a LowCardinality(Int64), b LowCardinality(Int64), c LowCardinality(Int64), d LowCardinality(Int64) )'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
<substitution>
|
||||
<name>low_cardinality_tuple_uuid</name>
|
||||
<values>
|
||||
<value>'Tuple(a LowCardinality(UUID), b LowCardinality(UUID), c LowCardinality(UUID), d LowCardinality(UUID) )'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
</substitutions>
|
||||
|
||||
<query>SELECT 'fixed_string_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({string_json}), {low_cardinality_tuple_fixed_string})) FORMAT Null </query>
|
||||
<query>SELECT 'string_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({string_json}), {low_cardinality_tuple_string})) FORMAT Null </query>
|
||||
<query>SELECT 'int8_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({int_json}), {low_cardinality_tuple_int8})) FORMAT Null </query>
|
||||
<query>SELECT 'int16_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({int_json}), {low_cardinality_tuple_int16})) FORMAT Null </query>
|
||||
<query>SELECT 'int32_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({int_json}), {low_cardinality_tuple_int32})) FORMAT Null </query>
|
||||
<query>SELECT 'int64_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({int_json}), {low_cardinality_tuple_int64})) FORMAT Null </query>
|
||||
<query>SELECT 'uuid_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({uuid_json}), {low_cardinality_tuple_uuid})) FORMAT Null </query>
|
||||
</test>
|
@ -1,13 +1,13 @@
|
||||
select toTypeName(rand(cast(4 as Nullable(UInt8))));
|
||||
select toTypeName(canonicalRand(CAST(4 as Nullable(UInt8))));
|
||||
select toTypeName(randCanonical(CAST(4 as Nullable(UInt8))));
|
||||
select toTypeName(randConstant(CAST(4 as Nullable(UInt8))));
|
||||
select toTypeName(rand(Null));
|
||||
select toTypeName(canonicalRand(Null));
|
||||
select toTypeName(randCanonical(Null));
|
||||
select toTypeName(randConstant(Null));
|
||||
|
||||
select rand(cast(4 as Nullable(UInt8))) * 0;
|
||||
select canonicalRand(cast(4 as Nullable(UInt8))) * 0;
|
||||
select randCanonical(cast(4 as Nullable(UInt8))) * 0;
|
||||
select randConstant(CAST(4 as Nullable(UInt8))) * 0;
|
||||
select rand(Null) * 0;
|
||||
select canonicalRand(Null) * 0;
|
||||
select randCanonical(Null) * 0;
|
||||
select randConstant(Null) * 0;
|
||||
|
@ -0,0 +1,7 @@
|
||||
('hi','hello','hola','see you, bye, bye')
|
||||
('hi\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','hello\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','hola\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','see you, bye, bye\0\0\0')
|
||||
(11,0,0,0)
|
||||
(11,2222,0,0)
|
||||
(11,2222,33333333,0)
|
||||
(11,2222,33333333,4444444444444444)
|
||||
('2d49dc6e-ddce-4cd0-afb8-790956df54c4','2d49dc6e-ddce-4cd0-afb8-790956df54c3','2d49dc6e-ddce-4cd0-afb8-790956df54c1','2d49dc6e-ddce-4cd0-afb8-790956df54c1')
|
55
tests/queries/0_stateless/02452_check_low_cardinality.sql
Normal file
55
tests/queries/0_stateless/02452_check_low_cardinality.sql
Normal file
@ -0,0 +1,55 @@
|
||||
-- Tags: no-fasttest
|
||||
DROP TABLE IF EXISTS test_low_cardinality_string;
|
||||
DROP TABLE IF EXISTS test_low_cardinality_uuid;
|
||||
DROP TABLE IF EXISTS test_low_cardinality_int;
|
||||
CREATE TABLE test_low_cardinality_string (data String) ENGINE MergeTree ORDER BY data;
|
||||
CREATE TABLE test_low_cardinality_uuid (data String) ENGINE MergeTree ORDER BY data;
|
||||
CREATE TABLE test_low_cardinality_int (data String) ENGINE MergeTree ORDER BY data;
|
||||
INSERT INTO test_low_cardinality_string (data) VALUES ('{"a": "hi", "b": "hello", "c": "hola", "d": "see you, bye, bye"}');
|
||||
INSERT INTO test_low_cardinality_int (data) VALUES ('{"a": 11, "b": 2222, "c": 33333333, "d": 4444444444444444}');
|
||||
INSERT INTO test_low_cardinality_uuid (data) VALUES ('{"a": "2d49dc6e-ddce-4cd0-afb8-790956df54c4", "b": "2d49dc6e-ddce-4cd0-afb8-790956df54c3", "c": "2d49dc6e-ddce-4cd0-afb8-790956df54c1", "d": "2d49dc6e-ddce-4cd0-afb8-790956df54c1"}');
|
||||
SELECT JSONExtract(data, 'Tuple(
|
||||
a LowCardinality(String),
|
||||
b LowCardinality(String),
|
||||
c LowCardinality(String),
|
||||
d LowCardinality(String)
|
||||
)') AS json FROM test_low_cardinality_string;
|
||||
SELECT JSONExtract(data, 'Tuple(
|
||||
a LowCardinality(FixedString(20)),
|
||||
b LowCardinality(FixedString(20)),
|
||||
c LowCardinality(FixedString(20)),
|
||||
d LowCardinality(FixedString(20))
|
||||
)') AS json FROM test_low_cardinality_string;
|
||||
SELECT JSONExtract(data, 'Tuple(
|
||||
a LowCardinality(Int8),
|
||||
b LowCardinality(Int8),
|
||||
c LowCardinality(Int8),
|
||||
d LowCardinality(Int8)
|
||||
)') AS json FROM test_low_cardinality_int;
|
||||
SELECT JSONExtract(data, 'Tuple(
|
||||
a LowCardinality(Int16),
|
||||
b LowCardinality(Int16),
|
||||
c LowCardinality(Int16),
|
||||
d LowCardinality(Int16)
|
||||
)') AS json FROM test_low_cardinality_int;
|
||||
SELECT JSONExtract(data, 'Tuple(
|
||||
a LowCardinality(Int32),
|
||||
b LowCardinality(Int32),
|
||||
c LowCardinality(Int32),
|
||||
d LowCardinality(Int32)
|
||||
)') AS json FROM test_low_cardinality_int;
|
||||
SELECT JSONExtract(data, 'Tuple(
|
||||
a LowCardinality(Int64),
|
||||
b LowCardinality(Int64),
|
||||
c LowCardinality(Int64),
|
||||
d LowCardinality(Int64)
|
||||
)') AS json FROM test_low_cardinality_int;
|
||||
SELECT JSONExtract(data, 'Tuple(
|
||||
a LowCardinality(UUID),
|
||||
b LowCardinality(UUID),
|
||||
c LowCardinality(UUID),
|
||||
d LowCardinality(UUID)
|
||||
)') AS json FROM test_low_cardinality_uuid;
|
||||
DROP TABLE test_low_cardinality_string;
|
||||
DROP TABLE test_low_cardinality_uuid;
|
||||
DROP TABLE test_low_cardinality_int;
|
@ -0,0 +1 @@
|
||||
('{"b":{"c":1,"d":"str"}}\0')
|
@ -0,0 +1,6 @@
|
||||
-- Tags: no-fasttest
|
||||
DROP TABLE IF EXISTS test_fixed_string_nested_json;
|
||||
CREATE TABLE test_fixed_string_nested_json (data String) ENGINE MergeTree ORDER BY data;
|
||||
INSERT INTO test_fixed_string_nested_json (data) VALUES ('{"a" : {"b" : {"c" : 1, "d" : "str"}}}');
|
||||
SELECT JSONExtract(data, 'Tuple(a FixedString(24))') AS json FROM test_fixed_string_nested_json;
|
||||
DROP TABLE test_fixed_string_nested_json;
|
@ -0,0 +1,2 @@
|
||||
('{"b":{"c":1,"d":"str"}}','','','')
|
||||
('{"b":{"c":1,"d":"str"}}','','','')
|
@ -0,0 +1,3 @@
|
||||
-- Tags: no-fasttest
|
||||
SELECT JSONExtract('{"a" : {"b" : {"c" : 1, "d" : "str"}}}', 'Tuple( a LowCardinality(String), b LowCardinality(String), c LowCardinality(String), d LowCardinality(String))');
|
||||
SELECT JSONExtract('{"a" : {"b" : {"c" : 1, "d" : "str"}}}', 'Tuple( a String, b LowCardinality(String), c LowCardinality(String), d LowCardinality(String))');
|
@ -0,0 +1,10 @@
|
||||
\0\0\0\0\0\0\0\0\0\0\0
|
||||
{"a":123456}
|
||||
\0\0\0\0\0
|
||||
123456
|
||||
\0\0\0\0\0
|
||||
123456
|
||||
\0\0\0\0\0
|
||||
\0\0\0\0\0
|
||||
131231
|
||||
131231
|
@ -0,0 +1,9 @@
|
||||
-- Tags: no-fasttest
|
||||
SELECT JSONExtract('{"a": 123456}', 'FixedString(11)');
|
||||
SELECT JSONExtract('{"a": 123456}', 'FixedString(12)');
|
||||
SELECT JSONExtract('{"a": "123456"}', 'a', 'FixedString(5)');
|
||||
SELECT JSONExtract('{"a": "123456"}', 'a', 'FixedString(6)');
|
||||
SELECT JSONExtract('{"a": 123456}', 'a', 'FixedString(5)');
|
||||
SELECT JSONExtract('{"a": 123456}', 'a', 'FixedString(6)');
|
||||
SELECT JSONExtract(materialize('{"a": 131231}'), 'a', 'LowCardinality(FixedString(5))') FROM numbers(2);
|
||||
SELECT JSONExtract(materialize('{"a": 131231}'), 'a', 'LowCardinality(FixedString(6))') FROM numbers(2);
|
@ -0,0 +1,3 @@
|
||||
SET allow_experimental_analyzer = 1;
|
||||
|
||||
SELECT min(b), x AS b FROM (SELECT max(number) FROM numbers(1)); -- { serverError UNKNOWN_IDENTIFIER }
|
119
tests/queries/0_stateless/02480_max_map_null_totals.reference
Normal file
119
tests/queries/0_stateless/02480_max_map_null_totals.reference
Normal file
@ -0,0 +1,119 @@
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([0],[0])
|
||||
([2],[2])
|
||||
([1],[1])
|
||||
|
||||
([0,2],[0,2])
|
||||
([0],[0])
|
||||
([2],[2])
|
||||
([1],[1])
|
||||
([0,2],[0,2])
|
||||
([0],[0])
|
||||
([2],[2])
|
||||
([1],[1])
|
||||
([0,2],[0,2])
|
||||
-
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([0],[0])
|
||||
([2],[2])
|
||||
([1],[1])
|
||||
|
||||
([0,2],[0,2])
|
||||
([0],[0])
|
||||
([2],[2])
|
||||
([1],[1])
|
||||
([0,2],[0,2])
|
||||
([0],[0])
|
||||
([2],[2])
|
||||
([1],[1])
|
||||
([0,2],[0,2])
|
||||
-
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([-1,0],[0,0])
|
||||
([1,2],[0,2])
|
||||
([0,1],[0,1])
|
||||
([-1,0,1,2],[0,0,0,2])
|
||||
([0],[0])
|
||||
([2],[2])
|
||||
([1],[1])
|
||||
|
||||
([0,2],[0,2])
|
||||
([0],[0])
|
||||
([2],[2])
|
||||
([1],[1])
|
||||
([0,2],[0,2])
|
||||
([0],[0])
|
||||
([2],[2])
|
||||
([1],[1])
|
||||
([0,2],[0,2])
|
39
tests/queries/0_stateless/02480_max_map_null_totals.sql
Normal file
39
tests/queries/0_stateless/02480_max_map_null_totals.sql
Normal file
@ -0,0 +1,39 @@
|
||||
SELECT maxMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS;
|
||||
SELECT maxMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP;
|
||||
SELECT maxMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH CUBE;
|
||||
|
||||
SELECT minMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS;
|
||||
SELECT minMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP;
|
||||
SELECT minMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH CUBE;
|
||||
|
||||
SELECT sumMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS;
|
||||
SELECT sumMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP;
|
||||
SELECT sumMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH CUBE;
|
||||
|
||||
SELECT '-';
|
||||
|
||||
SELECT maxMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS;
|
||||
SELECT maxMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP;
|
||||
SELECT maxMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH CUBE;
|
||||
|
||||
SELECT minMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS;
|
||||
SELECT minMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP;
|
||||
SELECT minMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH CUBE;
|
||||
|
||||
SELECT sumMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS;
|
||||
SELECT sumMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP;
|
||||
SELECT sumMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH CUBE;
|
||||
|
||||
SELECT '-';
|
||||
|
||||
SELECT maxMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS;
|
||||
SELECT maxMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP;
|
||||
SELECT maxMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH CUBE;
|
||||
|
||||
SELECT minMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS;
|
||||
SELECT minMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP;
|
||||
SELECT minMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH CUBE;
|
||||
|
||||
SELECT sumMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS;
|
||||
SELECT sumMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP;
|
||||
SELECT sumMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH CUBE;
|
@ -0,0 +1 @@
|
||||
0123456789012345678901234567890123456789012345678901234567890123
|
@ -0,0 +1,7 @@
|
||||
-- https://github.com/ClickHouse/ClickHouse/issues/43247
|
||||
SELECT finalizeAggregation(CAST('AggregateFunction(categoricalInformationValue, Nullable(UInt8), UInt8)AggregateFunction(categoricalInformationValue, Nullable(UInt8), UInt8)',
|
||||
'AggregateFunction(min, String)')); -- { serverError 131 }
|
||||
|
||||
-- Value from hex(minState('0123456789012345678901234567890123456789012345678901234567890123')). Size 63 + 1 (64)
|
||||
SELECT finalizeAggregation(CAST(unhex('4000000030313233343536373839303132333435363738393031323334353637383930313233343536373839303132333435363738393031323334353637383930313233'),
|
||||
'AggregateFunction(min, String)'));
|
Loading…
Reference in New Issue
Block a user