tuple as literal

This commit is contained in:
Anton Popov 2020-03-24 15:55:35 +03:00
parent f08771e563
commit c1291e9348
5 changed files with 90 additions and 42 deletions

View File

@ -101,9 +101,9 @@ namespace
using Conjunction = ColumnsWithTypeAndName;
using Disjunction = std::vector<Conjunction>;
Disjunction analyzeEquals(const ASTIdentifier * identifier, const ASTLiteral * literal, const ExpressionActionsPtr & expr)
Disjunction analyzeEquals(const ASTIdentifier * identifier, const Field & value, const ExpressionActionsPtr & expr)
{
if (!identifier || !literal)
if (!identifier || value.isNull())
{
return {};
}
@ -117,7 +117,7 @@ namespace
{
ColumnWithTypeAndName column;
// FIXME: what to do if field is not convertable?
column.column = type->createColumnConst(1, convertFieldToType(literal->value, *type));
column.column = type->createColumnConst(1, convertFieldToType(value, *type));
column.name = name;
column.type = type;
return {{std::move(column)}};
@ -127,6 +127,16 @@ namespace
return {};
}
Disjunction analyzeEquals(const ASTIdentifier * identifier, const ASTLiteral * literal, const ExpressionActionsPtr & expr)
{
if (!identifier || !literal)
{
return {};
}
return analyzeEquals(identifier, literal->value, expr);
}
Disjunction andDNF(const Disjunction & left, const Disjunction & right)
{
if (left.empty())
@ -172,33 +182,44 @@ namespace
const auto * left = fn->arguments->children.front().get();
const auto * right = fn->arguments->children.back().get();
const auto * identifier = left->as<ASTIdentifier>();
const auto * inner_fn = right->as<ASTFunction>();
if (!inner_fn)
{
return {};
}
const auto * tuple = inner_fn->children.front()->as<ASTExpressionList>();
if (!tuple)
{
return {};
}
Disjunction result;
for (const auto & child : tuple->children)
if (const auto * tuple_func = right->as<ASTFunction>(); tuple_func && tuple_func->name == "tuple")
{
const auto * literal = child->as<ASTLiteral>();
const auto dnf = analyzeEquals(identifier, literal, expr);
if (dnf.empty())
const auto * tuple_elements = tuple_func->children.front()->as<ASTExpressionList>();
for (const auto & child : tuple_elements->children)
{
return {};
}
const auto * literal = child->as<ASTLiteral>();
const auto dnf = analyzeEquals(identifier, literal, expr);
result.insert(result.end(), dnf.begin(), dnf.end());
if (dnf.empty())
{
return {};
}
result.insert(result.end(), dnf.begin(), dnf.end());
}
}
else if (const auto * tuple_literal = right->as<ASTLiteral>();
tuple_literal && tuple_literal->value.getType() == Field::Types::Tuple)
{
const auto & tuple = tuple_literal->value.get<const Tuple &>();
for (const auto & child : tuple)
{
const auto dnf = analyzeEquals(identifier, child, expr);
if (dnf.empty())
{
return {};
}
result.insert(result.end(), dnf.begin(), dnf.end());
}
}
else
{
return {};
}
return result;

View File

@ -999,11 +999,9 @@ bool ParserCollectionOfLiterals<Collection>::parseImpl(Pos & pos, ASTPtr & node,
Pos literal_begin = pos;
Collection arr;
ParserLiteral literal_p;
++pos;
while (pos.isValid())
{
if (!arr.empty())
@ -1012,12 +1010,11 @@ bool ParserCollectionOfLiterals<Collection>::parseImpl(Pos & pos, ASTPtr & node,
{
std::shared_ptr<ASTLiteral> literal;
/// Parse one-element tuples (e.g. (1)) as single values for backward compatibility.
/// Parse one-element tuples (e.g. (1)) later as single values for backward compatibility.
if (std::is_same_v<Collection, Tuple> && arr.size() == 1)
literal = std::make_shared<ASTLiteral>(arr[0]);
else
literal = std::make_shared<ASTLiteral>(arr);
return false;
literal = std::make_shared<ASTLiteral>(arr);
literal->begin = literal_begin;
literal->end = ++pos;
node = literal;
@ -1029,9 +1026,8 @@ bool ParserCollectionOfLiterals<Collection>::parseImpl(Pos & pos, ASTPtr & node,
}
else
{
std::stringstream msg;
msg << "comma or " << getTokenName(closing_bracket);
expected.add(pos, msg.str().c_str());
String message = String("comma or ") + getTokenName(closing_bracket);
expected.add(pos, message.c_str());
return false;
}
}

View File

@ -239,6 +239,7 @@ private:
TokenType closing_bracket;
};
/// A tuple of literals with same type.
class ParserTupleOfLiterals : public IParserBase
{
public:

View File

@ -273,6 +273,7 @@ private:
void executeSingleThread(size_t thread_num, size_t num_threads);
void finish();
public:
String dumpPipeline() const;
};

View File

@ -1,4 +1,5 @@
#include <Columns/ColumnConst.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/FieldToDataType.h>
@ -148,11 +149,18 @@ private:
info.type = std::make_shared<DataTypeFloat64>();
else if (field_type == Field::Types::String)
info.type = std::make_shared<DataTypeString>();
else if (field_type == Field::Types::Array)
else if (field_type == Field::Types::Array || field_type == Field::Types::Tuple)
{
info.special_parser.is_array = true;
info.type = applyVisitor(FieldToDataType(), info.literal->value);
auto nested_type = assert_cast<const DataTypeArray &>(*info.type).getNestedType();
DataTypePtr nested_type;
if (auto array_type = typeid_cast<const DataTypeArray *>(info.type.get()))
nested_type = array_type->getNestedType();
else if (auto tuple_type = typeid_cast<const DataTypeTuple *>(info.type.get()))
nested_type = tuple_type->getElements()[0];
else
throw Exception("Unexpected type " + info.type->getName(), ErrorCodes::LOGICAL_ERROR);
/// It can be Array(Nullable(nested_type))
bool array_of_nullable = false;
@ -192,7 +200,18 @@ private:
info.special_parser.is_nullable = true;
}
info.type = std::make_shared<DataTypeArray>(nested_type);
if (field_type == Field::Types::Tuple)
{
const auto & tuple = info.literal->value.get<const Tuple &>();
DataTypes elements(tuple.size());
for (size_t i = 0; i < tuple.size(); ++i)
elements[i] = nested_type;
info.type = std::make_shared<DataTypeTuple>(elements);
}
else
{
info.type = std::make_shared<DataTypeArray>(nested_type);
}
}
else
throw Exception(String("Unexpected literal type ") + info.literal->value.getTypeName() + ". It's a bug",
@ -408,18 +427,28 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, co
{
/// TODO faster way to check types without using Parsers
ParserArrayOfLiterals parser_array;
ParserTupleOfLiterals parser_tuple;
Tokens tokens_number(istr.position(), istr.buffer().end());
IParser::Pos iterator(tokens_number, settings.max_parser_depth);
Expected expected;
ASTPtr ast;
if (!parser_array.parse(iterator, ast, expected))
if (!parser_array.parse(iterator, ast, expected) && !parser_tuple.parse(iterator, ast, expected))
return false;
istr.position() = const_cast<char *>(iterator->begin);
const Field & array = ast->as<ASTLiteral &>().value;
auto array_type = applyVisitor(FieldToDataType(), array);
auto nested_type = assert_cast<const DataTypeArray &>(*array_type).getNestedType();
const Field & collection = ast->as<ASTLiteral &>().value;
auto collection_type = applyVisitor(FieldToDataType(), collection);
DataTypePtr nested_type;
if (auto array_type = typeid_cast<const DataTypeArray *>(collection_type.get()))
nested_type = array_type->getNestedType();
else if (auto tuple_type = typeid_cast<const DataTypeTuple *>(collection_type.get()))
nested_type = tuple_type->getElements()[0];
if (!nested_type)
return false;
if (type_info.is_nullable)
if (auto nullable = dynamic_cast<const DataTypeNullable *>(nested_type.get()))
nested_type = nullable->getNestedType();
@ -429,7 +458,7 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, co
(nested_type_info.isNativeInt() && type_info.nested_type == Type::Int64) ||
(nested_type_info.isFloat64() && type_info.nested_type == Type::Float64))
{
Field array_same_types = convertFieldToType(array, *complex_type, nullptr);
Field array_same_types = convertFieldToType(collection, *complex_type, nullptr);
columns[column_idx]->insert(array_same_types);
return true;
}