support complex datatype

This commit is contained in:
Alexander Tretiakov 2019-06-04 21:15:32 +03:00
parent 2f1a03dac0
commit d97c2ccdc8
12 changed files with 101 additions and 37 deletions

View File

@ -1,4 +1,6 @@
#include <boost/algorithm/string/replace.hpp>
#include <Common/typeid_cast.h>
#include <Common/StringUtils/StringUtils.h>
#include <Columns/IColumn.h>
#include <Core/Field.h>
#include <DataTypes/IDataType.h>
@ -35,9 +37,15 @@ String ReplaceQueryParameterVisitor::getParamValue(const String & name)
void ReplaceQueryParameterVisitor::visitQP(ASTPtr & ast)
{
auto ast_param = ast->as<ASTQueryParameter>();
String value = getParamValue(ast_param->name);
const auto data_type = DataTypeFactory::instance().get(ast_param->type);
const String value = getParamValue(ast_param->name);
String type = ast_param->type;
/// Replacing all occurrences of types Date and DateTime with String.
/// String comparison is used in "WHERE" conditions with this types.
boost::replace_all(type, "DateTime", "String");
boost::replace_all(type, "Date", "String");
const auto data_type = DataTypeFactory::instance().get(type);
auto temp_column_ptr = data_type->createColumn();
IColumn & temp_column = *temp_column_ptr;
ReadBufferFromString read_buffer{value};

View File

@ -208,6 +208,10 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
try
{
if (context.hasQueryParameters()) /// Avoid change from TCPHandler.
/// Get new query after substitutions.
query = serializeAST(*ast);
logQuery(query.substr(0, settings.log_queries_cut_to_length), context, internal);
/// Check the limits.

View File

@ -1211,7 +1211,7 @@ bool ParserSubstitution::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
if (pos->type != TokenType::BareWord)
{
expected.add(pos, "string literal");
expected.add(pos, "substitution name (identifier)");
return false;
}
@ -1228,12 +1228,20 @@ bool ParserSubstitution::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
if (pos->type != TokenType::BareWord)
{
expected.add(pos, "string literal");
expected.add(pos, "substitution type");
return false;
}
type = String(pos->begin, pos->end);
++pos;
auto old_pos = pos;
while ((pos->type == TokenType::OpeningRoundBracket || pos->type == TokenType::ClosingRoundBracket
|| pos->type == TokenType::Comma || pos->type == TokenType::BareWord)
&& pos->type != TokenType::ClosingCurlyBrace)
{
++pos;
}
type = String(old_pos->begin, pos->begin);
if (pos->type != TokenType::ClosingCurlyBrace)
{

View File

@ -1,3 +0,0 @@
1 Hello, world
1 Hello, world
2 test

View File

@ -1,3 +0,0 @@
1 Hello, world
1 Hello, world
2 test

View File

@ -1,19 +0,0 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS ps";
$CLICKHOUSE_CLIENT -q "CREATE TABLE ps (i UInt8, s String) ENGINE = Memory";
$CLICKHOUSE_CLIENT -q "INSERT INTO ps VALUES (1, 'Hello, world')";
$CLICKHOUSE_CLIENT -q "INSERT INTO ps VALUES (2, 'test')";
$CLICKHOUSE_CLIENT --max_threads=1 --param_id=1\
-q "SELECT * FROM ps WHERE i = {id:UInt8}";
$CLICKHOUSE_CLIENT --max_threads=1 --param_phrase='Hello, world'\
-q "SELECT * FROM ps WHERE s = {phrase:String}";
$CLICKHOUSE_CLIENT --max_threads=1 --param_id=2 --param_phrase='test'\
-q "SELECT * FROM ps WHERE i = {id:UInt8} and s = {phrase:String}";
$CLICKHOUSE_CLIENT -q "DROP TABLE ps";

View File

@ -0,0 +1,4 @@
1 Hello, world 2005-05-05
1 Hello, world 2005-05-05
2 test 2019-05-25
2 test 2019-05-25

View File

@ -4,16 +4,18 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
${CLICKHOUSE_CURL} -sS $CLICKHOUSE_URL -d "DROP TABLE IF EXISTS ps";
${CLICKHOUSE_CURL} -sS $CLICKHOUSE_URL -d "CREATE TABLE ps (i UInt8, s String) ENGINE = Memory";
${CLICKHOUSE_CURL} -sS $CLICKHOUSE_URL -d "CREATE TABLE ps (i UInt8, s String, d Date) ENGINE = Memory";
${CLICKHOUSE_CURL} -sS $CLICKHOUSE_URL -d "INSERT INTO ps VALUES (1, 'Hello, world')";
${CLICKHOUSE_CURL} -sS $CLICKHOUSE_URL -d "INSERT INTO ps VALUES (2, 'test')";
${CLICKHOUSE_CURL} -sS $CLICKHOUSE_URL -d "INSERT INTO ps VALUES (1, 'Hello, world', '2005-05-05')";
${CLICKHOUSE_CURL} -sS $CLICKHOUSE_URL -d "INSERT INTO ps VALUES (2, 'test', '2019-05-25')";
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?param_id=1"\
-d "SELECT * FROM ps WHERE i = {id:UInt8} ORDER BY i, s";
-d "SELECT * FROM ps WHERE i = {id:UInt8} ORDER BY i, s, d";
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?param_phrase=Hello,+world"\
-d "SELECT * FROM ps WHERE s = {phrase:String} ORDER BY i, s";
-d "SELECT * FROM ps WHERE s = {phrase:String} ORDER BY i, s, d";
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?param_date=2019-05-25"\
-d "SELECT * FROM ps WHERE d = {date:Date} ORDER BY i, s, d";
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?param_id=2&param_phrase=test"\
-d "SELECT * FROM ps WHERE i = {id:UInt8} and s = {phrase:String} ORDER BY i, s";
-d "SELECT * FROM ps WHERE i = {id:UInt8} and s = {phrase:String} ORDER BY i, s, d";
${CLICKHOUSE_CURL} -sS $CLICKHOUSE_URL -d "DROP TABLE ps";

View File

@ -0,0 +1,4 @@
1 Hello, world 2005-05-05 05:05:05
1 Hello, world 2005-05-05 05:05:05
2 test 2005-05-25 15:00:00
2 test 2005-05-25 15:00:00

View File

@ -0,0 +1,21 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS ps";
$CLICKHOUSE_CLIENT -q "CREATE TABLE ps (i UInt8, s String, d DateTime) ENGINE = Memory";
$CLICKHOUSE_CLIENT -q "INSERT INTO ps VALUES (1, 'Hello, world', '2005-05-05 05:05:05')";
$CLICKHOUSE_CLIENT -q "INSERT INTO ps VALUES (2, 'test', '2005-05-25 15:00:00')";
$CLICKHOUSE_CLIENT --max_threads=1 --param_id=1\
-q "SELECT * FROM ps WHERE i = {id:UInt8}";
$CLICKHOUSE_CLIENT --max_threads=1 --param_phrase='Hello, world'\
-q "SELECT * FROM ps WHERE s = {phrase:String}";
$CLICKHOUSE_CLIENT --max_threads=1 --param_date='2005-05-25 15:00:00'\
-q "SELECT * FROM ps WHERE d = {date:DateTime}";
$CLICKHOUSE_CLIENT --max_threads=1 --param_id=2 --param_phrase='test'\
-q "SELECT * FROM ps WHERE i = {id:UInt8} and s = {phrase:String}";
$CLICKHOUSE_CLIENT -q "DROP TABLE ps";

View File

@ -0,0 +1,5 @@
(1,'Hello')
(1,('dt',2))
[10,10,10]
[[10],[10],[10]]
[10,10,10] [[10],[10],[10]] (10,'Test') (10,('dt',10)) 2015-02-15

View File

@ -0,0 +1,33 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS ps";
$CLICKHOUSE_CLIENT -q "CREATE TABLE ps (
a Array(UInt32), da Array(Array(UInt8)),
t Tuple(Int16, String), dt Tuple(UInt8, Tuple(String, UInt8)),
n Nullable(Date)
) ENGINE = Memory";
$CLICKHOUSE_CLIENT -q "INSERT INTO ps VALUES (
[1, 2], [[1, 1], [2, 2]],
(1, 'Hello'), (1, ('dt', 2)),
NULL)";
$CLICKHOUSE_CLIENT -q "INSERT INTO ps VALUES (
[10, 10, 10], [[10], [10], [10]],
(10, 'Test'), (10, ('dt', 10)),
'2015-02-15')";
$CLICKHOUSE_CLIENT --max_threads=1 --param_aui="[1, 2]"\
-q "SELECT t FROM ps WHERE a = {aui:Array(UInt16)}";
$CLICKHOUSE_CLIENT --max_threads=1 --param_d_a="[[1, 1], [2, 2]]"\
-q "SELECT dt FROM ps WHERE da = {d_a:Array(Array(UInt8))}";
$CLICKHOUSE_CLIENT --max_threads=1 --param_tisd="(10, 'Test')"\
-q "SELECT a FROM ps WHERE t = {tisd:Tuple(Int16, String)}";
$CLICKHOUSE_CLIENT --max_threads=1 --param_d_t="(10, ('dt', 10)))"\
-q "SELECT da FROM ps WHERE dt = {d_t:Tuple(UInt8, Tuple(String, UInt8))}";
$CLICKHOUSE_CLIENT --max_threads=1 --param_nd="2015-02-15"\
-q "SELECT * FROM ps WHERE n = {nd:Nullable(Date)}";
$CLICKHOUSE_CLIENT -q "DROP TABLE ps";