clickhouse-client/local: make multi-query mode the default

This commit is contained in:
wxybear 2024-07-26 01:40:37 +08:00 committed by Robert Schulze
parent 37210d29ee
commit 906d709d84
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
78 changed files with 335 additions and 180 deletions

View File

@ -1135,8 +1135,6 @@ void Client::processOptions(const OptionsDescription & options_description,
if ((query_fuzzer_runs = options["query-fuzzer-runs"].as<int>()))
{
// Fuzzer implies multiquery.
config().setBool("multiquery", true);
// Ignore errors in parsing queries.
config().setBool("ignore-error", true);
ignore_error = true;
@ -1144,8 +1142,6 @@ void Client::processOptions(const OptionsDescription & options_description,
if ((create_query_fuzzer_runs = options["create-query-fuzzer-runs"].as<int>()))
{
// Fuzzer implies multiquery.
config().setBool("multiquery", true);
// Ignore errors in parsing queries.
config().setBool("ignore-error", true);
@ -1201,9 +1197,6 @@ void Client::processConfig()
}
print_stack_trace = config().getBool("stacktrace", false);
if (config().has("multiquery"))
is_multiquery = true;
pager = config().getString("pager", "");
setDefaultFormatsAndCompressionFromConfiguration();
@ -1359,13 +1352,6 @@ void Client::readArguments(
allow_repeated_settings = true;
else if (arg == "--allow_merge_tree_settings")
allow_merge_tree_settings = true;
else if (arg == "--multiquery" && (arg_num + 1) < argc && !std::string_view(argv[arg_num + 1]).starts_with('-'))
{
/// Transform the abbreviated syntax '--multiquery <SQL>' into the full syntax '--multiquery -q <SQL>'
++arg_num;
arg = argv[arg_num];
addMultiquery(arg, common_arguments);
}
else if (arg == "--password" && ((arg_num + 1) >= argc || std::string_view(argv[arg_num + 1]).starts_with('-')))
{
common_arguments.emplace_back(arg);

View File

@ -569,9 +569,6 @@ void LocalServer::processConfig()
if (!queries.empty() && getClientConfiguration().has("queries-file"))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Options '--query' and '--queries-file' cannot be specified at the same time");
if (getClientConfiguration().has("multiquery"))
is_multiquery = true;
pager = getClientConfiguration().getString("pager", "");
delayed_interactive = getClientConfiguration().has("interactive") && (!queries.empty() || getClientConfiguration().has("queries-file"));
@ -936,13 +933,6 @@ void LocalServer::readArguments(int argc, char ** argv, Arguments & common_argum
query_parameters.emplace(param_continuation.substr(0, equal_pos), param_continuation.substr(equal_pos + 1));
}
}
else if (arg == "--multiquery" && (arg_num + 1) < argc && !std::string_view(argv[arg_num + 1]).starts_with('-'))
{
/// Transform the abbreviated syntax '--multiquery <SQL>' into the full syntax '--multiquery -q <SQL>'
++arg_num;
arg = argv[arg_num];
addMultiquery(arg, common_arguments);
}
else
{
common_arguments.emplace_back(arg);

View File

@ -80,6 +80,8 @@
#include <Common/config_version.h>
#include "config.h"
#include <IO/ReadHelpers.h>
#include <Processors/Formats/Impl/ValuesBlockInputFormat.h>
#if USE_GWP_ASAN
# include <Common/GWPAsan.h>
@ -745,14 +747,6 @@ void ClientBase::adjustSettings()
/// NOTE: Do not forget to set changed=false to avoid sending it to the server (to avoid breakage read only profiles)
/// In case of multi-query we allow data after semicolon since it will be
/// parsed by the client and interpreted as new query
if (is_multiquery && !global_context->getSettingsRef().input_format_values_allow_data_after_semicolon.changed)
{
settings.input_format_values_allow_data_after_semicolon = true;
settings.input_format_values_allow_data_after_semicolon.changed = false;
}
/// Do not limit pretty format output in case of --pager specified or in case of stdout is not a tty.
if (!pager.empty() || !stdout_is_a_tty)
{
@ -1525,13 +1519,6 @@ void ClientBase::setInsertionTable(const ASTInsertQuery & insert_query)
}
void ClientBase::addMultiquery(std::string_view query, Arguments & common_arguments) const
{
common_arguments.emplace_back("--multiquery");
common_arguments.emplace_back("-q");
common_arguments.emplace_back(query);
}
namespace
{
bool isStdinNotEmptyAndValid(ReadBufferFromFileDescriptor & std_in)
@ -2186,23 +2173,48 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
return MultiQueryProcessingStage::PARSING_FAILED;
}
// INSERT queries may have the inserted data in the query text
// that follow the query itself, e.g. "insert into t format CSV 1;2".
// They need special handling. First of all, here we find where the
// inserted data ends. In multi-query mode, it is delimited by a
// newline.
// The VALUES format needs even more handling - we also allow the
// data to be delimited by semicolon. This case is handled later by
// the format parser itself.
// We can't do multiline INSERTs with inline data, because most
// row input formats (e.g. TSV) can't tell when the input stops,
// unlike VALUES.
// INSERT queries may have the inserted data in the query text that follow the query itself, e.g. "insert into t format CSV 1,2". They
// need special handling.
// - If the INSERT statement FORMAT is VALUES, we use the VALUES format parser to skip the inserted data until we reach the trailing single semicolon.
// - Other formats (e.g. FORMAT CSV) are arbitrarily more complex and tricky to parse. For example, we may be unable to distinguish if the semicolon
// is part of the data or ends the statement. In this case, we simply assume that the end of the INSERT statement is determined by \n\n (two newlines).
auto * insert_ast = parsed_query->as<ASTInsertQuery>();
const char * query_to_execute_end = this_query_end;
if (insert_ast && insert_ast->data)
{
this_query_end = find_first_symbols<'\n'>(insert_ast->data, all_queries_end);
if (insert_ast->format == "Values")
{
// Invoke the VALUES format parser to skip the inserted data
ReadBufferFromMemory data_in(insert_ast->data, all_queries_end - insert_ast->data);
skipBOMIfExists(data_in);
do
{
skipWhitespaceIfAny(data_in);
if (data_in.eof() || *data_in.position() == ';')
break;
}
while (ValuesBlockInputFormat::skipToNextRow(&data_in, 1, 0));
// Handle the case of a comment followed by a semicolon
// Example: INSERT INTO tab VALUES xx; -- {serverError xx}
// If we use this error hint, the next query should not be placed on the same line
this_query_end = insert_ast->data + data_in.count();
const auto * pos_newline = find_first_symbols<'\n'>(this_query_end, all_queries_end);
if (pos_newline != this_query_end)
{
TestHint hint(String(this_query_end, pos_newline - this_query_end));
if (hint.hasClientErrors() || hint.hasServerErrors())
this_query_end = pos_newline;
}
}
else
{
// Handling of generic formats
auto pos_newline = String(insert_ast->data, all_queries_end).find("\n\n");
if (pos_newline != std::string::npos)
this_query_end = insert_ast->data + pos_newline;
else
this_query_end = all_queries_end;
}
insert_ast->end = this_query_end;
query_to_execute_end = isSyncInsertWithData(*insert_ast, client_context) ? insert_ast->data : this_query_end;
}
@ -2237,7 +2249,10 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
size_t test_tags_length = getTestTagsLength(all_queries_text);
/// Several queries separated by ';'.
/// INSERT data is ended by the end of line, not ';'.
/// INSERT data is ended by the empty line (\n\n), not ';'.
/// Unnecessary semicolons may cause data to be parsed containing ';'
/// e.g. 'insert into xx format csv val;' will insert "val;" instead of "val"
/// 'insert into xx format csv val\n;' will insert "val" and ";"
/// An exception is VALUES format where we also support semicolon in
/// addition to end of line.
const char * this_query_begin = all_queries_text.data() + test_tags_length;
@ -2248,8 +2263,8 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
String query_to_execute;
ASTPtr parsed_query;
std::unique_ptr<Exception> current_exception;
size_t retries_count = 0;
bool is_first = true;
while (true)
{
@ -2258,16 +2273,24 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
switch (stage)
{
case MultiQueryProcessingStage::QUERIES_END:
{
/// Compatible with old version when run interactive, e.g. "", "\ld"
if (is_first && is_interactive)
processTextAsSingleQuery(all_queries_text);
return true;
}
case MultiQueryProcessingStage::PARSING_FAILED:
{
return true;
}
case MultiQueryProcessingStage::CONTINUE_PARSING:
{
is_first = false;
continue;
}
case MultiQueryProcessingStage::PARSING_EXCEPTION:
{
is_first = false;
this_query_end = find_first_symbols<'\n'>(this_query_end, all_queries_end);
// Try to find test hint for syntax error. We don't know where
@ -2297,6 +2320,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
}
case MultiQueryProcessingStage::EXECUTE_QUERY:
{
is_first = false;
full_query = all_queries_text.substr(this_query_begin - all_queries_text.data(), this_query_end - this_query_begin);
if (query_fuzzer_runs)
{
@ -2306,6 +2330,8 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
this_query_begin = this_query_end;
continue;
}
if (suggest)
updateSuggest(parsed_query);
// Now we know for sure where the query ends.
// Look for the hint in the text of query + insert data + trailing
@ -2456,14 +2482,6 @@ bool ClientBase::processQueryText(const String & text)
return processMultiQueryFromFile(file_name);
}
if (!is_multiquery)
{
assert(!query_fuzzer_runs);
processTextAsSingleQuery(text);
return true;
}
if (query_fuzzer_runs)
{
processWithFuzzing(text);
@ -2901,9 +2919,9 @@ void ClientBase::init(int argc, char ** argv)
("config-file,C", po::value<std::string>(), "config-file path")
("query,q", po::value<std::vector<std::string>>()->multitoken(), R"(query; can be specified multiple times (--query "SELECT 1" --query "SELECT 2"...))")
("query,q", po::value<std::vector<std::string>>()->multitoken(), R"(Query. Can be specified multiple times (--query "SELECT 1" --query "SELECT 2") or once with multiple comma-separated queries (--query "SELECT 1; SELECT 2;"). In the latter case, INSERT queries with non-VALUE format must be separated by empty lines.)")
("queries-file", po::value<std::vector<std::string>>()->multitoken(), "file path with queries to execute; multiple files can be specified (--queries-file file1 file2...)")
("multiquery,n", "If specified, multiple queries separated by semicolons can be listed after --query. For convenience, it is also possible to omit --query and pass the queries directly after --multiquery.")
("multiquery,n", "Obsolete, does nothing")
("multiline,m", "If specified, allow multiline queries (do not send the query on Enter)")
("database,d", po::value<std::string>(), "database")
("query_kind", po::value<std::string>()->default_value("initial_query"), "One of initial_query/secondary_query/no_query")
@ -2932,7 +2950,7 @@ void ClientBase::init(int argc, char ** argv)
("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command")
("highlight", po::value<bool>()->default_value(true), "enable or disable basic syntax highlight in interactive command line")
("ignore-error", "do not stop processing in multiquery mode")
("ignore-error", "do not stop processing when an error occurs")
("stacktrace", "print stack traces of exceptions")
("hardware-utilization", "print hardware utilization information in progress bar")
("print-profile-events", po::value(&profile_events.print)->zero_tokens(), "Printing ProfileEvents packets")
@ -3025,8 +3043,6 @@ void ClientBase::init(int argc, char ** argv)
queries_files = options["queries-file"].as<std::vector<std::string>>();
if (options.count("multiline"))
getClientConfiguration().setBool("multiline", true);
if (options.count("multiquery"))
getClientConfiguration().setBool("multiquery", true);
if (options.count("ignore-error"))
getClientConfiguration().setBool("ignore-error", true);
if (options.count("format"))

View File

@ -156,8 +156,6 @@ protected:
void setInsertionTable(const ASTInsertQuery & insert_query);
void addMultiquery(std::string_view query, Arguments & common_arguments) const;
private:
void receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, bool partial_result_on_first_cancel);
bool receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_);
@ -229,7 +227,6 @@ protected:
std::unique_ptr<Poco::Runnable> signal_listener;
bool is_interactive = false; /// Use either interactive line editing interface or batch mode.
bool is_multiquery = false;
bool delayed_interactive = false;
bool echo_queries = false; /// Print queries before execution in batch mode.

View File

@ -1156,7 +1156,6 @@ class IColumn;
M(Bool, input_format_values_interpret_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.", 0) \
M(Bool, input_format_values_deduce_templates_of_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \
M(Bool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \
M(Bool, input_format_values_allow_data_after_semicolon, false, "For Values format: allow extra data after semicolon (used by client to interpret comments).", 0) \
M(Bool, input_format_avro_allow_missing_fields, false, "For Avro/AvroConfluent format: when field is not found in schema use default value instead of error", 0) \
/** This setting is obsolete and do nothing, left for compatibility reasons. */ \
M(Bool, input_format_avro_null_as_default, false, "For Avro/AvroConfluent format: insert default in case of null and non Nullable column", 0) \

View File

@ -215,7 +215,6 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se
format_settings.tsv.allow_variable_number_of_columns = settings.input_format_tsv_allow_variable_number_of_columns;
format_settings.tsv.crlf_end_of_line_input = settings.input_format_tsv_crlf_end_of_line;
format_settings.values.accurate_types_of_literals = settings.input_format_values_accurate_types_of_literals;
format_settings.values.allow_data_after_semicolon = settings.input_format_values_allow_data_after_semicolon;
format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions;
format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions;
format_settings.values.escape_quote_with_quote = settings.output_format_values_escape_quote_with_quote;

View File

@ -625,8 +625,6 @@ void ValuesBlockInputFormat::readSuffix()
skipWhitespaceIfAny(*buf);
if (buf->hasUnreadData())
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read data after semicolon");
if (!format_settings.values.allow_data_after_semicolon && !buf->eof())
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read data after semicolon (and input_format_values_allow_data_after_semicolon=0)");
return;
}

View File

@ -3,8 +3,10 @@
1
1
Syntax error
Syntax error
Syntax error
1
2
1
2
Syntax error
Syntax error
1

View File

@ -10,8 +10,8 @@ $CLICKHOUSE_CLIENT --query="SELECT 1; "
$CLICKHOUSE_CLIENT --query="SELECT 1 ; "
$CLICKHOUSE_CLIENT --query="SELECT 1; S" 2>&1 | grep -o 'Syntax error'
$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2" 2>&1 | grep -o 'Syntax error'
$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2;" 2>&1 | grep -o 'Syntax error'
$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2"
$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2;"
$CLICKHOUSE_CLIENT --query="SELECT 1; SELECT 2; SELECT" 2>&1 | grep -o 'Syntax error'
$CLICKHOUSE_CLIENT -n --query="SELECT 1; S" 2>&1 | grep -o 'Syntax error'

View File

@ -4,12 +4,15 @@ DROP TABLE IF EXISTS json;
CREATE TABLE json (x Enum8('browser' = 1, 'mobile' = 2), y String) ENGINE = Memory;
INSERT INTO json (y) VALUES ('Hello');
SELECT * FROM json ORDER BY y;
INSERT INTO json (y) FORMAT JSONEachRow {"y": "World 1"};
SELECT * FROM json ORDER BY y;
INSERT INTO json (x, y) FORMAT JSONEachRow {"y": "World 2"};
SELECT * FROM json ORDER BY y;
DROP TABLE json;

View File

@ -7,6 +7,7 @@ INSERT INTO date VALUES ('2017-01-02'), ('2017-1-02'), ('2017-01-2'), ('2017-1-2
SELECT * FROM date;
INSERT INTO date FORMAT JSONEachRow {"d": "2017-01-02"}, {"d": "2017-1-02"}, {"d": "2017-01-2"}, {"d": "2017-1-2"}, {"d": "2017/01/02"}, {"d": "2017/1/02"}, {"d": "2017/01/2"}, {"d": "2017/1/2"}, {"d": "2017-11-12"};
SELECT * FROM date ORDER BY d;
DROP TABLE date;

View File

@ -1,3 +1,5 @@
CREATE TEMPORARY TABLE test_float (x Float64);
INSERT INTO test_float FORMAT TabSeparated 1.075e+06
SELECT * FROM test_float;

View File

@ -9,22 +9,22 @@ CREATE TABLE IF NOT EXISTS decimal
INSERT INTO decimal (a, b, c) VALUES (42.0, -42.0, 42) (0.42, -0.42, .42) (42.42, -42.42, 42.42);
INSERT INTO decimal (a, b, c) FORMAT JSONEachRow {"a":1.1, "b":-1.1, "c":1.1} {"a":1.0, "b":-1.0, "c":1} {"a":0.1, "b":-0.1, "c":.1};
INSERT INTO decimal (a, b, c) FORMAT CSV 2.0,-2.0,2
;
INSERT INTO decimal (a, b, c) FORMAT CSV 0.2 ,-0.2 ,.2
;
INSERT INTO decimal (a, b, c) FORMAT CSV 2.2 , -2.2 , 2.2
;
INSERT INTO decimal (a, b, c) FORMAT TabSeparated 3.3 -3.3 3.3
;
INSERT INTO decimal (a, b, c) FORMAT TabSeparated 3.0 -3.0 3
;
INSERT INTO decimal (a, b, c) FORMAT TabSeparated 0.3 -0.3 .3
;
INSERT INTO decimal (a, b, c) FORMAT CSV 4.4E+5,-4E+8,.4E+20
;
INSERT INTO decimal (a, b, c) FORMAT CSV 5.5e-2, -5e-9 ,.5e-17
;
SELECT * FROM decimal ORDER BY a FORMAT JSONEachRow;
SELECT * FROM decimal ORDER BY b DESC FORMAT CSV;

View File

@ -15,20 +15,29 @@ CREATE TABLE defaults
) ENGINE = MergeTree ORDER BY x;
INSERT INTO defaults FORMAT JSONEachRow {"x":1, "y":1};
INSERT INTO defaults (x, y) SELECT x, y FROM defaults LIMIT 1;
INSERT INTO defaults FORMAT JSONEachRow {"x":2, "y":2, "c":2};
INSERT INTO defaults FORMAT JSONEachRow {"x":3, "y":3, "a":3, "b":3, "c":3};
INSERT INTO defaults FORMAT JSONEachRow {"x":4} {"y":5, "c":5} {"a":6, "b":6, "c":6};
SELECT * FROM defaults ORDER BY (x, y);
ALTER TABLE defaults ADD COLUMN n Nested(a UInt64, b String);
INSERT INTO defaults FORMAT JSONEachRow {"x":7, "y":7, "n.a":[1,2], "n.b":["a","b"]};
SELECT * FROM defaults WHERE x = 7 FORMAT JSONEachRow;
ALTER TABLE defaults ADD COLUMN n.c Array(UInt8) DEFAULT arrayMap(x -> 0, n.a) AFTER n.a;
INSERT INTO defaults FORMAT JSONEachRow {"x":8, "y":8, "n.a":[3,4], "n.b":["c","d"]};
INSERT INTO defaults FORMAT JSONEachRow {"x":9, "y":9};
SELECT * FROM defaults WHERE x > 7 ORDER BY x FORMAT JSONEachRow;
DROP TABLE defaults;

View File

@ -2,15 +2,14 @@ DROP TABLE IF EXISTS test_00861;
CREATE TABLE test_00861 (key UInt64, d32 Decimal32(2), d64 Decimal64(2), d128 Decimal128(2)) ENGINE = Memory;
INSERT INTO test_00861 FORMAT CSV "1","1","1","1"
;
INSERT INTO test_00861 FORMAT CSV "2","-1","-1","-1"
;
INSERT INTO test_00861 FORMAT CSV "3","1.0","1.0","1.0"
;
INSERT INTO test_00861 FORMAT CSV "4","-0.1","-0.1","-0.1"
;
INSERT INTO test_00861 FORMAT CSV "5","0.010","0.010","0.010"
;
SELECT * FROM test_00861 ORDER BY key;

View File

@ -7,7 +7,7 @@ CREATE TABLE IF NOT EXISTS defaults
predict1 Float64,
predict2 Float64
) ENGINE = Memory;
insert into defaults values (1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2)
insert into defaults values (1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2);
DROP TABLE IF EXISTS model;
create table model engine = Memory as select stochasticLogisticRegressionState(0.1, 0.0, 1.0, 'SGD')(target, param1, param2) as state from defaults;

File diff suppressed because one or more lines are too long

View File

@ -4,7 +4,7 @@ create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDa
system stop ttl merges ttl;
insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1), (toDateTime('2000-10-10 00:00:00'), 2)
insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1), (toDateTime('2000-10-10 00:00:00'), 2);
insert into ttl values (toDateTime('2100-10-10 00:00:00'), 3), (toDateTime('2100-10-10 00:00:00'), 4);
select sleep(1) format Null; -- wait if very fast merge happen

View File

@ -22,51 +22,64 @@ SELECT 5;
/* Check JSONCompactEachRow Input */
CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2;
INSERT INTO test_table FORMAT JSONCompactEachRow ["first", 1, "2", null] ["second", 2, null, 6];
SELECT * FROM test_table FORMAT JSONCompactEachRow;
TRUNCATE TABLE test_table;
SELECT 6;
/* Check input_format_null_as_default = 1 */
SET input_format_null_as_default = 1;
INSERT INTO test_table FORMAT JSONCompactEachRow ["first", 1, "2", null] ["second", 2, null, 6];
SELECT * FROM test_table FORMAT JSONCompactEachRow;
TRUNCATE TABLE test_table;
SELECT 7;
/* Check Nested */
CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1;
INSERT INTO test_table_2 FORMAT JSONCompactEachRow [16, [15, 16, null], ["first", "second", "third"]];
SELECT * FROM test_table_2 FORMAT JSONCompactEachRow;
TRUNCATE TABLE test_table_2;
SELECT 8;
/* Check JSONCompactEachRowWithNamesAndTypes and JSONCompactEachRowWithNamesAndTypes Input */
SET input_format_null_as_default = 0;
INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", 1, "2", null]["second", 2, null, 6];
INSERT INTO test_table FORMAT JSONCompactEachRowWithNames ["v1", "v2", "v3", "v4"]["first", 1, "2", null]["second", 2, null, 6];
SELECT * FROM test_table FORMAT JSONCompactEachRow;
TRUNCATE TABLE test_table;
SELECT 9;
/* Check input_format_null_as_default = 1 */
SET input_format_null_as_default = 1;
INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", 1, "2", null] ["second", 2, null, 6];
INSERT INTO test_table FORMAT JSONCompactEachRowWithNames ["v1", "v2", "v3", "v4"]["first", 1, "2", null] ["second", 2, null, 6];
SELECT * FROM test_table FORMAT JSONCompactEachRow;
SELECT 10;
/* Check Header */
TRUNCATE TABLE test_table;
SET input_format_skip_unknown_fields = 1;
INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", 1, 32]["second", 2, "64"];
INSERT INTO test_table FORMAT JSONCompactEachRowWithNames ["v1", "v2", "invalid_column"]["first", 1, 32]["second", 2, "64"];
SELECT * FROM test_table FORMAT JSONCompactEachRow;
SELECT 11;
TRUNCATE TABLE test_table;
INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"][1, 2, 3]
INSERT INTO test_table FORMAT JSONCompactEachRowWithNames ["v4", "v2", "v3"][1, 2, 3]
SELECT * FROM test_table FORMAT JSONCompactEachRowWithNamesAndTypes;
SELECT '----------';
SELECT * FROM test_table FORMAT JSONCompactEachRowWithNames;
SELECT 12;
/* Check Nested */
INSERT INTO test_table_2 FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"][16, [15, 16, null], ["first", "second", "third"]];
INSERT INTO test_table_2 FORMAT JSONCompactEachRowWithNames ["v1", "n.id", "n.name"][16, [15, 16, null], ["first", "second", "third"]];
SELECT * FROM test_table_2 FORMAT JSONCompactEachRowWithNamesAndTypes;
SELECT '----------';
SELECT * FROM test_table_2 FORMAT JSONCompactEachRowWithNames;

View File

@ -3,6 +3,7 @@ DROP TABLE IF EXISTS defaults_all_columns;
CREATE TABLE defaults_all_columns (n UInt8 DEFAULT 42, s String DEFAULT concat('test', CAST(n, 'String'))) ENGINE = Memory;
INSERT INTO defaults_all_columns FORMAT JSONEachRow {"n": 1, "s": "hello"} {};
INSERT INTO defaults_all_columns FORMAT JSONEachRow {"n": 2}, {"s": "world"};
SELECT * FROM defaults_all_columns ORDER BY n, s;

View File

@ -2,9 +2,13 @@
DROP TABLE IF EXISTS json_square_brackets;
CREATE TABLE json_square_brackets (id UInt32, name String) ENGINE = Memory;
INSERT INTO json_square_brackets FORMAT JSONEachRow [{"id": 1, "name": "name1"}, {"id": 2, "name": "name2"}];
INSERT INTO json_square_brackets FORMAT JSONEachRow[];
INSERT INTO json_square_brackets FORMAT JSONEachRow [ ] ;
INSERT INTO json_square_brackets FORMAT JSONEachRow ;
SELECT * FROM json_square_brackets ORDER BY id;

View File

@ -37,10 +37,10 @@ send -- "INSERT INTO test_01179 values ('foo'); \r"
expect "Ok."
send -- "INSERT INTO test_01179 values ('foo'); ('bar') \r"
expect "Cannot read data after semicolon"
expect "Syntax error"
send -- "SELECT val, count() FROM test_01179 GROUP BY val FORMAT TSV\r"
expect "foo\t2"
expect "foo\t3"
send -- "DROP TABLE test_01179\r"
expect "Ok."

View File

@ -112,7 +112,7 @@ DROP TABLE IF EXISTS null_in_subquery;
DROP TABLE IF EXISTS null_in_tuple;
CREATE TABLE null_in_tuple (dt DateTime, idx int, t Tuple(Nullable(UInt64), Nullable(String))) ENGINE = MergeTree() PARTITION BY dt ORDER BY idx SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi';
INSERT INTO null_in_tuple VALUES (1, 1, (1, '1')) (2, 2, (2, NULL)) (3, 3, (NULL, '3')) (4, 4, (NULL, NULL))
INSERT INTO null_in_tuple VALUES (1, 1, (1, '1')) (2, 2, (2, NULL)) (3, 3, (NULL, '3')) (4, 4, (NULL, NULL));
SET transform_null_in = 0;

View File

@ -76,7 +76,7 @@ insert into data_01278 select
reinterpretAsString(number), // s6
reinterpretAsString(number), // s7
reinterpretAsString(number) // s8
from numbers(200000); -- { serverError 241 }" > /dev/null 2>&1
from numbers(200000);" > /dev/null 2>&1
local ret_code=$?
if [[ $ret_code -eq 0 ]];
then

View File

@ -3,11 +3,11 @@ drop table if exists merge_tree;
set allow_deprecated_syntax_for_merge_tree=1;
create table merge_tree ( CounterID UInt32, StartDate Date, Sign Int8, VisitID UInt64, UserID UInt64, StartTime DateTime, ClickLogID UInt64) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), tuple(CounterID, StartDate, intHash32(UserID), VisitID, ClickLogID), 8192, Sign);
insert into merge_tree values (1, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3)
insert into merge_tree values (1, '2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3);
alter table merge_tree add column dummy String after CounterID;
describe table merge_tree;
insert into merge_tree values (1, 'Hello, Alter Table!','2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3)
insert into merge_tree values (1, 'Hello, Alter Table!','2013-09-19', 1, 0, 2, '2013-09-19 12:43:06', 3);
select CounterID, dummy from merge_tree where dummy <> '' limit 10;

View File

@ -1,7 +1,6 @@
drop table if exists tsv_raw;
create table tsv_raw (strval String, intval Int64, b1 String, b2 String, b3 String, b4 String) engine = Memory;
insert into tsv_raw format TSVRaw "a 1 \ \\ "\"" "\\""
;
select * from tsv_raw format TSVRaw;
select * from tsv_raw format JSONCompactEachRow;

View File

@ -6,6 +6,7 @@ CREATE TABLE test
) ENGINE = MergeTree() PARTITION BY id ORDER BY id;
INSERT INTO test FORMAT CSV 1,RU
INSERT INTO test FORMAT CSV 1,
SELECT * FROM test ORDER BY code;

View File

@ -24,51 +24,64 @@ SELECT 5;
/* Check JSONCompactStringsEachRow Input */
CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2;
INSERT INTO test_table FORMAT JSONCompactStringsEachRow ["first", "1", "2", "3"] ["second", "2", "3", "6"];
SELECT * FROM test_table FORMAT JSONCompactStringsEachRow;
TRUNCATE TABLE test_table;
SELECT 6;
/* Check input_format_null_as_default = 1 */
SET input_format_null_as_default = 1;
INSERT INTO test_table FORMAT JSONCompactStringsEachRow ["first", "1", "2", "ᴺᵁᴸᴸ"] ["second", "2", "null", "6"];
SELECT * FROM test_table FORMAT JSONCompactStringsEachRow;
TRUNCATE TABLE test_table;
SELECT 7;
/* Check Nested */
CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1;
INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRow ["16", "[15, 16, 17]", "['first', 'second', 'third']"];
SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRow;
TRUNCATE TABLE test_table_2;
SELECT 8;
/* Check JSONCompactStringsEachRowWithNames and JSONCompactStringsEachRowWithNamesAndTypes Input */
SET input_format_null_as_default = 0;
INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "3"]["second", "2", "3", "6"];
INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNames ["v1", "v2", "v3", "v4"]["first", "1", "2", "3"]["second", "2", "3", "6"];
SELECT * FROM test_table FORMAT JSONCompactStringsEachRow;
TRUNCATE TABLE test_table;
SELECT 9;
/* Check input_format_null_as_default = 1 */
SET input_format_null_as_default = 1;
INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"] ["second", "2", "null", "6"];
INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNames ["v1", "v2", "v3", "v4"]["first", "1", "2", "null"] ["second", "2", "null", "6"];
SELECT * FROM test_table FORMAT JSONCompactStringsEachRow;
SELECT 10;
/* Check Header */
TRUNCATE TABLE test_table;
SET input_format_skip_unknown_fields = 1;
INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", "1", "32"]["second", "2", "64"];
INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNames ["v1", "v2", "invalid_column"]["first", "1", "32"]["second", "2", "64"];
SELECT * FROM test_table FORMAT JSONCompactStringsEachRow;
SELECT 11;
TRUNCATE TABLE test_table;
INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"]["1", "2", "3"]
INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNames ["v4", "v2", "v3"]["1", "2", "3"]
SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes;
SELECT '---------';
SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNames;
SELECT 12;
/* Check Nested */
INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"]["16", "[15, 16, 17]", "['first', 'second', 'third']"];
INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRowWithNames ["v1", "n.id", "n.name"]["16", "[15, 16, 17]", "['first', 'second', 'third']"];
SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRowWithNamesAndTypes;
SELECT '---------';
SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRowWithNames;

View File

@ -8,6 +8,7 @@ CREATE TABLE table_with_enum_column_for_csv_insert (
SET input_format_csv_enum_as_number = 1;
INSERT INTO table_with_enum_column_for_csv_insert FORMAT CSV 102,2
SELECT * FROM table_with_enum_column_for_csv_insert;
SET input_format_csv_enum_as_number = 0;

View File

@ -8,6 +8,7 @@ CREATE TABLE table_with_enum_column_for_json_insert (
) ENGINE=Memory();
INSERT INTO table_with_enum_column_for_json_insert FORMAT JSONEachRow {"Id":102,"Value":2}
SELECT * FROM table_with_enum_column_for_json_insert;
DROP TABLE IF EXISTS table_with_enum_column_for_json_insert;

View File

@ -8,7 +8,9 @@ CREATE TABLE table_with_enum_column_for_tsv_insert (
SET input_format_tsv_enum_as_number = 1;
INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 102 2
INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TabSeparatedRaw 103 1
SELECT * FROM table_with_enum_column_for_tsv_insert ORDER BY Id;
SET input_format_tsv_enum_as_number = 0;

View File

@ -6,15 +6,23 @@ CREATE TABLE enum_as_num (
) ENGINE=Memory();
INSERT INTO enum_as_num FORMAT TSV 1 1
INSERT INTO enum_as_num FORMAT TSV 2 2
INSERT INTO enum_as_num FORMAT TSV 3 3
INSERT INTO enum_as_num FORMAT TSV 4 a
INSERT INTO enum_as_num FORMAT TSV 5 b
INSERT INTO enum_as_num FORMAT CSV 6,1
INSERT INTO enum_as_num FORMAT CSV 7,2
INSERT INTO enum_as_num FORMAT CSV 8,3
INSERT INTO enum_as_num FORMAT CSV 9,a
INSERT INTO enum_as_num FORMAT CSV 10,b
SELECT * FROM enum_as_num ORDER BY Id;

View File

@ -20,7 +20,7 @@ select 1;
insert into values_01564 values (11); -- { serverError VIOLATED_CONSTRAINT }
select nonexistent column; -- { serverError UNKNOWN_IDENTIFIER }
-- query after values on the same line
insert into values_01564 values (1); select 1;
insert into values_01564 values (1);
select 1;
1
-- a failing insert and then a normal insert (#https://github.com/ClickHouse/ClickHouse/issues/19353)

View File

@ -10,6 +10,7 @@ ENGINE = MergeTree ORDER BY tuple();
SYSTEM STOP MERGES t_json;
INSERT INTO t_json FORMAT JSONEachRow {"id": 1, "data": {"k1": "aa", "k2": {"k3": "bb", "k4": "c"}}} {"id": 2, "data": {"k1": "ee", "k5": "ff"}};
INSERT INTO t_json FORMAT JSONEachRow {"id": 3, "data": {"k5":"foo"}};
SELECT id, data.k1, data.k2.k3, data.k2.k4, data.k5 FROM t_json ORDER BY id;
@ -32,6 +33,7 @@ SELECT '============';
TRUNCATE TABLE t_json;
INSERT INTO t_json FORMAT JSONEachRow {"id": 1, "data": {"k1":[{"k2":"aaa","k3":[{"k4":"bbb"},{"k4":"ccc"}]},{"k2":"ddd","k3":[{"k4":"eee"},{"k4":"fff"}]}]}};
SELECT id, data.k1.k2, data.k1.k3.k4 FROM t_json ORDER BY id;
SELECT name, column, type
@ -82,4 +84,5 @@ ORDER BY name;
DROP TABLE IF EXISTS t_json;
CREATE TABLE t_json(id UInt64, data Object('JSON')) ENGINE = Log; -- { serverError ILLEGAL_COLUMN }
-- Restore it after it is fixed.
-- CREATE TABLE t_json(id UInt64, data Object('JSON')) ENGINE = Log; -- { serverError ILLEGAL_COLUMN }

View File

@ -7,6 +7,7 @@ DROP TABLE IF EXISTS t_json_10;
CREATE TABLE t_json_10 (o JSON) ENGINE = Memory;
INSERT INTO t_json_10 FORMAT JSONAsObject {"a": {"b": 1, "c": [{"d": 10, "e": [31]}, {"d": 20, "e": [63, 127]}]}} {"a": {"b": 2, "c": []}}
INSERT INTO t_json_10 FORMAT JSONAsObject {"a": {"b": 3, "c": [{"f": 20, "e": [32]}, {"f": 30, "e": [64, 128]}]}} {"a": {"b": 4, "c": []}}
SELECT DISTINCT toTypeName(o) FROM t_json_10;

View File

@ -13,6 +13,7 @@ CREATE FUNCTION hasValidSizes17 AS (arr1, arr2) -> length(arr1) = length(arr2) A
SYSTEM STOP MERGES t_json_17;
INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 1, "arr": [{"k1": [{"k2": "aaa", "k3": "bbb"}, {"k2": "ccc"}]}]}
INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 2, "arr": [{"k1": [{"k3": "ddd", "k4": 10}, {"k4": 20}], "k5": {"k6": "foo"}}]}
SELECT toTypeName(obj) FROM t_json_17 LIMIT 1;
@ -24,6 +25,7 @@ SELECT obj.arr.k1.k4 FROM t_json_17 ORDER BY obj.id;
TRUNCATE TABLE t_json_17;
INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 1, "arr": [{"k1": [{"k2": "aaa"}]}]}
INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 2, "arr": [{"k1": [{"k2": "bbb", "k3": [{"k4": 10}]}, {"k2": "ccc", "k3": [{"k4": 20}]}]}]}
SELECT toTypeName(obj) FROM t_json_17 LIMIT 1;
@ -35,7 +37,9 @@ SELECT obj.arr.k1.k3.k4 FROM t_json_17 ORDER BY obj.id;
TRUNCATE TABLE t_json_17;
INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 1, "arr": [{"k3": "qqq"}, {"k3": "www"}]}
INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 2, "arr": [{"k1": [{"k2": "aaa"}], "k3": "eee"}]}
INSERT INTO t_json_17 FORMAT JSONAsObject {"id": 3, "arr": [{"k1": [{"k2": "bbb", "k4": [{"k5": 10}]}, {"k2": "ccc", "k4": [{"k5": 20}]}], "k3": "rrr"}]}
SELECT toTypeName(obj) FROM t_json_17 LIMIT 1;

View File

@ -8,9 +8,11 @@ CREATE TABLE t_json_2(id UInt64, data Object('JSON'))
ENGINE = MergeTree ORDER BY tuple();
INSERT INTO t_json_2 FORMAT JSONEachRow {"id": 1, "data" : {"k1": 1}};
SELECT id, data, toTypeName(data) FROM t_json_2 ORDER BY id;
TRUNCATE TABLE t_json_2;
INSERT INTO t_json_2 FORMAT JSONEachRow {"id": 1, "data" : {"k1": [1, 2]}};
SELECT id, data, toTypeName(data) FROM t_json_2 ORDER BY id;

View File

@ -7,6 +7,7 @@ SET allow_experimental_object_type = 1;
CREATE TABLE t_json(id UInt64, obj JSON) ENGINE = MergeTree ORDER BY id;
INSERT INTO t_json format JSONEachRow {"id": 1, "obj": {"foo": 1, "k1": 2}};
INSERT INTO t_json format JSONEachRow {"id": 2, "obj": {"foo": 1, "k2": 2}};
OPTIMIZE TABLE t_json FINAL;

View File

@ -6,6 +6,7 @@ DROP TABLE IF EXISTS t_json_array;
CREATE TABLE t_json_array (id UInt32, arr Array(JSON)) ENGINE = MergeTree ORDER BY id;
INSERT INTO t_json_array FORMAT JSONEachRow {"id": 1, "arr": [{"k1": 1, "k2": {"k3": 2, "k4": 3}}, {"k1": 2, "k2": {"k5": "foo"}}]}
INSERT INTO t_json_array FORMAT JSONEachRow {"id": 2, "arr": [{"k1": 3, "k2": {"k3": 4, "k4": 5}}]}
SET output_format_json_named_tuples_as_objects = 1;
@ -18,6 +19,7 @@ SELECT toTypeName(arr) FROM t_json_array LIMIT 1;
TRUNCATE TABLE t_json_array;
INSERT INTO t_json_array FORMAT JSONEachRow {"id": 1, "arr": [{"k1": [{"k2": "aaa", "k3": "bbb"}, {"k2": "ccc"}]}]}
INSERT INTO t_json_array FORMAT JSONEachRow {"id": 2, "arr": [{"k1": [{"k3": "ddd", "k4": 10}, {"k4": 20}], "k5": {"k6": "foo"}}]}
SELECT * FROM t_json_array ORDER BY id FORMAT JSONEachRow;

View File

@ -9,12 +9,14 @@ CREATE TABLE type_json_src (id UInt32, data JSON) ENGINE = MergeTree ORDER BY id
CREATE TABLE type_json_dst AS type_json_src;
INSERT INTO type_json_src VALUES (1, '{"k1": 1, "k2": "foo"}');
INSERT INTO type_json_dst SELECT * FROM type_json_src;
SELECT DISTINCT toTypeName(data) FROM type_json_dst;
SELECT id, data FROM type_json_dst ORDER BY id;
INSERT INTO type_json_src VALUES (2, '{"k1": 2, "k2": "bar"}') (3, '{"k1": 3, "k3": "aaa"}');
INSERT INTO type_json_dst SELECT * FROM type_json_src WHERE id > 1;
SELECT DISTINCT toTypeName(data) FROM type_json_dst;
@ -23,10 +25,12 @@ SELECT id, data FROM type_json_dst ORDER BY id;
INSERT INTO type_json_dst VALUES (4, '{"arr": [{"k11": 5, "k22": 6}, {"k11": 7, "k33": 8}]}');
INSERT INTO type_json_src VALUES (5, '{"arr": "not array"}');
INSERT INTO type_json_dst SELECT * FROM type_json_src WHERE id = 5; -- { serverError INCOMPATIBLE_COLUMNS }
TRUNCATE TABLE type_json_src;
INSERT INTO type_json_src VALUES (5, '{"arr": [{"k22": "str1"}]}')
INSERT INTO type_json_src VALUES (5, '{"arr": [{"k22": "str1"}]}');
INSERT INTO type_json_dst SELECT * FROM type_json_src WHERE id = 5;
SELECT DISTINCT toTypeName(data) FROM type_json_dst;
@ -45,6 +49,7 @@ SET max_insert_threads = 1;
SET output_format_json_named_tuples_as_objects = 1;
INSERT INTO type_json_src FORMAT JSONAsString {"k1": 1, "k10": [{"a": "1", "b": "2"}, {"a": "2", "b": "3"}]};
INSERT INTO type_json_src FORMAT JSONAsString {"k1": 2, "k10": [{"a": "1", "b": "2", "c": {"k11": "haha"}}]};
INSERT INTO type_json_dst SELECT data FROM type_json_src;
@ -57,6 +62,7 @@ TRUNCATE TABLE type_json_dst;
-- Insert in another order. Order is important, because a way how defaults are filled differs.
INSERT INTO type_json_src FORMAT JSONAsString {"k1": 2, "k10": [{"a": "1", "b": "2", "c": {"k11": "haha"}}]};
INSERT INTO type_json_src FORMAT JSONAsString {"k1": 1, "k10": [{"a": "1", "b": "2"}, {"a": "2", "b": "3"}]};
INSERT INTO type_json_dst SELECT data FROM type_json_src;

View File

@ -3,8 +3,11 @@
DROP TABLE IF EXISTS json_square_brackets;
CREATE TABLE json_square_brackets (field String) ENGINE = Memory;
INSERT INTO json_square_brackets FORMAT JSONAsString [{"id": 1, "name": "name1"}, {"id": 2, "name": "name2"}];
INSERT INTO json_square_brackets FORMAT JSONAsString[];
INSERT INTO json_square_brackets FORMAT JSONAsString [ ] ;
INSERT INTO json_square_brackets FORMAT JSONEachRow ;
SELECT * FROM json_square_brackets;

View File

@ -16,7 +16,7 @@ SELECT query FROM system.query_log WHERE initial_query_id = (SELECT * FROM tmp)
DROP TABLE tmp;
CREATE TABLE tmp (str String) ENGINE = Log;
INSERT INTO tmp (*) VALUES ('a')
INSERT INTO tmp (*) VALUES ('a');
SELECT count() FROM (SELECT initialQueryID() FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp') GROUP BY queryID());
SELECT count() FROM (SELECT queryID() FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp') GROUP BY queryID());
SELECT count() FROM (SELECT queryID() AS t FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp') GROUP BY queryID() HAVING t == initialQueryID());

View File

@ -2,7 +2,6 @@ DROP TABLE IF EXISTS t_map_int_key;
CREATE TABLE t_map_int_key (m1 Map(UInt32, UInt32), m2 Map(Date, UInt32)) ENGINE = Memory;
INSERT INTO t_map_int_key FORMAT CSV "{1:2, 3: 4, 5 :6, 7 : 8}","{'2021-05-20':1, '2021-05-21': 2, '2021-05-22' :3, '2021-05-23' : 4}"
;
SELECT m1, m2 FROM t_map_int_key;

View File

@ -33,7 +33,6 @@ expect "Syntax error: "
expect ":) "
send -- " \\l ; \\d; \r"
expect "Syntax error (Multi-statements are not allowed): "
expect ":) "
send -- " \\l ;\r"

View File

@ -15,22 +15,31 @@ SELECT value,f FROM bool_test where value > 0;
set bool_true_representation='True';
set bool_false_representation='False';
INSERT INTO bool_test (value,f) FORMAT CSV True,test
INSERT INTO bool_test (value,f) FORMAT TSV False test
SELECT value,f FROM bool_test order by value FORMAT CSV;
SELECT value,f FROM bool_test order by value FORMAT TSV;
set bool_true_representation='Yes';
set bool_false_representation='No';
INSERT INTO bool_test (value,f) FORMAT CSV Yes,test
INSERT INTO bool_test (value,f) FORMAT TSV No test
SELECT value,f FROM bool_test order by value FORMAT CSV;
SELECT value,f FROM bool_test order by value FORMAT TSV;
set bool_true_representation='On';
set bool_false_representation='Off';
INSERT INTO bool_test (value,f) FORMAT CSV On,test
INSERT INTO bool_test (value,f) FORMAT TSV Off test
SELECT value,f FROM bool_test order by value FORMAT CSV;
SELECT value,f FROM bool_test order by value FORMAT TSV;

View File

@ -2,8 +2,11 @@ drop table if exists ips_v6;
create table ips_v6(i IPv6) Engine=Memory;
INSERT INTO ips_v6 SELECT toIPv6('::ffff:127.0.0.1');
INSERT INTO ips_v6 values ('::ffff:127.0.0.1');
INSERT INTO ips_v6 FORMAT TSV ::ffff:127.0.0.1
INSERT INTO ips_v6 SELECT ('::ffff:127.0.0.1');
SELECT * FROM ips_v6;

View File

@ -1,5 +1,5 @@
OK
OK
FAIL
FAIL
OK
OK
OK

View File

@ -4,8 +4,10 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
# just stop, no exception
${CLICKHOUSE_CLIENT} --query="#" 2>&1 | grep -F -q 'Syntax error' && echo 'OK' || echo 'FAIL'
${CLICKHOUSE_CLIENT} --query="#not a comemnt" 2>&1 | grep -F -q 'Syntax error' && echo 'OK' || echo 'FAIL'
# syntax error
${CLICKHOUSE_CLIENT} --query="select 1 #not a comemnt" 2>&1 | grep -F -q 'Syntax error' && echo 'OK' || echo 'FAIL'
${CLICKHOUSE_CLIENT} --query="select 1 #" 2>&1 | grep -F -q 'Syntax error' && echo 'OK' || echo 'FAIL'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "select 42 #" 2>&1 | grep -F -q 'Syntax error' && echo 'OK' || echo 'FAIL'

View File

@ -6,9 +6,7 @@ DROP TABLE IF EXISTS t_async_insert_02193_1;
CREATE TABLE t_async_insert_02193_1 (id UInt32, s String) ENGINE = Memory;
INSERT INTO t_async_insert_02193_1 SETTINGS async_insert = 1 FORMAT CSV
1,aaa
;
INSERT INTO t_async_insert_02193_1 SETTINGS async_insert = 1 FORMAT CSV 1,aaa
INSERT INTO t_async_insert_02193_1 SETTINGS async_insert = 1 FORMAT Values (2, 'bbb');

View File

@ -1,2 +1,2 @@
OK
Missing columns
OK

View File

@ -5,5 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_LOCAL --query="SELECT n SETTINGS allow_experimental_analyzer = 1" 2>&1 | grep -q "Code: 47. DB::Exception:" && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_LOCAL --query="SELECT n SETTINGS allow_experimental_analyzer = 0 -- { serverError 47 }" 2>&1 | grep -o 'Missing columns'
$CLICKHOUSE_LOCAL --query="SELECT n SETTINGS allow_experimental_analyzer = 1" 2>&1 | grep -q "Code: 47. DB::Exception:" && echo 'OK' || echo 'FAIL'
$CLICKHOUSE_LOCAL --query="SELECT n SETTINGS allow_experimental_analyzer = 0" 2>&1 | grep -q "Code: 47. DB::Exception:" && echo 'OK' || echo 'FAIL'

View File

@ -9,6 +9,7 @@ ENGINE = MergeTree ORDER BY id
SETTINGS min_bytes_for_wide_part = 0;
INSERT INTO t_subcolumns_sizes FORMAT JSONEachRow {"id": 1, "arr": [1, 2, 3], "n": null, "d": {"k1": "v1", "k2": [{"k3": 1, "k4": "v2"}, {"k3": 3}]}}
INSERT INTO t_subcolumns_sizes FORMAT JSONEachRow {"id": 2, "arr": [0], "n": "foo", "d": {"k1": "v3", "k2": [{"k4": "v4"}, {"k3": "v5", "k5": 5}]}}
OPTIMIZE TABLE t_subcolumns_sizes FINAL;

View File

@ -3,5 +3,6 @@ set use_structure_from_insertion_table_in_table_functions = 1;
drop table if exists test_02249;
create table test_02249 (x UInt32, y String) engine=Memory();
insert into test_02249 select * from input() format JSONEachRow {"x" : 1, "y" : "string1"}, {"y" : "string2", "x" : 2};
select * from test_02249;
drop table test_02249;

View File

@ -6,6 +6,7 @@ DROP TABLE IF EXISTS 02267_t;
CREATE TABLE 02267_t (n1 UInt32, n2 UInt32) ENGINE = Memory;
INSERT INTO 02267_t FORMAT JSONLines {"n1": 1, "n2": 2} {"n1": 3, "n2": 4} {"n1": 5, "n2": 6};
INSERT INTO 02267_t FORMAT NDJSON {"n1": 1, "n2": 2} {"n1": 3, "n2": 4} {"n1": 5, "n2": 6};
SELECT * FROM 02267_t ORDER BY n1, n2 FORMAT JSONLines;

View File

@ -40,7 +40,7 @@ create table Dates
EventTime DateTime,
) ENGINE = Memory;
Insert into Dates VALUES ('2015-10-12') , ('2016-10-12')
Insert into Dates VALUES ('2015-10-12') , ('2016-10-12');
Select '-- test summarize --' ;
set dialect='kusto';
Customers | summarize count(), min(Age), max(Age), avg(Age), sum(Age);

View File

@ -1,13 +1,19 @@
CREATE TEMPORARY TABLE test (`i` Int64, `d` DateTime);
INSERT INTO test FORMAT JSONEachRow {"i": 123, "d": "2022-05-03"};
INSERT INTO test FORMAT JSONEachRow {"i": 456, "d": "2022-05-03 01:02:03"};
SELECT * FROM test ORDER BY i;
DROP TABLE test;
CREATE TEMPORARY TABLE test (`i` Int64, `d` DateTime64);
INSERT INTO test FORMAT JSONEachRow {"i": 123, "d": "2022-05-03"};
INSERT INTO test FORMAT JSONEachRow {"i": 456, "d": "2022-05-03 01:02:03"};
SELECT * FROM test ORDER BY i;
DROP TABLE test;

View File

@ -25,7 +25,7 @@ function insert_data
$CLICKHOUSE_CURL -sS -d 'begin transaction' "$CLICKHOUSE_URL&$TXN_SETTINGS"
SETTINGS="$SETTINGS&session_check=1"
BEGIN="begin transaction;"
COMMIT=$(echo -ne "\n\ncommit")
COMMIT=$(echo -ne "\n\n\ncommit")
fi
# max_block_size=10000, so external table will contain smaller blocks that will be squashed on insert-select (more chances to catch a bug on query cancellation)

View File

@ -7,21 +7,22 @@ create table test (x Nullable(UInt32), y UInt32) engine=Memory();
set use_structure_from_insertion_table_in_table_functions=2;
set input_format_json_infer_incomplete_types_as_strings=0;
insert into test select * from file(02458_data.jsonl);
insert into test select x, 1 from file(02458_data.jsonl);
insert into test select x, y from file(02458_data.jsonl);
insert into test select x + 1, y from file(02458_data.jsonl); -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE}
insert into test select x, z from file(02458_data.jsonl);
insert into test select * from file(02458_data.jsoncompacteachrow);
insert into test select x, 1 from file(02458_data.jsoncompacteachrow); -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE}
insert into test select x, y from file(02458_data.jsoncompacteachrow); -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE}
insert into test select x + 1, y from file(02458_data.jsoncompacteachrow); -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE}
insert into test select x, z from file(02458_data.jsoncompacteachrow); -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE}
insert into test select * from input() format CSV 1,2
insert into test select x, y from input() format CSV 1,2 -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE}
insert into test select x, y from input() format JSONEachRow {"x" : null, "y" : 42}
insert into test select x, y from input() format JSONEachRow {"x" : null, "y" : 42};
select * from test order by y;
@ -31,9 +32,9 @@ insert into test select * from file(02458_data.jsonl);
insert into test select x from file(02458_data.jsonl);
insert into test select y from file(02458_data.jsonl);
insert into test select y as x from file(02458_data.jsonl);
insert into test select c1 from input() format CSV 1,2; -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE}
insert into test select x from input() format JSONEachRow {"x" : null, "y" : 42}
insert into test select x from input() format JSONEachRow {"x" : null, "y" : 42};
select * from test order by x;

View File

@ -23,7 +23,7 @@ SELECT x FROM t1 ALL RIGHT JOIN tj ON t1.id1 == tj.key1 AND t1.id2 == tj.key2 OR
SELECT '--- name clashes ---';
CREATE TABLE t (key2 UInt64, key1 Int64, b UInt64, x UInt64, val UInt64) ENGINE = Memory;
INSERT INTO t VALUES (1, -1, 11, 111, 1111), (2, -2, 22, 222, 2222), (3, -3, 33, 333, 2222), (4, -4, 44, 444, 4444), (5, -5, 55, 555, 5555)
INSERT INTO t VALUES (1, -1, 11, 111, 1111), (2, -2, 22, 222, 2222), (3, -3, 33, 333, 2222), (4, -4, 44, 444, 4444), (5, -5, 55, 555, 5555);
SELECT '-- using --';

View File

@ -14,12 +14,12 @@ CREATE TABLE regexp_dictionary_source_table
-- test back reference.
INSERT INTO regexp_dictionary_source_table VALUES (1, 0, 'Linux/(\d+[\.\d]*).+tlinux', ['name', 'version'], ['TencentOS', '\1'])
INSERT INTO regexp_dictionary_source_table VALUES (2, 0, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2'])
INSERT INTO regexp_dictionary_source_table VALUES (3, 2, '33/tclwebkit', ['version'], ['13'])
INSERT INTO regexp_dictionary_source_table VALUES (4, 2, '3[12]/tclwebkit', ['version'], ['12'])
INSERT INTO regexp_dictionary_source_table VALUES (5, 2, '3[12]/tclwebkit', ['version'], ['11'])
INSERT INTO regexp_dictionary_source_table VALUES (6, 2, '3[12]/tclwebkit', ['version'], ['10'])
INSERT INTO regexp_dictionary_source_table VALUES (1, 0, 'Linux/(\d+[\.\d]*).+tlinux', ['name', 'version'], ['TencentOS', '\1']);
INSERT INTO regexp_dictionary_source_table VALUES (2, 0, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']);
INSERT INTO regexp_dictionary_source_table VALUES (3, 2, '33/tclwebkit', ['version'], ['13']);
INSERT INTO regexp_dictionary_source_table VALUES (4, 2, '3[12]/tclwebkit', ['version'], ['12']);
INSERT INTO regexp_dictionary_source_table VALUES (5, 2, '3[12]/tclwebkit', ['version'], ['11']);
INSERT INTO regexp_dictionary_source_table VALUES (6, 2, '3[12]/tclwebkit', ['version'], ['10']);
create dictionary regexp_dict1
(
@ -50,23 +50,24 @@ CREATE TABLE needle_table
ENGINE=TinyLog;
INSERT INTO needle_table select concat(toString(number + 30), '/tclwebkit', toString(number)) from system.numbers limit 15;
select * from needle_table;
select dictGet(regexp_dict1, ('name', 'version'), key) from needle_table;
-- test invalid
INSERT INTO regexp_dictionary_source_table VALUES (6, 2, '3[12]/tclwebkit', ['version'], ['10'])
INSERT INTO regexp_dictionary_source_table VALUES (6, 2, '3[12]/tclwebkit', ['version'], ['10']);
SYSTEM RELOAD dictionary regexp_dict1; -- { serverError INCORRECT_DICTIONARY_DEFINITION }
truncate table regexp_dictionary_source_table;
INSERT INTO regexp_dictionary_source_table VALUES (6, 2, '3[12]/tclwebkit', ['version'], ['10'])
INSERT INTO regexp_dictionary_source_table VALUES (6, 2, '3[12]/tclwebkit', ['version'], ['10']);
SYSTEM RELOAD dictionary regexp_dict1; -- { serverError INCORRECT_DICTIONARY_DEFINITION }
truncate table regexp_dictionary_source_table;
INSERT INTO regexp_dictionary_source_table VALUES (1, 2, 'Linux/(\d+[\.\d]*).+tlinux', ['name', 'version'], ['TencentOS', '\1'])
INSERT INTO regexp_dictionary_source_table VALUES (2, 3, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2'])
INSERT INTO regexp_dictionary_source_table VALUES (3, 1, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2'])
INSERT INTO regexp_dictionary_source_table VALUES (1, 2, 'Linux/(\d+[\.\d]*).+tlinux', ['name', 'version'], ['TencentOS', '\1']);
INSERT INTO regexp_dictionary_source_table VALUES (2, 3, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']);
INSERT INTO regexp_dictionary_source_table VALUES (3, 1, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']);
SYSTEM RELOAD dictionary regexp_dict1; -- { serverError INCORRECT_DICTIONARY_DEFINITION }
-- test priority

View File

@ -7,7 +7,7 @@ CREATE TABLE test_tab
engine = MergeTree()
ORDER BY id;
INSERT INTO test_tab VALUES (1, 'Hello World', 'l', 'xx') (2, 'Hello World', 'll', 'x') (3, 'Hello World', 'not_found', 'x') (4, 'Hello World', '[eo]', 'x') (5, 'Hello World', '.', 'x')
INSERT INTO test_tab VALUES (1, 'Hello World', 'l', 'xx') (2, 'Hello World', 'll', 'x') (3, 'Hello World', 'not_found', 'x') (4, 'Hello World', '[eo]', 'x') (5, 'Hello World', '.', 'x');
SELECT '** replaceAll() **';
@ -77,7 +77,7 @@ CREATE TABLE test_tab
engine = MergeTree()
ORDER BY id;
INSERT INTO test_tab VALUES (1, 'Hello World', 'l', 'x') (2, 'Hello World', '', 'y')
INSERT INTO test_tab VALUES (1, 'Hello World', 'l', 'x') (2, 'Hello World', '', 'y');
-- needle: non-const, replacement: const
SELECT replaceAll(haystack, needle, 'x') FROM test_tab; -- { serverError ARGUMENT_OUT_OF_BOUND }

View File

@ -5,6 +5,7 @@ DROP TABLE IF EXISTS t_json_attach_partition;
CREATE TABLE t_json_attach_partition(b UInt64, c JSON) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO t_json_attach_partition FORMAT JSONEachRow {"b": 1, "c" : {"k1": 1}};
ALTER TABLE t_json_attach_partition DETACH PARTITION tuple();
INSERT INTO t_json_attach_partition FORMAT JSONEachRow {"b": 1, "c" : {"k1": [1, 2]}};

View File

@ -11,9 +11,13 @@ SET async_insert_use_adaptive_busy_timeout=0;
SET async_insert_busy_timeout_max_ms = 10000000;
INSERT INTO t_async_inserts_flush VALUES (1) (2);
INSERT INTO t_async_inserts_flush FORMAT JSONEachRow {"a": 10} {"a": 20};
INSERT INTO t_async_inserts_flush FORMAT JSONEachRow {"a": "str"}
INSERT INTO t_async_inserts_flush FORMAT JSONEachRow {"a": 100} {"a": 200}
INSERT INTO t_async_inserts_flush FORMAT JSONEachRow {"a": "str"};
INSERT INTO t_async_inserts_flush FORMAT JSONEachRow {"a": 100} {"a": 200};
INSERT INTO t_async_inserts_flush VALUES (3) (4) (5);
SELECT sleep(1) FORMAT Null;

View File

@ -17,8 +17,6 @@ Bad arguments
Bad arguments
Bad arguments
Bad arguments
Bad arguments
Bad arguments
320
317
Bad arguments

View File

@ -22,14 +22,17 @@ $CLICKHOUSE_CLIENT --queries-file "queries.csv" --multiquery "SELECT 251;" 2>&1
$CLICKHOUSE_LOCAL -n "SELECT 301"
$CLICKHOUSE_LOCAL -n "SELECT 302;"
$CLICKHOUSE_LOCAL -n "SELECT 304;SELECT 305;"
# --multiquery and -n are obsolete by now and no-ops.
# The only exception is a single --multiquery "<some_query>"
$CLICKHOUSE_LOCAL --multiquery --multiquery 2>&1 | grep -o 'Bad arguments'
$CLICKHOUSE_LOCAL -n --multiquery 2>&1 | grep -o 'Bad arguments'
$CLICKHOUSE_LOCAL --multiquery -n 2>&1 | grep -o 'Bad arguments'
$CLICKHOUSE_LOCAL --multiquery --multiquery "SELECT 306; SELECT 307;" 2>&1 | grep -o 'Bad arguments'
$CLICKHOUSE_LOCAL -n --multiquery "SELECT 307; SELECT 308;" 2>&1 | grep -o 'Bad arguments'
$CLICKHOUSE_LOCAL --multiquery "SELECT 309; SELECT 310;" --multiquery 2>&1 | grep -o 'Bad arguments'
$CLICKHOUSE_LOCAL --multiquery "SELECT 309; SELECT 310;" --multiquery 2>&1 | grep -o 'Bad arguments'
$CLICKHOUSE_LOCAL --multiquery "SELECT 311;" --multiquery "SELECT 312;" 2>&1 | grep -o 'Bad arguments'
$CLICKHOUSE_LOCAL --multiquery "SELECT 313;" -n "SELECT 314;" 2>&1 | grep -o 'Bad arguments'
$CLICKHOUSE_LOCAL -n "SELECT 320" --query "SELECT 317;"
# --query should be followed by SQL
$CLICKHOUSE_LOCAL --query -n "SELECT 400;" 2>&1 | grep -o 'Bad arguments'
$CLICKHOUSE_LOCAL --query -n --multiquery "SELECT 401;" 2>&1 | grep -o 'Bad arguments'

View File

@ -2,15 +2,17 @@
101
202
202
Multi-statements are not allowed
Empty query
303
303
303
Bad arguments
Syntax error
101
101
202
202
Multi-statements are not allowed
Empty query
303
303
303
Bad arguments
Syntax error

View File

@ -7,15 +7,15 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# clickhouse-client
$CLICKHOUSE_CLIENT --query "SELECT 101" --query "SELECT 101"
$CLICKHOUSE_CLIENT --query "SELECT 202;" --query "SELECT 202;"
$CLICKHOUSE_CLIENT --query "SELECT 303" --query "SELECT 303; SELECT 303" 2>&1 | grep -o 'Multi-statements are not allowed'
$CLICKHOUSE_CLIENT --query "" --query "" 2>&1 | grep -o 'Empty query'
$CLICKHOUSE_CLIENT --query "SELECT 303" --query "SELECT 303; SELECT 303"
$CLICKHOUSE_CLIENT --query "" --query "" 2>&1
$CLICKHOUSE_CLIENT --query "SELECT 303" --query 2>&1 | grep -o 'Bad arguments'
$CLICKHOUSE_CLIENT --query "SELECT 303" --query "SELE" 2>&1 | grep -o 'Syntax error'
# clickhouse-local
$CLICKHOUSE_LOCAL --query "SELECT 101" --query "SELECT 101"
$CLICKHOUSE_LOCAL --query "SELECT 202;" --query "SELECT 202;"
$CLICKHOUSE_LOCAL --query "SELECT 303" --query "SELECT 303; SELECT 303" 2>&1 | grep -o 'Multi-statements are not allowed' # behaves differently than clickhouse-client, TODO make it consistent
$CLICKHOUSE_LOCAL --query "" --query "" 2>&1 | grep -o 'Empty query' # behaves equally different than clickhouse-client TODO
$CLICKHOUSE_LOCAL --query "SELECT 303" --query "SELECT 303; SELECT 303"
$CLICKHOUSE_LOCAL --query "" --query ""
$CLICKHOUSE_LOCAL --query "SELECT 303" --query 2>&1 | grep -o 'Bad arguments'
$CLICKHOUSE_LOCAL --query "SELECT 303" --query "SELE" 2>&1 | grep -o 'Syntax error'

View File

@ -1,5 +1,7 @@
Multi-statements are not allowed
Multi-statements are not allowed
1
2
1
2
1
2
1

View File

@ -4,10 +4,11 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
# throw exception
$CLICKHOUSE_CLIENT -q "select 1; select 2;" 2>&1 | grep -o 'Multi-statements are not allowed'
$CLICKHOUSE_LOCAL -q "select 1; select 2;" 2>&1 | grep -o 'Multi-statements are not allowed'
# execute correctly
# clickhouse-local and clickhouse-client behave the same
$CLICKHOUSE_CLIENT -q "select 1; select 2;"
$CLICKHOUSE_LOCAL -q "select 1; select 2;"
# -n is a no-op
$CLICKHOUSE_CLIENT -n -q "select 1; select 2;"
$CLICKHOUSE_LOCAL -n -q "select 1; select 2;"

View File

@ -1,6 +0,0 @@
client no multiquery
Cannot read data after semicolon (and input_format_values_allow_data_after_semicolon=0)
client multiquery
local no multiquery
Cannot read data after semicolon (and input_format_values_allow_data_after_semicolon=0)
local multiquery

View File

@ -4,12 +4,5 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
echo "client no multiquery"
$CLICKHOUSE_CLIENT -q "insert into function null() values (1); -- { foo }" |& grep -F -o "Cannot read data after semicolon (and input_format_values_allow_data_after_semicolon=0)"
echo "client multiquery"
$CLICKHOUSE_CLIENT -n -q "insert into function null() values (1); -- { foo }"
echo "local no multiquery"
$CLICKHOUSE_LOCAL -q "insert into function null() values (1); -- { foo }" |& grep -F -o "Cannot read data after semicolon (and input_format_values_allow_data_after_semicolon=0)"
echo "local multiquery"
$CLICKHOUSE_LOCAL -n -q "insert into function null() values (1); -- { foo }"
$CLICKHOUSE_CLIENT -q "insert into function null() values (1); -- { foo }"
$CLICKHOUSE_LOCAL -q "insert into function null() values (1); -- { foo }"

View File

@ -18,7 +18,7 @@ ENGINE = MergeTree
PARTITION BY toYYYYMM(time)
ORDER BY (key, id, time);
INSERT INTO test1 VALUES ('id0', now(), 3, false)
INSERT INTO test1 VALUES ('id0', now(), 3, false);
SELECT last_value(value) OVER (PARTITION BY id ORDER BY time ASC) as last_value
FROM test1

View File

@ -195,7 +195,7 @@ $CLICKHOUSE_CLIENT -nq "
$CLICKHOUSE_CLIENT -nq "
drop table d;
truncate src;
insert into src values (1)
insert into src values (1);
create materialized view e refresh every 1 second (x Int64) engine MergeTree order by x empty as select x + sleepEachRow(1) as x from src settings max_block_size = 1;"
while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ]
do
@ -213,6 +213,7 @@ done
# we wait for a slow refresh, not a previous fast one.)
$CLICKHOUSE_CLIENT -nq "
insert into src select * from numbers(1000) settings max_block_size=1;
system start view e;"
while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes -- $LINENO" | xargs`" != 'Running' ]
do

View File

@ -241,12 +241,12 @@ CREATE TABLE regexp_dictionary_source_table
values Array(String),
) ENGINE=TinyLog;
INSERT INTO regexp_dictionary_source_table VALUES (1, 0, 'Linux/(\d+[\.\d]*).+tlinux', ['name', 'version'], ['TencentOS', '\1'])
INSERT INTO regexp_dictionary_source_table VALUES (2, 0, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2'])
INSERT INTO regexp_dictionary_source_table VALUES (3, 2, '33/tclwebkit', ['version'], ['13'])
INSERT INTO regexp_dictionary_source_table VALUES (4, 2, '3[12]/tclwebkit', ['version'], ['12'])
INSERT INTO regexp_dictionary_source_table VALUES (5, 2, '3[12]/tclwebkit', ['version'], ['11'])
INSERT INTO regexp_dictionary_source_table VALUES (6, 2, '3[12]/tclwebkit', ['version'], ['10'])
INSERT INTO regexp_dictionary_source_table VALUES (1, 0, 'Linux/(\d+[\.\d]*).+tlinux', ['name', 'version'], ['TencentOS', '\1']);
INSERT INTO regexp_dictionary_source_table VALUES (2, 0, '(\d+)/tclwebkit(\d+[\.\d]*)', ['name', 'version', 'comment'], ['Android', '$1', 'test $1 and $2']);
INSERT INTO regexp_dictionary_source_table VALUES (3, 2, '33/tclwebkit', ['version'], ['13']);
INSERT INTO regexp_dictionary_source_table VALUES (4, 2, '3[12]/tclwebkit', ['version'], ['12']);
INSERT INTO regexp_dictionary_source_table VALUES (5, 2, '3[12]/tclwebkit', ['version'], ['11']);
INSERT INTO regexp_dictionary_source_table VALUES (6, 2, '3[12]/tclwebkit', ['version'], ['10']);
DROP DICTIONARY IF EXISTS regexp_dict;
create dictionary regexp_dict

View File

@ -2,7 +2,7 @@ DROP TABLE IF EXISTS t2;
DROP TABLE IF EXISTS t3;
CREATE TABLE t2 (id Int32, pe Map(String, Tuple(a UInt64, b UInt64))) ENGINE = MergeTree ORDER BY id;
INSERT INTO t2 VALUES (1, {'a': (1, 2), 'b': (2, 3)}),
INSERT INTO t2 VALUES (1, {'a': (1, 2), 'b': (2, 3)}),;
CREATE TABLE t3 (id Int32, c Tuple(v String, pe Map(String, Tuple(a UInt64, b UInt64)))) ENGINE = MergeTree ORDER BY id;
INSERT INTO t3 VALUES (1, ('A', {'a':(1, 2),'b':(2, 3)}));

View File

@ -8,7 +8,7 @@ SELECT x.number FROM (
) AS x
INNER JOIN input('a UInt64') AS y ON x.number = y.a
Format CSV 2
;
select * from test;
drop table test;

View File

@ -0,0 +1,10 @@
Syntax error
101
102
1
2;
3
4
7
8
9

View File

@ -0,0 +1,59 @@
#!/usr/bin/env bash
# Tags: no-parallel, no-ordinary-database
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
SQL_FILE_NAME=$"03156_default_multiquery_split_$$.sql"
# The old multiquery implementation uses '\n' to split INSERT query segmentation
# this case is mainly to test the following situations
# 1. INSERT(format=Values) query: split by ';'
# 2. INSERT(format is not Values) query: split by '\n\n' instead of ';', then discard the ramaining data
# 3. INSERT(format is not Values) query: split by '\n\n', causing the trailing ';'' to be treated as the part of last value
# 4. The client uses multiquery by default, regardless of whether multiquery option is used.
# create table test1, test2, then
# 1. insert 101, 102 into test1
# 2. insert 1, 2; into test2, ';' will be treated as a part of a value
# 3. insert 3, 4; '6' will be treated as the next query because of the empty line, we use empty line to determine the end of insert query(format IS NOT VALUES)
# '6' will cause Syntax error
cat << EOF > "$SQL_FILE_NAME"
DROP TABLE IF EXISTS TEST1;
DROP TABLE IF EXISTS TEST2;
CREATE TABLE TEST1 (value Float64) ENGINE=MergeTree ORDER BY tuple();
CREATE TABLE TEST2 (value String) ENGINE=MergeTree ORDER BY tuple();
INSERT INTO TEST1 VALUES
(101),
(102);
INSERT INTO TEST2 FORMAT CSV
1
2;
INSERT INTO TEST2 FORMAT CSV
3
4
6
EOF
$CLICKHOUSE_CLIENT -m < "$SQL_FILE_NAME" 2>&1 | grep -o 'Syntax error'
# insert 7, 8, 9 into test2, because we use semicolon to determine the end of insert query(format is VALUES)
# then select all data from test1 and test2
cat << EOF > "$SQL_FILE_NAME"
INSERT INTO TEST2 VALUES
('7'),
('8'),
('9');
SELECT * FROM TEST1 ORDER BY value;
SELECT * FROM TEST2 ORDER BY value;
DROP TABLE TEST1; DROP TABLE TEST2;
EOF
$CLICKHOUSE_CLIENT -m -n < "$SQL_FILE_NAME"
rm "$SQL_FILE_NAME"