From 7dc31a3ba339e7773262ad55f359a7bf3e9333ad Mon Sep 17 00:00:00 2001 From: Pavel Kartavyy Date: Wed, 19 Mar 2014 20:00:56 +0400 Subject: [PATCH 01/56] olap compability: removed ClickPhraseID, ClickTargetPhraseID [#METR-10322] --- dbms/src/Server/OLAPAttributesMetadata.h | 4 ---- dbms/src/Server/OLAPQueryConverter.cpp | 2 -- 2 files changed, 6 deletions(-) diff --git a/dbms/src/Server/OLAPAttributesMetadata.h b/dbms/src/Server/OLAPAttributesMetadata.h index 586ff7e7aca..fa5d6827812 100644 --- a/dbms/src/Server/OLAPAttributesMetadata.h +++ b/dbms/src/Server/OLAPAttributesMetadata.h @@ -523,7 +523,6 @@ struct UserAgentID : public IAttributeMetadata typedef AttributeIntBase ClickGoodEvent; typedef AttributeIntBase ClickPriorityID; typedef AttributeIntBase ClickBannerID; -typedef AttributeIntBase ClickPhraseID; typedef AttributeIntBase ClickPageID; typedef AttributeIntBase ClickPlaceID; typedef AttributeIntBase ClickTypeID; @@ -532,7 +531,6 @@ typedef AttributeUIntBase ClickDomainID; typedef AttributeUIntBase ClickCost; typedef AttributeHashBase ClickURLHash; typedef AttributeUIntBase ClickOrderID; -typedef AttributeUIntBase ClickTargetPhraseID; typedef AttributeUIntBase GoalReachesAny; typedef AttributeUIntBase GoalReachesDepth; typedef AttributeUIntBase GoalReachesURL; @@ -728,7 +726,6 @@ inline AttributeMetadatas GetOLAPAttributeMetadata() ("ClickGoodEvent", new ClickGoodEvent) ("ClickPriorityID", new ClickPriorityID) ("ClickBannerID", new ClickBannerID) - ("ClickPhraseID", new ClickPhraseID) ("ClickPageID", new ClickPageID) ("ClickPlaceID", new ClickPlaceID) ("ClickTypeID", new ClickTypeID) @@ -737,7 +734,6 @@ inline AttributeMetadatas GetOLAPAttributeMetadata() ("ClickCost", new ClickCost) ("ClickURLHash", new ClickURLHash) ("ClickOrderID", new ClickOrderID) - ("ClickTargetPhraseID", new ClickTargetPhraseID) ("GoalReaches", new GoalReaches) ("GoalReachesAny", new GoalReachesAny) ("GoalReachesDepth", new GoalReachesDepth) diff --git a/dbms/src/Server/OLAPQueryConverter.cpp b/dbms/src/Server/OLAPQueryConverter.cpp index 0309ac1e1d0..9fb4b3b3d4b 100644 --- a/dbms/src/Server/OLAPQueryConverter.cpp +++ b/dbms/src/Server/OLAPQueryConverter.cpp @@ -550,7 +550,6 @@ void QueryConverter::fillNumericAttributeMap() M("ClickGoodEvent", "Clicks.GoodEvent[1]") M("ClickPriorityID", "Clicks.PriorityID[1]") M("ClickBannerID", "Clicks.BannerID[1]") - M("ClickPhraseID", "Clicks.PhraseID[1]") M("ClickPageID", "Clicks.PageID[1]") M("ClickPlaceID", "Clicks.PlaceID[1]") M("ClickTypeID", "Clicks.TypeID[1]") @@ -559,7 +558,6 @@ void QueryConverter::fillNumericAttributeMap() M("ClickCost", "Clicks.Cost[1]") M("ClickURLHash", "Clicks.URLHash[1]") M("ClickOrderID", "Clicks.OrderID[1]") - M("ClickTargetPhraseID", "Clicks.TargetPhraseID[1]") M("GoalReachesAny", "GoalReachesAny") M("GoalReachesDepth", "GoalReachesDepth") M("GoalReachesURL", "GoalReachesURL") From 3a6d19d6391ee174690cc889700378e3ef8a19a5 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 21 Mar 2014 13:26:01 +0300 Subject: [PATCH 02/56] Removed unnecessary context lock during ALTER. [METR-10202] --- dbms/src/Interpreters/InterpreterAlterQuery.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index 5de38fc697c..ae4808d338b 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -133,8 +133,6 @@ void InterpreterAlterQuery::execute() /// Это позволит сделать большую часть первого MODIFY, не останавливая чтение из таблицы. IStorage::TableStructureWriteLockPtr table_hard_lock; - Poco::ScopedLock lock(context.getMutex()); - /// todo cycle over sub tables and tables /// Применяем изменения for (ASTAlterQuery::ParameterContainer::const_iterator alter_it = alter.parameters.begin(); From 2ff699ce6f81dceb579e416cbdee1970b97bff14 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Mar 2014 22:27:19 +0400 Subject: [PATCH 03/56] Merge From 40b02cae63cfca6fcab8d61329b8c92e9e0e9f9e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Mar 2014 07:26:07 +0400 Subject: [PATCH 04/56] dbms: added test script [#METR-2944]. --- dbms/tests/clickhouse-test | 66 +++++++++++++++++++ .../queries/format_ast_and_remote_table.sql | 1 + dbms/tests/queries/select_1.reference | 1 + dbms/tests/queries/select_1.sql | 1 + dbms/tests/queries/system_numbers.reference | 10 +++ dbms/tests/queries/system_numbers.sql | 1 + 6 files changed, 80 insertions(+) create mode 100755 dbms/tests/clickhouse-test create mode 100644 dbms/tests/queries/format_ast_and_remote_table.sql create mode 100644 dbms/tests/queries/select_1.reference create mode 100644 dbms/tests/queries/select_1.sql create mode 100644 dbms/tests/queries/system_numbers.reference create mode 100644 dbms/tests/queries/system_numbers.sql diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test new file mode 100755 index 00000000000..40986eb6e52 --- /dev/null +++ b/dbms/tests/clickhouse-test @@ -0,0 +1,66 @@ +#!/bin/bash + +# Скрипт для тестирования запросов к ClickHouse. +# Из файлов *.sql в заданной директории, в алфавитном порядке, отправляются все запросы. +# Результат сравнивается с эталоном. + +QUERIES_DIR="./queries" +CLIENT_PROGRAM="clickhouse-client" + + +COLOR_RESET="\033[0m" +COLOR_WHITE="\033[1;37m" +COLOR_FAIL="\033[1;31m" +COLOR_UNKNOWN="\033[1;30m" +COLOR_OK="\033[1;32m" + +MSG_FAIL="${COLOR_WHITE}[ ${COLOR_FAIL}FAIL${COLOR_WHITE} ]${COLOR_RESET}" +MSG_UNKNOWN="${COLOR_WHITE}[ ${COLOR_UNKNOWN}UNKNOWN${COLOR_WHITE} ]${COLOR_RESET}" +MSG_OK="${COLOR_WHITE}[ ${COLOR_OK}OK${COLOR_WHITE} ]${COLOR_RESET}" +MSG_GENERATED="${COLOR_WHITE}[ ${COLOR_UNKNOWN}GENERATED${COLOR_WHITE} ]${COLOR_RESET}" + + +for query_file in $(ls $QUERIES_DIR/*.sql) +do + test_name=$(basename -s .sql $query_file) + + result_file=$QUERIES_DIR/$test_name.result + error_file=$QUERIES_DIR/$test_name.error + reference_file=$QUERIES_DIR/$test_name.reference + diff_file=$QUERIES_DIR/$test_name.diff + + printf "%-30s" "$test_name: " + + $CLIENT_PROGRAM < $query_file > $result_file 2> $error_file + ret_code=$? + + if [ $ret_code -ne 0 ]; then + echo -e "$MSG_FAIL - return code $ret_code" + if [ -s "$error_file" ]; then + cat $error_file + fi + # разорвано соединение с сервером + if grep -q -E "Connection refused|Attempt to read after eof" $error_file; then + exit 1; + fi + elif [ -s "$error_file" ]; then + echo -e "$MSG_FAIL - having stderror:" + cat $error_file + elif [ ! -e "$reference_file" ]; then + # надо сгенерировать эталонный результат + if [ "$1" == "--generate" ]; then + cp $result_file $reference_file + echo -e "$MSG_GENERATED - no reference file" + else + echo -e "$MSG_UNKNOWN - no reference file (use --generate to create)" + fi + else + diff $reference_file $result_file > $diff_file + if [ -s "$diff_file" ]; then + echo -e "$MSG_FAIL - result differs with reference:" + cat $diff_file + else + echo -e "$MSG_OK" + fi + fi +done diff --git a/dbms/tests/queries/format_ast_and_remote_table.sql b/dbms/tests/queries/format_ast_and_remote_table.sql new file mode 100644 index 00000000000..dd20fd1d8e4 --- /dev/null +++ b/dbms/tests/queries/format_ast_and_remote_table.sql @@ -0,0 +1 @@ +SELECT (dummy AS x) - 1 FROM remote('127.0.0.{1,2}', system, one) diff --git a/dbms/tests/queries/select_1.reference b/dbms/tests/queries/select_1.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/select_1.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/select_1.sql b/dbms/tests/queries/select_1.sql new file mode 100644 index 00000000000..2e3761f7a2c --- /dev/null +++ b/dbms/tests/queries/select_1.sql @@ -0,0 +1 @@ +SELECT 1 diff --git a/dbms/tests/queries/system_numbers.reference b/dbms/tests/queries/system_numbers.reference new file mode 100644 index 00000000000..8b1acc12b63 --- /dev/null +++ b/dbms/tests/queries/system_numbers.reference @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/dbms/tests/queries/system_numbers.sql b/dbms/tests/queries/system_numbers.sql new file mode 100644 index 00000000000..bc9269495bc --- /dev/null +++ b/dbms/tests/queries/system_numbers.sql @@ -0,0 +1 @@ +SELECT * FROM system.numbers LIMIT 10 From 18a7749a0491cc84a7185812fb14293f443a6ae5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Mar 2014 07:37:51 +0400 Subject: [PATCH 05/56] dbms: modified test [#METR-2944]. --- dbms/tests/clickhouse-test | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 40986eb6e52..cfd6c85f82b 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -61,6 +61,7 @@ do cat $diff_file else echo -e "$MSG_OK" + rm $error_file $result_file $diff_file fi fi done From 84ce7d62258a077b0b738cb0498632236520d89b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Mar 2014 07:40:09 +0400 Subject: [PATCH 06/56] dbms: modified test [#METR-2944]. --- dbms/tests/clickhouse-test | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index cfd6c85f82b..2b7bec67bdc 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -19,6 +19,8 @@ MSG_UNKNOWN="${COLOR_WHITE}[ ${COLOR_UNKNOWN}UNKNOWN${COLOR_WHITE} ]${COLOR_RESE MSG_OK="${COLOR_WHITE}[ ${COLOR_OK}OK${COLOR_WHITE} ]${COLOR_RESET}" MSG_GENERATED="${COLOR_WHITE}[ ${COLOR_UNKNOWN}GENERATED${COLOR_WHITE} ]${COLOR_RESET}" +HAS_ERROR=0 + for query_file in $(ls $QUERIES_DIR/*.sql) do @@ -35,6 +37,7 @@ do ret_code=$? if [ $ret_code -ne 0 ]; then + HAS_ERROR=1 echo -e "$MSG_FAIL - return code $ret_code" if [ -s "$error_file" ]; then cat $error_file @@ -44,6 +47,7 @@ do exit 1; fi elif [ -s "$error_file" ]; then + HAS_ERROR=1 echo -e "$MSG_FAIL - having stderror:" cat $error_file elif [ ! -e "$reference_file" ]; then @@ -57,6 +61,7 @@ do else diff $reference_file $result_file > $diff_file if [ -s "$diff_file" ]; then + HAS_ERROR=1 echo -e "$MSG_FAIL - result differs with reference:" cat $diff_file else @@ -65,3 +70,5 @@ do fi fi done + +exit $HAS_ERROR From a646848ded048582bd83c019af0471f50d2aba9d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Mar 2014 07:46:25 +0400 Subject: [PATCH 07/56] dbms: modified test [#METR-2944]. --- dbms/tests/clickhouse-test | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 2b7bec67bdc..5b853748301 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -19,7 +19,7 @@ MSG_UNKNOWN="${COLOR_WHITE}[ ${COLOR_UNKNOWN}UNKNOWN${COLOR_WHITE} ]${COLOR_RESE MSG_OK="${COLOR_WHITE}[ ${COLOR_OK}OK${COLOR_WHITE} ]${COLOR_RESET}" MSG_GENERATED="${COLOR_WHITE}[ ${COLOR_UNKNOWN}GENERATED${COLOR_WHITE} ]${COLOR_RESET}" -HAS_ERROR=0 +ERRORS=0 for query_file in $(ls $QUERIES_DIR/*.sql) @@ -37,7 +37,7 @@ do ret_code=$? if [ $ret_code -ne 0 ]; then - HAS_ERROR=1 + ERRORS=$(($ERRORS + 1)) echo -e "$MSG_FAIL - return code $ret_code" if [ -s "$error_file" ]; then cat $error_file @@ -47,7 +47,7 @@ do exit 1; fi elif [ -s "$error_file" ]; then - HAS_ERROR=1 + ERRORS=$(($ERRORS + 1)) echo -e "$MSG_FAIL - having stderror:" cat $error_file elif [ ! -e "$reference_file" ]; then @@ -61,7 +61,7 @@ do else diff $reference_file $result_file > $diff_file if [ -s "$diff_file" ]; then - HAS_ERROR=1 + ERRORS=$(($ERRORS + 1)) echo -e "$MSG_FAIL - result differs with reference:" cat $diff_file else @@ -71,4 +71,13 @@ do fi done -exit $HAS_ERROR + +echo + +if [ $ERRORS -gt 0 ]; then + echo -e "${COLOR_FAIL}Having $ERRORS errors!${COLOR_RESET}" + exit 1 +else + echo -e "${COLOR_OK}All tests succeeded.${COLOR_RESET}" + exit 0 +fi From 93472c9103e736dc583401e1204fd50aa6e9e529 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Mar 2014 19:08:31 +0400 Subject: [PATCH 08/56] dbms: added test [#METR-2944]. --- dbms/tests/queries/format_ast_and_remote_table.reference | 2 ++ 1 file changed, 2 insertions(+) create mode 100644 dbms/tests/queries/format_ast_and_remote_table.reference diff --git a/dbms/tests/queries/format_ast_and_remote_table.reference b/dbms/tests/queries/format_ast_and_remote_table.reference new file mode 100644 index 00000000000..343ee5c2f6c --- /dev/null +++ b/dbms/tests/queries/format_ast_and_remote_table.reference @@ -0,0 +1,2 @@ +-1 +-1 From fc45d3cc5b6b322f8683055adc2c98ac7fe91d6e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Mar 2014 19:16:58 +0400 Subject: [PATCH 09/56] dbms: mofified test [#METR-2944]. --- dbms/tests/clickhouse-test | 4 ++-- .../queries/{select_1.reference => 00001_select_1.reference} | 0 dbms/tests/queries/{select_1.sql => 00001_select_1.sql} | 0 ...ystem_numbers.reference => 00002_system_numbers.reference} | 0 .../queries/{system_numbers.sql => 00002_system_numbers.sql} | 0 dbms/tests/queries/00003_reinterpret_as_string.reference | 1 + dbms/tests/queries/00003_reinterpret_as_string.sql | 1 + ....reference => 00004_format_ast_and_remote_table.reference} | 0 ...remote_table.sql => 00004_format_ast_and_remote_table.sql} | 0 9 files changed, 4 insertions(+), 2 deletions(-) rename dbms/tests/queries/{select_1.reference => 00001_select_1.reference} (100%) rename dbms/tests/queries/{select_1.sql => 00001_select_1.sql} (100%) rename dbms/tests/queries/{system_numbers.reference => 00002_system_numbers.reference} (100%) rename dbms/tests/queries/{system_numbers.sql => 00002_system_numbers.sql} (100%) create mode 100644 dbms/tests/queries/00003_reinterpret_as_string.reference create mode 100644 dbms/tests/queries/00003_reinterpret_as_string.sql rename dbms/tests/queries/{format_ast_and_remote_table.reference => 00004_format_ast_and_remote_table.reference} (100%) rename dbms/tests/queries/{format_ast_and_remote_table.sql => 00004_format_ast_and_remote_table.sql} (100%) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 5b853748301..60d66269645 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -2,7 +2,7 @@ # Скрипт для тестирования запросов к ClickHouse. # Из файлов *.sql в заданной директории, в алфавитном порядке, отправляются все запросы. -# Результат сравнивается с эталоном. +# Результаты сравниваются с эталонами. QUERIES_DIR="./queries" CLIENT_PROGRAM="clickhouse-client" @@ -31,7 +31,7 @@ do reference_file=$QUERIES_DIR/$test_name.reference diff_file=$QUERIES_DIR/$test_name.diff - printf "%-30s" "$test_name: " + printf "%-60s" "$test_name: " $CLIENT_PROGRAM < $query_file > $result_file 2> $error_file ret_code=$? diff --git a/dbms/tests/queries/select_1.reference b/dbms/tests/queries/00001_select_1.reference similarity index 100% rename from dbms/tests/queries/select_1.reference rename to dbms/tests/queries/00001_select_1.reference diff --git a/dbms/tests/queries/select_1.sql b/dbms/tests/queries/00001_select_1.sql similarity index 100% rename from dbms/tests/queries/select_1.sql rename to dbms/tests/queries/00001_select_1.sql diff --git a/dbms/tests/queries/system_numbers.reference b/dbms/tests/queries/00002_system_numbers.reference similarity index 100% rename from dbms/tests/queries/system_numbers.reference rename to dbms/tests/queries/00002_system_numbers.reference diff --git a/dbms/tests/queries/system_numbers.sql b/dbms/tests/queries/00002_system_numbers.sql similarity index 100% rename from dbms/tests/queries/system_numbers.sql rename to dbms/tests/queries/00002_system_numbers.sql diff --git a/dbms/tests/queries/00003_reinterpret_as_string.reference b/dbms/tests/queries/00003_reinterpret_as_string.reference new file mode 100644 index 00000000000..fc75583cf02 --- /dev/null +++ b/dbms/tests/queries/00003_reinterpret_as_string.reference @@ -0,0 +1 @@ +33232 diff --git a/dbms/tests/queries/00003_reinterpret_as_string.sql b/dbms/tests/queries/00003_reinterpret_as_string.sql new file mode 100644 index 00000000000..1204f6280f2 --- /dev/null +++ b/dbms/tests/queries/00003_reinterpret_as_string.sql @@ -0,0 +1 @@ +SELECT number FROM system.numbers WHERE reinterpretAsString(number) = 'Ё' LIMIT 1 diff --git a/dbms/tests/queries/format_ast_and_remote_table.reference b/dbms/tests/queries/00004_format_ast_and_remote_table.reference similarity index 100% rename from dbms/tests/queries/format_ast_and_remote_table.reference rename to dbms/tests/queries/00004_format_ast_and_remote_table.reference diff --git a/dbms/tests/queries/format_ast_and_remote_table.sql b/dbms/tests/queries/00004_format_ast_and_remote_table.sql similarity index 100% rename from dbms/tests/queries/format_ast_and_remote_table.sql rename to dbms/tests/queries/00004_format_ast_and_remote_table.sql From dae349ba00069620d212efb44160869eeff75947 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Mar 2014 00:40:55 +0400 Subject: [PATCH 10/56] dbms: added tests [#METR-2944]. --- dbms/tests/clickhouse-test | 6 +++++- ...ormat_ast_and_remote_table_lambda.reference | 1 + ...0005_format_ast_and_remote_table_lambda.sql | 1 + .../queries/00006_1_set_extremes.reference | 0 dbms/tests/queries/00006_1_set_extremes.sql | 1 + ...0006_2_extremes_and_subquery_from.reference | 18 ++++++++++++++++++ .../00006_2_extremes_and_subquery_from.sql | 2 ++ .../queries/00006_3_unset_extremes.reference | 0 dbms/tests/queries/00006_3_unset_extremes.sql | 1 + dbms/tests/queries/00007_array.reference | 1 + dbms/tests/queries/00007_array.sql | 1 + dbms/tests/queries/00008_array_join.reference | 2 ++ dbms/tests/queries/00008_array_join.sql | 1 + .../00009_array_join_subquery.reference | 2 ++ .../queries/00009_array_join_subquery.sql | 1 + .../queries/00010_big_array_join.reference | 6 ++++++ dbms/tests/queries/00010_big_array_join.sql | 1 + .../queries/00011_array_join_alias.reference | 6 ++++++ dbms/tests/queries/00011_array_join_alias.sql | 1 + .../queries/00012_array_join_alias_2.reference | 6 ++++++ .../tests/queries/00012_array_join_alias_2.sql | 1 + ..._drop_if_exists_table_with_arrays.reference | 0 ...0013_1_drop_if_exists_table_with_arrays.sql | 1 + .../00013_2_create_table_with_arrays.reference | 0 .../00013_2_create_table_with_arrays.sql | 1 + ...3_3_insert_into_table_with_arrays.reference | 0 .../00013_3_insert_into_table_with_arrays.sql | 1 + ...3_4_select_from_table_with_arrays.reference | 3 +++ .../00013_4_select_from_table_with_arrays.sql | 1 + ...3_5_select_from_table_with_arrays.reference | 5 +++++ .../00013_5_select_from_table_with_arrays.sql | 1 + ...3_6_select_from_table_with_arrays.reference | 5 +++++ .../00013_6_select_from_table_with_arrays.sql | 1 + ...3_7_select_from_table_with_arrays.reference | 5 +++++ .../00013_7_select_from_table_with_arrays.sql | 1 + ...3_8_select_from_table_with_arrays.reference | 5 +++++ .../00013_8_select_from_table_with_arrays.sql | 1 + ...3_9_select_from_table_with_arrays.reference | 5 +++++ .../00013_9_select_from_table_with_arrays.sql | 1 + ...3_a_select_from_table_with_arrays.reference | 5 +++++ .../00013_a_select_from_table_with_arrays.sql | 1 + ..._drop_if_exists_table_with_nested.reference | 0 ...0014_1_drop_if_exists_table_with_nested.sql | 1 + .../00014_2_create_table_with_nested.reference | 0 .../00014_2_create_table_with_nested.sql | 1 + ...4_3_insert_into_table_with_nested.reference | 0 .../00014_3_insert_into_table_with_nested.sql | 1 + ...4_4_select_from_table_with_nested.reference | 3 +++ .../00014_4_select_from_table_with_nested.sql | 1 + ...4_5_select_from_table_with_nested.reference | 5 +++++ .../00014_5_select_from_table_with_nested.sql | 1 + ...4_6_select_from_table_with_nested.reference | 5 +++++ .../00014_6_select_from_table_with_nested.sql | 1 + ...4_7_select_from_table_with_nested.reference | 5 +++++ .../00014_7_select_from_table_with_nested.sql | 1 + ...4_9_select_from_table_with_nested.reference | 5 +++++ .../00014_9_select_from_table_with_nested.sql | 1 + ...4_a_select_from_table_with_nested.reference | 5 +++++ .../00014_a_select_from_table_with_nested.sql | 1 + ...4_b_select_from_table_with_nested.reference | 5 +++++ .../00014_b_select_from_table_with_nested.sql | 1 + ...4_c_select_from_table_with_nested.reference | 5 +++++ .../00014_c_select_from_table_with_nested.sql | 1 + .../00015_totals_having_constants.reference | 12 ++++++++++++ .../queries/00015_totals_having_constants.sql | 1 + .../00016_totals_having_constants.reference | 3 +++ .../queries/00016_totals_having_constants.sql | 1 + ...017_in_subquery_with_empty_result.reference | 18 ++++++++++++++++++ .../00017_in_subquery_with_empty_result.sql | 2 ++ .../00018_distinct_in_subquery.reference | 2 ++ .../queries/00018_distinct_in_subquery.sql | 1 + 71 files changed, 190 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/00005_format_ast_and_remote_table_lambda.reference create mode 100644 dbms/tests/queries/00005_format_ast_and_remote_table_lambda.sql create mode 100644 dbms/tests/queries/00006_1_set_extremes.reference create mode 100644 dbms/tests/queries/00006_1_set_extremes.sql create mode 100644 dbms/tests/queries/00006_2_extremes_and_subquery_from.reference create mode 100644 dbms/tests/queries/00006_2_extremes_and_subquery_from.sql create mode 100644 dbms/tests/queries/00006_3_unset_extremes.reference create mode 100644 dbms/tests/queries/00006_3_unset_extremes.sql create mode 100644 dbms/tests/queries/00007_array.reference create mode 100644 dbms/tests/queries/00007_array.sql create mode 100644 dbms/tests/queries/00008_array_join.reference create mode 100644 dbms/tests/queries/00008_array_join.sql create mode 100644 dbms/tests/queries/00009_array_join_subquery.reference create mode 100644 dbms/tests/queries/00009_array_join_subquery.sql create mode 100644 dbms/tests/queries/00010_big_array_join.reference create mode 100644 dbms/tests/queries/00010_big_array_join.sql create mode 100644 dbms/tests/queries/00011_array_join_alias.reference create mode 100644 dbms/tests/queries/00011_array_join_alias.sql create mode 100644 dbms/tests/queries/00012_array_join_alias_2.reference create mode 100644 dbms/tests/queries/00012_array_join_alias_2.sql create mode 100644 dbms/tests/queries/00013_1_drop_if_exists_table_with_arrays.reference create mode 100644 dbms/tests/queries/00013_1_drop_if_exists_table_with_arrays.sql create mode 100644 dbms/tests/queries/00013_2_create_table_with_arrays.reference create mode 100644 dbms/tests/queries/00013_2_create_table_with_arrays.sql create mode 100644 dbms/tests/queries/00013_3_insert_into_table_with_arrays.reference create mode 100644 dbms/tests/queries/00013_3_insert_into_table_with_arrays.sql create mode 100644 dbms/tests/queries/00013_4_select_from_table_with_arrays.reference create mode 100644 dbms/tests/queries/00013_4_select_from_table_with_arrays.sql create mode 100644 dbms/tests/queries/00013_5_select_from_table_with_arrays.reference create mode 100644 dbms/tests/queries/00013_5_select_from_table_with_arrays.sql create mode 100644 dbms/tests/queries/00013_6_select_from_table_with_arrays.reference create mode 100644 dbms/tests/queries/00013_6_select_from_table_with_arrays.sql create mode 100644 dbms/tests/queries/00013_7_select_from_table_with_arrays.reference create mode 100644 dbms/tests/queries/00013_7_select_from_table_with_arrays.sql create mode 100644 dbms/tests/queries/00013_8_select_from_table_with_arrays.reference create mode 100644 dbms/tests/queries/00013_8_select_from_table_with_arrays.sql create mode 100644 dbms/tests/queries/00013_9_select_from_table_with_arrays.reference create mode 100644 dbms/tests/queries/00013_9_select_from_table_with_arrays.sql create mode 100644 dbms/tests/queries/00013_a_select_from_table_with_arrays.reference create mode 100644 dbms/tests/queries/00013_a_select_from_table_with_arrays.sql create mode 100644 dbms/tests/queries/00014_1_drop_if_exists_table_with_nested.reference create mode 100644 dbms/tests/queries/00014_1_drop_if_exists_table_with_nested.sql create mode 100644 dbms/tests/queries/00014_2_create_table_with_nested.reference create mode 100644 dbms/tests/queries/00014_2_create_table_with_nested.sql create mode 100644 dbms/tests/queries/00014_3_insert_into_table_with_nested.reference create mode 100644 dbms/tests/queries/00014_3_insert_into_table_with_nested.sql create mode 100644 dbms/tests/queries/00014_4_select_from_table_with_nested.reference create mode 100644 dbms/tests/queries/00014_4_select_from_table_with_nested.sql create mode 100644 dbms/tests/queries/00014_5_select_from_table_with_nested.reference create mode 100644 dbms/tests/queries/00014_5_select_from_table_with_nested.sql create mode 100644 dbms/tests/queries/00014_6_select_from_table_with_nested.reference create mode 100644 dbms/tests/queries/00014_6_select_from_table_with_nested.sql create mode 100644 dbms/tests/queries/00014_7_select_from_table_with_nested.reference create mode 100644 dbms/tests/queries/00014_7_select_from_table_with_nested.sql create mode 100644 dbms/tests/queries/00014_9_select_from_table_with_nested.reference create mode 100644 dbms/tests/queries/00014_9_select_from_table_with_nested.sql create mode 100644 dbms/tests/queries/00014_a_select_from_table_with_nested.reference create mode 100644 dbms/tests/queries/00014_a_select_from_table_with_nested.sql create mode 100644 dbms/tests/queries/00014_b_select_from_table_with_nested.reference create mode 100644 dbms/tests/queries/00014_b_select_from_table_with_nested.sql create mode 100644 dbms/tests/queries/00014_c_select_from_table_with_nested.reference create mode 100644 dbms/tests/queries/00014_c_select_from_table_with_nested.sql create mode 100644 dbms/tests/queries/00015_totals_having_constants.reference create mode 100644 dbms/tests/queries/00015_totals_having_constants.sql create mode 100644 dbms/tests/queries/00016_totals_having_constants.reference create mode 100644 dbms/tests/queries/00016_totals_having_constants.sql create mode 100644 dbms/tests/queries/00017_in_subquery_with_empty_result.reference create mode 100644 dbms/tests/queries/00017_in_subquery_with_empty_result.sql create mode 100644 dbms/tests/queries/00018_distinct_in_subquery.reference create mode 100644 dbms/tests/queries/00018_distinct_in_subquery.sql diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 60d66269645..1bfd91ab468 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -5,7 +5,7 @@ # Результаты сравниваются с эталонами. QUERIES_DIR="./queries" -CLIENT_PROGRAM="clickhouse-client" +CLIENT_PROGRAM="curl -sS http://localhost:8123/ --data-binary @-" COLOR_RESET="\033[0m" @@ -50,6 +50,10 @@ do ERRORS=$(($ERRORS + 1)) echo -e "$MSG_FAIL - having stderror:" cat $error_file + elif grep -q "Exception" $result_file; then + ERRORS=$(($ERRORS + 1)) + echo -e "$MSG_FAIL - having exception:" + cat $result_file elif [ ! -e "$reference_file" ]; then # надо сгенерировать эталонный результат if [ "$1" == "--generate" ]; then diff --git a/dbms/tests/queries/00005_format_ast_and_remote_table_lambda.reference b/dbms/tests/queries/00005_format_ast_and_remote_table_lambda.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/dbms/tests/queries/00005_format_ast_and_remote_table_lambda.reference @@ -0,0 +1 @@ +2 diff --git a/dbms/tests/queries/00005_format_ast_and_remote_table_lambda.sql b/dbms/tests/queries/00005_format_ast_and_remote_table_lambda.sql new file mode 100644 index 00000000000..fb401df5184 --- /dev/null +++ b/dbms/tests/queries/00005_format_ast_and_remote_table_lambda.sql @@ -0,0 +1 @@ +SELECT count() FROM remote('127.0.0.{1,2}', system, one) WHERE arrayExists((x) -> x = 1, [1, 2, 3]) diff --git a/dbms/tests/queries/00006_1_set_extremes.reference b/dbms/tests/queries/00006_1_set_extremes.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/00006_1_set_extremes.sql b/dbms/tests/queries/00006_1_set_extremes.sql new file mode 100644 index 00000000000..53bd0c96ce2 --- /dev/null +++ b/dbms/tests/queries/00006_1_set_extremes.sql @@ -0,0 +1 @@ +SET GLOBAL extremes = 1 diff --git a/dbms/tests/queries/00006_2_extremes_and_subquery_from.reference b/dbms/tests/queries/00006_2_extremes_and_subquery_from.reference new file mode 100644 index 00000000000..841ae17708f --- /dev/null +++ b/dbms/tests/queries/00006_2_extremes_and_subquery_from.reference @@ -0,0 +1,18 @@ +{ + "meta": + [ + { + "name": "'Hello, world'", + "type": "String" + } + ], + + "data": + [ + + ], + + "rows": 0, + + "rows_before_limit_at_least": 10 +} diff --git a/dbms/tests/queries/00006_2_extremes_and_subquery_from.sql b/dbms/tests/queries/00006_2_extremes_and_subquery_from.sql new file mode 100644 index 00000000000..c8e1e1bf66f --- /dev/null +++ b/dbms/tests/queries/00006_2_extremes_and_subquery_from.sql @@ -0,0 +1,2 @@ +SELECT 'Hello, world' FROM (SELECT number FROM system.numbers LIMIT 10) WHERE number < 0 +FORMAT JSONCompact \ No newline at end of file diff --git a/dbms/tests/queries/00006_3_unset_extremes.reference b/dbms/tests/queries/00006_3_unset_extremes.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/00006_3_unset_extremes.sql b/dbms/tests/queries/00006_3_unset_extremes.sql new file mode 100644 index 00000000000..31e8a97da82 --- /dev/null +++ b/dbms/tests/queries/00006_3_unset_extremes.sql @@ -0,0 +1 @@ +SET GLOBAL extremes = 0 diff --git a/dbms/tests/queries/00007_array.reference b/dbms/tests/queries/00007_array.reference new file mode 100644 index 00000000000..2a64c8ea7b2 --- /dev/null +++ b/dbms/tests/queries/00007_array.reference @@ -0,0 +1 @@ +['Hello','Goodbye'] diff --git a/dbms/tests/queries/00007_array.sql b/dbms/tests/queries/00007_array.sql new file mode 100644 index 00000000000..7c1f27f1978 --- /dev/null +++ b/dbms/tests/queries/00007_array.sql @@ -0,0 +1 @@ +SELECT ['Hello', 'Goodbye'] diff --git a/dbms/tests/queries/00008_array_join.reference b/dbms/tests/queries/00008_array_join.reference new file mode 100644 index 00000000000..c86756d1938 --- /dev/null +++ b/dbms/tests/queries/00008_array_join.reference @@ -0,0 +1,2 @@ +Hello +Goodbye diff --git a/dbms/tests/queries/00008_array_join.sql b/dbms/tests/queries/00008_array_join.sql new file mode 100644 index 00000000000..abb35cbbfc8 --- /dev/null +++ b/dbms/tests/queries/00008_array_join.sql @@ -0,0 +1 @@ +SELECT arrayJoin(['Hello', 'Goodbye']) diff --git a/dbms/tests/queries/00009_array_join_subquery.reference b/dbms/tests/queries/00009_array_join_subquery.reference new file mode 100644 index 00000000000..c86756d1938 --- /dev/null +++ b/dbms/tests/queries/00009_array_join_subquery.reference @@ -0,0 +1,2 @@ +Hello +Goodbye diff --git a/dbms/tests/queries/00009_array_join_subquery.sql b/dbms/tests/queries/00009_array_join_subquery.sql new file mode 100644 index 00000000000..378baadd026 --- /dev/null +++ b/dbms/tests/queries/00009_array_join_subquery.sql @@ -0,0 +1 @@ +SELECT x FROM (SELECT arrayJoin(['Hello', 'Goodbye']) AS x) diff --git a/dbms/tests/queries/00010_big_array_join.reference b/dbms/tests/queries/00010_big_array_join.reference new file mode 100644 index 00000000000..e9c80bae6d0 --- /dev/null +++ b/dbms/tests/queries/00010_big_array_join.reference @@ -0,0 +1,6 @@ +Hello +Hello +Hello +Goodbye +Goodbye +Goodbye diff --git a/dbms/tests/queries/00010_big_array_join.sql b/dbms/tests/queries/00010_big_array_join.sql new file mode 100644 index 00000000000..f7b9160b578 --- /dev/null +++ b/dbms/tests/queries/00010_big_array_join.sql @@ -0,0 +1 @@ +SELECT x FROM (SELECT arrayJoin(['Hello', 'Goodbye']) AS x, [1, 2, 3] AS arr) ARRAY JOIN arr diff --git a/dbms/tests/queries/00011_array_join_alias.reference b/dbms/tests/queries/00011_array_join_alias.reference new file mode 100644 index 00000000000..ff5ad16d763 --- /dev/null +++ b/dbms/tests/queries/00011_array_join_alias.reference @@ -0,0 +1,6 @@ +Hello 1 +Hello 2 +Hello 3 +Goodbye 1 +Goodbye 2 +Goodbye 3 diff --git a/dbms/tests/queries/00011_array_join_alias.sql b/dbms/tests/queries/00011_array_join_alias.sql new file mode 100644 index 00000000000..228038c1509 --- /dev/null +++ b/dbms/tests/queries/00011_array_join_alias.sql @@ -0,0 +1 @@ +SELECT x, a FROM (SELECT arrayJoin(['Hello', 'Goodbye']) AS x, [1, 2, 3] AS arr) ARRAY JOIN arr AS a diff --git a/dbms/tests/queries/00012_array_join_alias_2.reference b/dbms/tests/queries/00012_array_join_alias_2.reference new file mode 100644 index 00000000000..ebad7b98d93 --- /dev/null +++ b/dbms/tests/queries/00012_array_join_alias_2.reference @@ -0,0 +1,6 @@ +Hello 1 [1,2,3] +Hello 2 [1,2,3] +Hello 3 [1,2,3] +Goodbye 1 [1,2,3] +Goodbye 2 [1,2,3] +Goodbye 3 [1,2,3] diff --git a/dbms/tests/queries/00012_array_join_alias_2.sql b/dbms/tests/queries/00012_array_join_alias_2.sql new file mode 100644 index 00000000000..a45cf2d87b8 --- /dev/null +++ b/dbms/tests/queries/00012_array_join_alias_2.sql @@ -0,0 +1 @@ +SELECT x, a, arr FROM (SELECT arrayJoin(['Hello', 'Goodbye']) AS x, [1, 2, 3] AS arr) ARRAY JOIN arr AS a diff --git a/dbms/tests/queries/00013_1_drop_if_exists_table_with_arrays.reference b/dbms/tests/queries/00013_1_drop_if_exists_table_with_arrays.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/00013_1_drop_if_exists_table_with_arrays.sql b/dbms/tests/queries/00013_1_drop_if_exists_table_with_arrays.sql new file mode 100644 index 00000000000..8dbef3a0abb --- /dev/null +++ b/dbms/tests/queries/00013_1_drop_if_exists_table_with_arrays.sql @@ -0,0 +1 @@ +DROP TABLE IF EXISTS arrays_test diff --git a/dbms/tests/queries/00013_2_create_table_with_arrays.reference b/dbms/tests/queries/00013_2_create_table_with_arrays.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/00013_2_create_table_with_arrays.sql b/dbms/tests/queries/00013_2_create_table_with_arrays.sql new file mode 100644 index 00000000000..8585ba14140 --- /dev/null +++ b/dbms/tests/queries/00013_2_create_table_with_arrays.sql @@ -0,0 +1 @@ +CREATE TABLE arrays_test (s String, arr Array(UInt8)) ENGINE = Memory diff --git a/dbms/tests/queries/00013_3_insert_into_table_with_arrays.reference b/dbms/tests/queries/00013_3_insert_into_table_with_arrays.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/00013_3_insert_into_table_with_arrays.sql b/dbms/tests/queries/00013_3_insert_into_table_with_arrays.sql new file mode 100644 index 00000000000..5b41e56db66 --- /dev/null +++ b/dbms/tests/queries/00013_3_insert_into_table_with_arrays.sql @@ -0,0 +1 @@ +INSERT INTO arrays_test VALUES ('Hello', [1,2]), ('World', [3,4,5]), ('Goodbye', []) diff --git a/dbms/tests/queries/00013_4_select_from_table_with_arrays.reference b/dbms/tests/queries/00013_4_select_from_table_with_arrays.reference new file mode 100644 index 00000000000..5a5af40fc00 --- /dev/null +++ b/dbms/tests/queries/00013_4_select_from_table_with_arrays.reference @@ -0,0 +1,3 @@ +Hello [1,2] +World [3,4,5] +Goodbye [] diff --git a/dbms/tests/queries/00013_4_select_from_table_with_arrays.sql b/dbms/tests/queries/00013_4_select_from_table_with_arrays.sql new file mode 100644 index 00000000000..af4b5dd8538 --- /dev/null +++ b/dbms/tests/queries/00013_4_select_from_table_with_arrays.sql @@ -0,0 +1 @@ +SELECT * FROM arrays_test diff --git a/dbms/tests/queries/00013_5_select_from_table_with_arrays.reference b/dbms/tests/queries/00013_5_select_from_table_with_arrays.reference new file mode 100644 index 00000000000..02ff6407fd6 --- /dev/null +++ b/dbms/tests/queries/00013_5_select_from_table_with_arrays.reference @@ -0,0 +1,5 @@ +Hello 1 +Hello 2 +World 3 +World 4 +World 5 diff --git a/dbms/tests/queries/00013_5_select_from_table_with_arrays.sql b/dbms/tests/queries/00013_5_select_from_table_with_arrays.sql new file mode 100644 index 00000000000..238f888202d --- /dev/null +++ b/dbms/tests/queries/00013_5_select_from_table_with_arrays.sql @@ -0,0 +1 @@ +SELECT s, arr FROM arrays_test ARRAY JOIN arr diff --git a/dbms/tests/queries/00013_6_select_from_table_with_arrays.reference b/dbms/tests/queries/00013_6_select_from_table_with_arrays.reference new file mode 100644 index 00000000000..45d362a350e --- /dev/null +++ b/dbms/tests/queries/00013_6_select_from_table_with_arrays.reference @@ -0,0 +1,5 @@ +Hello [1,2] 1 +Hello [1,2] 2 +World [3,4,5] 3 +World [3,4,5] 4 +World [3,4,5] 5 diff --git a/dbms/tests/queries/00013_6_select_from_table_with_arrays.sql b/dbms/tests/queries/00013_6_select_from_table_with_arrays.sql new file mode 100644 index 00000000000..d1c9936af30 --- /dev/null +++ b/dbms/tests/queries/00013_6_select_from_table_with_arrays.sql @@ -0,0 +1 @@ +SELECT s, arr, a FROM arrays_test ARRAY JOIN arr AS a \ No newline at end of file diff --git a/dbms/tests/queries/00013_7_select_from_table_with_arrays.reference b/dbms/tests/queries/00013_7_select_from_table_with_arrays.reference new file mode 100644 index 00000000000..577260f94f0 --- /dev/null +++ b/dbms/tests/queries/00013_7_select_from_table_with_arrays.reference @@ -0,0 +1,5 @@ +Hello [1,2] 1 1 +Hello [1,2] 2 2 +World [3,4,5] 3 1 +World [3,4,5] 4 2 +World [3,4,5] 5 3 diff --git a/dbms/tests/queries/00013_7_select_from_table_with_arrays.sql b/dbms/tests/queries/00013_7_select_from_table_with_arrays.sql new file mode 100644 index 00000000000..a75b97e6faa --- /dev/null +++ b/dbms/tests/queries/00013_7_select_from_table_with_arrays.sql @@ -0,0 +1 @@ +SELECT s, arr, a, num FROM arrays_test ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num \ No newline at end of file diff --git a/dbms/tests/queries/00013_8_select_from_table_with_arrays.reference b/dbms/tests/queries/00013_8_select_from_table_with_arrays.reference new file mode 100644 index 00000000000..e29fe6d82a1 --- /dev/null +++ b/dbms/tests/queries/00013_8_select_from_table_with_arrays.reference @@ -0,0 +1,5 @@ +Hello [1,2] 1 1 [1,2] +Hello [1,2] 2 2 [1,2] +World [3,4,5] 3 1 [1,2,3] +World [3,4,5] 4 2 [1,2,3] +World [3,4,5] 5 3 [1,2,3] diff --git a/dbms/tests/queries/00013_8_select_from_table_with_arrays.sql b/dbms/tests/queries/00013_8_select_from_table_with_arrays.sql new file mode 100644 index 00000000000..d7b3a7b44b9 --- /dev/null +++ b/dbms/tests/queries/00013_8_select_from_table_with_arrays.sql @@ -0,0 +1 @@ +SELECT s, arr, a, num, arrayEnumerate(arr) FROM arrays_test ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num \ No newline at end of file diff --git a/dbms/tests/queries/00013_9_select_from_table_with_arrays.reference b/dbms/tests/queries/00013_9_select_from_table_with_arrays.reference new file mode 100644 index 00000000000..58328ca2588 --- /dev/null +++ b/dbms/tests/queries/00013_9_select_from_table_with_arrays.reference @@ -0,0 +1,5 @@ +Hello [1,2] 1 2 +Hello [1,2] 2 3 +World [3,4,5] 3 4 +World [3,4,5] 4 5 +World [3,4,5] 5 6 diff --git a/dbms/tests/queries/00013_9_select_from_table_with_arrays.sql b/dbms/tests/queries/00013_9_select_from_table_with_arrays.sql new file mode 100644 index 00000000000..010faea3d76 --- /dev/null +++ b/dbms/tests/queries/00013_9_select_from_table_with_arrays.sql @@ -0,0 +1 @@ +SELECT s, arr, a, mapped FROM arrays_test ARRAY JOIN arr AS a, arrayMap(x -> x + 1, arr) AS mapped diff --git a/dbms/tests/queries/00013_a_select_from_table_with_arrays.reference b/dbms/tests/queries/00013_a_select_from_table_with_arrays.reference new file mode 100644 index 00000000000..14686f65be3 --- /dev/null +++ b/dbms/tests/queries/00013_a_select_from_table_with_arrays.reference @@ -0,0 +1,5 @@ +Hello [1,2] 1 1 2 +Hello [1,2] 2 2 3 +World [3,4,5] 3 1 4 +World [3,4,5] 4 2 5 +World [3,4,5] 5 3 6 diff --git a/dbms/tests/queries/00013_a_select_from_table_with_arrays.sql b/dbms/tests/queries/00013_a_select_from_table_with_arrays.sql new file mode 100644 index 00000000000..1c4b2b02e19 --- /dev/null +++ b/dbms/tests/queries/00013_a_select_from_table_with_arrays.sql @@ -0,0 +1 @@ +SELECT s, arr, a, num, mapped FROM arrays_test ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num, arrayMap(x -> x + 1, arr) AS mapped diff --git a/dbms/tests/queries/00014_1_drop_if_exists_table_with_nested.reference b/dbms/tests/queries/00014_1_drop_if_exists_table_with_nested.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/00014_1_drop_if_exists_table_with_nested.sql b/dbms/tests/queries/00014_1_drop_if_exists_table_with_nested.sql new file mode 100644 index 00000000000..257afc58919 --- /dev/null +++ b/dbms/tests/queries/00014_1_drop_if_exists_table_with_nested.sql @@ -0,0 +1 @@ +DROP TABLE IF EXISTS nested_test diff --git a/dbms/tests/queries/00014_2_create_table_with_nested.reference b/dbms/tests/queries/00014_2_create_table_with_nested.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/00014_2_create_table_with_nested.sql b/dbms/tests/queries/00014_2_create_table_with_nested.sql new file mode 100644 index 00000000000..49f015ce1aa --- /dev/null +++ b/dbms/tests/queries/00014_2_create_table_with_nested.sql @@ -0,0 +1 @@ +CREATE TABLE nested_test (s String, nest Nested(x UInt8, y UInt32)) ENGINE = Memory diff --git a/dbms/tests/queries/00014_3_insert_into_table_with_nested.reference b/dbms/tests/queries/00014_3_insert_into_table_with_nested.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/00014_3_insert_into_table_with_nested.sql b/dbms/tests/queries/00014_3_insert_into_table_with_nested.sql new file mode 100644 index 00000000000..44fdbc820ff --- /dev/null +++ b/dbms/tests/queries/00014_3_insert_into_table_with_nested.sql @@ -0,0 +1 @@ +INSERT INTO nested_test VALUES ('Hello', [1,2], [10,20]), ('World', [3,4,5], [30,40,50]), ('Goodbye', [], []) diff --git a/dbms/tests/queries/00014_4_select_from_table_with_nested.reference b/dbms/tests/queries/00014_4_select_from_table_with_nested.reference new file mode 100644 index 00000000000..32bd08bad02 --- /dev/null +++ b/dbms/tests/queries/00014_4_select_from_table_with_nested.reference @@ -0,0 +1,3 @@ +Hello [1,2] [10,20] +World [3,4,5] [30,40,50] +Goodbye [] [] diff --git a/dbms/tests/queries/00014_4_select_from_table_with_nested.sql b/dbms/tests/queries/00014_4_select_from_table_with_nested.sql new file mode 100644 index 00000000000..509911959ac --- /dev/null +++ b/dbms/tests/queries/00014_4_select_from_table_with_nested.sql @@ -0,0 +1 @@ +SELECT * FROM nested_test diff --git a/dbms/tests/queries/00014_5_select_from_table_with_nested.reference b/dbms/tests/queries/00014_5_select_from_table_with_nested.reference new file mode 100644 index 00000000000..05200edd33b --- /dev/null +++ b/dbms/tests/queries/00014_5_select_from_table_with_nested.reference @@ -0,0 +1,5 @@ +Hello 1 10 +Hello 2 20 +World 3 30 +World 4 40 +World 5 50 diff --git a/dbms/tests/queries/00014_5_select_from_table_with_nested.sql b/dbms/tests/queries/00014_5_select_from_table_with_nested.sql new file mode 100644 index 00000000000..31e1ac1b05e --- /dev/null +++ b/dbms/tests/queries/00014_5_select_from_table_with_nested.sql @@ -0,0 +1 @@ +SELECT s, nest.x, nest.y FROM nested_test ARRAY JOIN nest diff --git a/dbms/tests/queries/00014_6_select_from_table_with_nested.reference b/dbms/tests/queries/00014_6_select_from_table_with_nested.reference new file mode 100644 index 00000000000..8ea4568b73c --- /dev/null +++ b/dbms/tests/queries/00014_6_select_from_table_with_nested.reference @@ -0,0 +1,5 @@ +Hello 1 [10,20] +Hello 2 [10,20] +World 3 [30,40,50] +World 4 [30,40,50] +World 5 [30,40,50] diff --git a/dbms/tests/queries/00014_6_select_from_table_with_nested.sql b/dbms/tests/queries/00014_6_select_from_table_with_nested.sql new file mode 100644 index 00000000000..3648cb4852c --- /dev/null +++ b/dbms/tests/queries/00014_6_select_from_table_with_nested.sql @@ -0,0 +1 @@ +SELECT s, nest.x, nest.y FROM nested_test ARRAY JOIN nest.x diff --git a/dbms/tests/queries/00014_7_select_from_table_with_nested.reference b/dbms/tests/queries/00014_7_select_from_table_with_nested.reference new file mode 100644 index 00000000000..05200edd33b --- /dev/null +++ b/dbms/tests/queries/00014_7_select_from_table_with_nested.reference @@ -0,0 +1,5 @@ +Hello 1 10 +Hello 2 20 +World 3 30 +World 4 40 +World 5 50 diff --git a/dbms/tests/queries/00014_7_select_from_table_with_nested.sql b/dbms/tests/queries/00014_7_select_from_table_with_nested.sql new file mode 100644 index 00000000000..9380d7d285f --- /dev/null +++ b/dbms/tests/queries/00014_7_select_from_table_with_nested.sql @@ -0,0 +1 @@ +SELECT s, nest.x, nest.y FROM nested_test ARRAY JOIN nest.x, nest.y \ No newline at end of file diff --git a/dbms/tests/queries/00014_9_select_from_table_with_nested.reference b/dbms/tests/queries/00014_9_select_from_table_with_nested.reference new file mode 100644 index 00000000000..05200edd33b --- /dev/null +++ b/dbms/tests/queries/00014_9_select_from_table_with_nested.reference @@ -0,0 +1,5 @@ +Hello 1 10 +Hello 2 20 +World 3 30 +World 4 40 +World 5 50 diff --git a/dbms/tests/queries/00014_9_select_from_table_with_nested.sql b/dbms/tests/queries/00014_9_select_from_table_with_nested.sql new file mode 100644 index 00000000000..945f3dc79a1 --- /dev/null +++ b/dbms/tests/queries/00014_9_select_from_table_with_nested.sql @@ -0,0 +1 @@ +SELECT s, n.x, n.y FROM nested_test ARRAY JOIN nest AS n \ No newline at end of file diff --git a/dbms/tests/queries/00014_a_select_from_table_with_nested.reference b/dbms/tests/queries/00014_a_select_from_table_with_nested.reference new file mode 100644 index 00000000000..d40754269c6 --- /dev/null +++ b/dbms/tests/queries/00014_a_select_from_table_with_nested.reference @@ -0,0 +1,5 @@ +Hello 1 10 [1,2] +Hello 2 20 [1,2] +World 3 30 [3,4,5] +World 4 40 [3,4,5] +World 5 50 [3,4,5] diff --git a/dbms/tests/queries/00014_a_select_from_table_with_nested.sql b/dbms/tests/queries/00014_a_select_from_table_with_nested.sql new file mode 100644 index 00000000000..4e275aa4e12 --- /dev/null +++ b/dbms/tests/queries/00014_a_select_from_table_with_nested.sql @@ -0,0 +1 @@ +SELECT s, n.x, n.y, nest.x FROM nested_test ARRAY JOIN nest AS n \ No newline at end of file diff --git a/dbms/tests/queries/00014_b_select_from_table_with_nested.reference b/dbms/tests/queries/00014_b_select_from_table_with_nested.reference new file mode 100644 index 00000000000..7129270657e --- /dev/null +++ b/dbms/tests/queries/00014_b_select_from_table_with_nested.reference @@ -0,0 +1,5 @@ +Hello 1 10 [1,2] [10,20] +Hello 2 20 [1,2] [10,20] +World 3 30 [3,4,5] [30,40,50] +World 4 40 [3,4,5] [30,40,50] +World 5 50 [3,4,5] [30,40,50] diff --git a/dbms/tests/queries/00014_b_select_from_table_with_nested.sql b/dbms/tests/queries/00014_b_select_from_table_with_nested.sql new file mode 100644 index 00000000000..4643e428942 --- /dev/null +++ b/dbms/tests/queries/00014_b_select_from_table_with_nested.sql @@ -0,0 +1 @@ +SELECT s, n.x, n.y, nest.x, nest.y FROM nested_test ARRAY JOIN nest AS n \ No newline at end of file diff --git a/dbms/tests/queries/00014_c_select_from_table_with_nested.reference b/dbms/tests/queries/00014_c_select_from_table_with_nested.reference new file mode 100644 index 00000000000..fa7a8052ce9 --- /dev/null +++ b/dbms/tests/queries/00014_c_select_from_table_with_nested.reference @@ -0,0 +1,5 @@ +Hello 1 10 [1,2] [10,20] 1 +Hello 2 20 [1,2] [10,20] 2 +World 3 30 [3,4,5] [30,40,50] 1 +World 4 40 [3,4,5] [30,40,50] 2 +World 5 50 [3,4,5] [30,40,50] 3 diff --git a/dbms/tests/queries/00014_c_select_from_table_with_nested.sql b/dbms/tests/queries/00014_c_select_from_table_with_nested.sql new file mode 100644 index 00000000000..6033dd1c011 --- /dev/null +++ b/dbms/tests/queries/00014_c_select_from_table_with_nested.sql @@ -0,0 +1 @@ +SELECT s, n.x, n.y, nest.x, nest.y, num FROM nested_test ARRAY JOIN nest AS n, arrayEnumerate(nest.x) AS num \ No newline at end of file diff --git a/dbms/tests/queries/00015_totals_having_constants.reference b/dbms/tests/queries/00015_totals_having_constants.reference new file mode 100644 index 00000000000..1b62b5f7c43 --- /dev/null +++ b/dbms/tests/queries/00015_totals_having_constants.reference @@ -0,0 +1,12 @@ +0 10 +1 10 +5 10 +7 10 +2 10 +6 10 +4 10 +8 10 +3 10 +9 10 + +0 100 diff --git a/dbms/tests/queries/00015_totals_having_constants.sql b/dbms/tests/queries/00015_totals_having_constants.sql new file mode 100644 index 00000000000..586bef3ac13 --- /dev/null +++ b/dbms/tests/queries/00015_totals_having_constants.sql @@ -0,0 +1 @@ +SELECT number, count() / 0.1 FROM (SELECT number FROM system.numbers LIMIT 10) GROUP BY number WITH TOTALS HAVING count() > 0.1 diff --git a/dbms/tests/queries/00016_totals_having_constants.reference b/dbms/tests/queries/00016_totals_having_constants.reference new file mode 100644 index 00000000000..4804131baba --- /dev/null +++ b/dbms/tests/queries/00016_totals_having_constants.reference @@ -0,0 +1,3 @@ +0 10 + +0 10 diff --git a/dbms/tests/queries/00016_totals_having_constants.sql b/dbms/tests/queries/00016_totals_having_constants.sql new file mode 100644 index 00000000000..c50659b8140 --- /dev/null +++ b/dbms/tests/queries/00016_totals_having_constants.sql @@ -0,0 +1 @@ +SELECT dummy, count() / 0.1 GROUP BY dummy WITH TOTALS HAVING count() > 0.1 diff --git a/dbms/tests/queries/00017_in_subquery_with_empty_result.reference b/dbms/tests/queries/00017_in_subquery_with_empty_result.reference new file mode 100644 index 00000000000..e25c5780b65 --- /dev/null +++ b/dbms/tests/queries/00017_in_subquery_with_empty_result.reference @@ -0,0 +1,18 @@ +{ + "meta": + [ + { + "name": "count()", + "type": "UInt64" + } + ], + + "data": + [ + + ], + + "rows": 0, + + "rows_before_limit_at_least": 1000 +} diff --git a/dbms/tests/queries/00017_in_subquery_with_empty_result.sql b/dbms/tests/queries/00017_in_subquery_with_empty_result.sql new file mode 100644 index 00000000000..b83d597159d --- /dev/null +++ b/dbms/tests/queries/00017_in_subquery_with_empty_result.sql @@ -0,0 +1,2 @@ +SELECT count() FROM (SELECT * FROM system.numbers LIMIT 1000) WHERE 1 IN (SELECT 0 WHERE 0) +FORMAT JSON diff --git a/dbms/tests/queries/00018_distinct_in_subquery.reference b/dbms/tests/queries/00018_distinct_in_subquery.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/dbms/tests/queries/00018_distinct_in_subquery.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/dbms/tests/queries/00018_distinct_in_subquery.sql b/dbms/tests/queries/00018_distinct_in_subquery.sql new file mode 100644 index 00000000000..b0bce846904 --- /dev/null +++ b/dbms/tests/queries/00018_distinct_in_subquery.sql @@ -0,0 +1 @@ +SELECT x FROM (SELECT DISTINCT 1 AS x, arrayJoin([1, 2]) AS y) From 2813909ed3c92f41ee83d4eeeacd9c27726cb21c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Mar 2014 00:45:42 +0400 Subject: [PATCH 11/56] dbms: modified test [#METR-2944]. --- dbms/tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 1bfd91ab468..4c918cf30a7 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -24,7 +24,7 @@ ERRORS=0 for query_file in $(ls $QUERIES_DIR/*.sql) do - test_name=$(basename -s .sql $query_file) + test_name=$(basename $query_file .sql) result_file=$QUERIES_DIR/$test_name.result error_file=$QUERIES_DIR/$test_name.error From e2292225ef4a1bfd6ab948ceea897d4541d7aeb4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Mar 2014 04:37:39 +0400 Subject: [PATCH 12/56] Added gitignore [#METR-2944]. --- dbms/tests/.gitignore | 4 ++++ 1 file changed, 4 insertions(+) create mode 100644 dbms/tests/.gitignore diff --git a/dbms/tests/.gitignore b/dbms/tests/.gitignore new file mode 100644 index 00000000000..3b3e487c63a --- /dev/null +++ b/dbms/tests/.gitignore @@ -0,0 +1,4 @@ +*.result +*.diff +*.error +test_data From 8aaecae5fbb7044c44939fe214899c2e9633b9bd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Mar 2014 06:50:25 +0400 Subject: [PATCH 13/56] dbms: updated tests [#METR-2944]. --- dbms/tests/queries/{ => 0_stateless}/00001_select_1.reference | 0 dbms/tests/queries/{ => 0_stateless}/00001_select_1.sql | 0 .../queries/{ => 0_stateless}/00002_system_numbers.reference | 0 dbms/tests/queries/{ => 0_stateless}/00002_system_numbers.sql | 0 .../{ => 0_stateless}/00003_reinterpret_as_string.reference | 0 .../queries/{ => 0_stateless}/00003_reinterpret_as_string.sql | 0 .../{ => 0_stateless}/00004_format_ast_and_remote_table.reference | 0 .../{ => 0_stateless}/00004_format_ast_and_remote_table.sql | 0 .../00005_format_ast_and_remote_table_lambda.reference | 0 .../00005_format_ast_and_remote_table_lambda.sql | 0 .../queries/{ => 0_stateless}/00006_1_set_extremes.reference | 0 dbms/tests/queries/{ => 0_stateless}/00006_1_set_extremes.sql | 0 .../00006_2_extremes_and_subquery_from.reference | 0 .../{ => 0_stateless}/00006_2_extremes_and_subquery_from.sql | 0 .../queries/{ => 0_stateless}/00006_3_unset_extremes.reference | 0 dbms/tests/queries/{ => 0_stateless}/00006_3_unset_extremes.sql | 0 dbms/tests/queries/{ => 0_stateless}/00007_array.reference | 0 dbms/tests/queries/{ => 0_stateless}/00007_array.sql | 0 dbms/tests/queries/{ => 0_stateless}/00008_array_join.reference | 0 dbms/tests/queries/{ => 0_stateless}/00008_array_join.sql | 0 .../queries/{ => 0_stateless}/00009_array_join_subquery.reference | 0 .../tests/queries/{ => 0_stateless}/00009_array_join_subquery.sql | 0 .../queries/{ => 0_stateless}/00010_big_array_join.reference | 0 dbms/tests/queries/{ => 0_stateless}/00010_big_array_join.sql | 0 .../queries/{ => 0_stateless}/00011_array_join_alias.reference | 0 dbms/tests/queries/{ => 0_stateless}/00011_array_join_alias.sql | 0 .../queries/{ => 0_stateless}/00012_array_join_alias_2.reference | 0 dbms/tests/queries/{ => 0_stateless}/00012_array_join_alias_2.sql | 0 .../00013_1_drop_if_exists_table_with_arrays.reference | 0 .../00013_1_drop_if_exists_table_with_arrays.sql | 0 .../{ => 0_stateless}/00013_2_create_table_with_arrays.reference | 0 .../{ => 0_stateless}/00013_2_create_table_with_arrays.sql | 0 .../00013_3_insert_into_table_with_arrays.reference | 0 .../{ => 0_stateless}/00013_3_insert_into_table_with_arrays.sql | 0 .../00013_4_select_from_table_with_arrays.reference | 0 .../{ => 0_stateless}/00013_4_select_from_table_with_arrays.sql | 0 .../00013_5_select_from_table_with_arrays.reference | 0 .../{ => 0_stateless}/00013_5_select_from_table_with_arrays.sql | 0 .../00013_6_select_from_table_with_arrays.reference | 0 .../{ => 0_stateless}/00013_6_select_from_table_with_arrays.sql | 0 .../00013_7_select_from_table_with_arrays.reference | 0 .../{ => 0_stateless}/00013_7_select_from_table_with_arrays.sql | 0 .../00013_8_select_from_table_with_arrays.reference | 0 .../{ => 0_stateless}/00013_8_select_from_table_with_arrays.sql | 0 .../00013_9_select_from_table_with_arrays.reference | 0 .../{ => 0_stateless}/00013_9_select_from_table_with_arrays.sql | 0 .../00013_a_select_from_table_with_arrays.reference | 0 .../{ => 0_stateless}/00013_a_select_from_table_with_arrays.sql | 0 .../00014_1_drop_if_exists_table_with_nested.reference | 0 .../00014_1_drop_if_exists_table_with_nested.sql | 0 .../{ => 0_stateless}/00014_2_create_table_with_nested.reference | 0 .../{ => 0_stateless}/00014_2_create_table_with_nested.sql | 0 .../00014_3_insert_into_table_with_nested.reference | 0 .../{ => 0_stateless}/00014_3_insert_into_table_with_nested.sql | 0 .../00014_4_select_from_table_with_nested.reference | 0 .../{ => 0_stateless}/00014_4_select_from_table_with_nested.sql | 0 .../00014_5_select_from_table_with_nested.reference | 0 .../{ => 0_stateless}/00014_5_select_from_table_with_nested.sql | 0 .../00014_6_select_from_table_with_nested.reference | 0 .../{ => 0_stateless}/00014_6_select_from_table_with_nested.sql | 0 .../00014_7_select_from_table_with_nested.reference | 0 .../{ => 0_stateless}/00014_7_select_from_table_with_nested.sql | 0 .../00014_9_select_from_table_with_nested.reference | 0 .../{ => 0_stateless}/00014_9_select_from_table_with_nested.sql | 0 .../00014_a_select_from_table_with_nested.reference | 0 .../{ => 0_stateless}/00014_a_select_from_table_with_nested.sql | 0 .../00014_b_select_from_table_with_nested.reference | 0 .../{ => 0_stateless}/00014_b_select_from_table_with_nested.sql | 0 .../00014_c_select_from_table_with_nested.reference | 0 .../{ => 0_stateless}/00014_c_select_from_table_with_nested.sql | 0 .../{ => 0_stateless}/00015_totals_having_constants.reference | 0 .../queries/{ => 0_stateless}/00015_totals_having_constants.sql | 0 .../{ => 0_stateless}/00016_totals_having_constants.reference | 0 .../queries/{ => 0_stateless}/00016_totals_having_constants.sql | 0 .../00017_in_subquery_with_empty_result.reference | 0 .../{ => 0_stateless}/00017_in_subquery_with_empty_result.sql | 0 .../{ => 0_stateless}/00018_distinct_in_subquery.reference | 0 .../queries/{ => 0_stateless}/00018_distinct_in_subquery.sql | 0 78 files changed, 0 insertions(+), 0 deletions(-) rename dbms/tests/queries/{ => 0_stateless}/00001_select_1.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00001_select_1.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00002_system_numbers.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00002_system_numbers.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00003_reinterpret_as_string.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00003_reinterpret_as_string.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00004_format_ast_and_remote_table.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00004_format_ast_and_remote_table.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00005_format_ast_and_remote_table_lambda.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00005_format_ast_and_remote_table_lambda.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00006_1_set_extremes.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00006_1_set_extremes.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00006_2_extremes_and_subquery_from.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00006_2_extremes_and_subquery_from.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00006_3_unset_extremes.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00006_3_unset_extremes.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00007_array.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00007_array.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00008_array_join.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00008_array_join.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00009_array_join_subquery.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00009_array_join_subquery.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00010_big_array_join.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00010_big_array_join.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00011_array_join_alias.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00011_array_join_alias.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00012_array_join_alias_2.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00012_array_join_alias_2.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_1_drop_if_exists_table_with_arrays.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_1_drop_if_exists_table_with_arrays.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_2_create_table_with_arrays.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_2_create_table_with_arrays.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_3_insert_into_table_with_arrays.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_3_insert_into_table_with_arrays.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_4_select_from_table_with_arrays.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_4_select_from_table_with_arrays.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_5_select_from_table_with_arrays.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_5_select_from_table_with_arrays.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_6_select_from_table_with_arrays.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_6_select_from_table_with_arrays.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_7_select_from_table_with_arrays.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_7_select_from_table_with_arrays.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_8_select_from_table_with_arrays.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_8_select_from_table_with_arrays.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_9_select_from_table_with_arrays.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_9_select_from_table_with_arrays.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_a_select_from_table_with_arrays.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00013_a_select_from_table_with_arrays.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_1_drop_if_exists_table_with_nested.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_1_drop_if_exists_table_with_nested.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_2_create_table_with_nested.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_2_create_table_with_nested.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_3_insert_into_table_with_nested.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_3_insert_into_table_with_nested.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_4_select_from_table_with_nested.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_4_select_from_table_with_nested.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_5_select_from_table_with_nested.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_5_select_from_table_with_nested.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_6_select_from_table_with_nested.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_6_select_from_table_with_nested.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_7_select_from_table_with_nested.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_7_select_from_table_with_nested.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_9_select_from_table_with_nested.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_9_select_from_table_with_nested.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_a_select_from_table_with_nested.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_a_select_from_table_with_nested.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_b_select_from_table_with_nested.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_b_select_from_table_with_nested.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_c_select_from_table_with_nested.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00014_c_select_from_table_with_nested.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00015_totals_having_constants.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00015_totals_having_constants.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00016_totals_having_constants.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00016_totals_having_constants.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00017_in_subquery_with_empty_result.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00017_in_subquery_with_empty_result.sql (100%) rename dbms/tests/queries/{ => 0_stateless}/00018_distinct_in_subquery.reference (100%) rename dbms/tests/queries/{ => 0_stateless}/00018_distinct_in_subquery.sql (100%) diff --git a/dbms/tests/queries/00001_select_1.reference b/dbms/tests/queries/0_stateless/00001_select_1.reference similarity index 100% rename from dbms/tests/queries/00001_select_1.reference rename to dbms/tests/queries/0_stateless/00001_select_1.reference diff --git a/dbms/tests/queries/00001_select_1.sql b/dbms/tests/queries/0_stateless/00001_select_1.sql similarity index 100% rename from dbms/tests/queries/00001_select_1.sql rename to dbms/tests/queries/0_stateless/00001_select_1.sql diff --git a/dbms/tests/queries/00002_system_numbers.reference b/dbms/tests/queries/0_stateless/00002_system_numbers.reference similarity index 100% rename from dbms/tests/queries/00002_system_numbers.reference rename to dbms/tests/queries/0_stateless/00002_system_numbers.reference diff --git a/dbms/tests/queries/00002_system_numbers.sql b/dbms/tests/queries/0_stateless/00002_system_numbers.sql similarity index 100% rename from dbms/tests/queries/00002_system_numbers.sql rename to dbms/tests/queries/0_stateless/00002_system_numbers.sql diff --git a/dbms/tests/queries/00003_reinterpret_as_string.reference b/dbms/tests/queries/0_stateless/00003_reinterpret_as_string.reference similarity index 100% rename from dbms/tests/queries/00003_reinterpret_as_string.reference rename to dbms/tests/queries/0_stateless/00003_reinterpret_as_string.reference diff --git a/dbms/tests/queries/00003_reinterpret_as_string.sql b/dbms/tests/queries/0_stateless/00003_reinterpret_as_string.sql similarity index 100% rename from dbms/tests/queries/00003_reinterpret_as_string.sql rename to dbms/tests/queries/0_stateless/00003_reinterpret_as_string.sql diff --git a/dbms/tests/queries/00004_format_ast_and_remote_table.reference b/dbms/tests/queries/0_stateless/00004_format_ast_and_remote_table.reference similarity index 100% rename from dbms/tests/queries/00004_format_ast_and_remote_table.reference rename to dbms/tests/queries/0_stateless/00004_format_ast_and_remote_table.reference diff --git a/dbms/tests/queries/00004_format_ast_and_remote_table.sql b/dbms/tests/queries/0_stateless/00004_format_ast_and_remote_table.sql similarity index 100% rename from dbms/tests/queries/00004_format_ast_and_remote_table.sql rename to dbms/tests/queries/0_stateless/00004_format_ast_and_remote_table.sql diff --git a/dbms/tests/queries/00005_format_ast_and_remote_table_lambda.reference b/dbms/tests/queries/0_stateless/00005_format_ast_and_remote_table_lambda.reference similarity index 100% rename from dbms/tests/queries/00005_format_ast_and_remote_table_lambda.reference rename to dbms/tests/queries/0_stateless/00005_format_ast_and_remote_table_lambda.reference diff --git a/dbms/tests/queries/00005_format_ast_and_remote_table_lambda.sql b/dbms/tests/queries/0_stateless/00005_format_ast_and_remote_table_lambda.sql similarity index 100% rename from dbms/tests/queries/00005_format_ast_and_remote_table_lambda.sql rename to dbms/tests/queries/0_stateless/00005_format_ast_and_remote_table_lambda.sql diff --git a/dbms/tests/queries/00006_1_set_extremes.reference b/dbms/tests/queries/0_stateless/00006_1_set_extremes.reference similarity index 100% rename from dbms/tests/queries/00006_1_set_extremes.reference rename to dbms/tests/queries/0_stateless/00006_1_set_extremes.reference diff --git a/dbms/tests/queries/00006_1_set_extremes.sql b/dbms/tests/queries/0_stateless/00006_1_set_extremes.sql similarity index 100% rename from dbms/tests/queries/00006_1_set_extremes.sql rename to dbms/tests/queries/0_stateless/00006_1_set_extremes.sql diff --git a/dbms/tests/queries/00006_2_extremes_and_subquery_from.reference b/dbms/tests/queries/0_stateless/00006_2_extremes_and_subquery_from.reference similarity index 100% rename from dbms/tests/queries/00006_2_extremes_and_subquery_from.reference rename to dbms/tests/queries/0_stateless/00006_2_extremes_and_subquery_from.reference diff --git a/dbms/tests/queries/00006_2_extremes_and_subquery_from.sql b/dbms/tests/queries/0_stateless/00006_2_extremes_and_subquery_from.sql similarity index 100% rename from dbms/tests/queries/00006_2_extremes_and_subquery_from.sql rename to dbms/tests/queries/0_stateless/00006_2_extremes_and_subquery_from.sql diff --git a/dbms/tests/queries/00006_3_unset_extremes.reference b/dbms/tests/queries/0_stateless/00006_3_unset_extremes.reference similarity index 100% rename from dbms/tests/queries/00006_3_unset_extremes.reference rename to dbms/tests/queries/0_stateless/00006_3_unset_extremes.reference diff --git a/dbms/tests/queries/00006_3_unset_extremes.sql b/dbms/tests/queries/0_stateless/00006_3_unset_extremes.sql similarity index 100% rename from dbms/tests/queries/00006_3_unset_extremes.sql rename to dbms/tests/queries/0_stateless/00006_3_unset_extremes.sql diff --git a/dbms/tests/queries/00007_array.reference b/dbms/tests/queries/0_stateless/00007_array.reference similarity index 100% rename from dbms/tests/queries/00007_array.reference rename to dbms/tests/queries/0_stateless/00007_array.reference diff --git a/dbms/tests/queries/00007_array.sql b/dbms/tests/queries/0_stateless/00007_array.sql similarity index 100% rename from dbms/tests/queries/00007_array.sql rename to dbms/tests/queries/0_stateless/00007_array.sql diff --git a/dbms/tests/queries/00008_array_join.reference b/dbms/tests/queries/0_stateless/00008_array_join.reference similarity index 100% rename from dbms/tests/queries/00008_array_join.reference rename to dbms/tests/queries/0_stateless/00008_array_join.reference diff --git a/dbms/tests/queries/00008_array_join.sql b/dbms/tests/queries/0_stateless/00008_array_join.sql similarity index 100% rename from dbms/tests/queries/00008_array_join.sql rename to dbms/tests/queries/0_stateless/00008_array_join.sql diff --git a/dbms/tests/queries/00009_array_join_subquery.reference b/dbms/tests/queries/0_stateless/00009_array_join_subquery.reference similarity index 100% rename from dbms/tests/queries/00009_array_join_subquery.reference rename to dbms/tests/queries/0_stateless/00009_array_join_subquery.reference diff --git a/dbms/tests/queries/00009_array_join_subquery.sql b/dbms/tests/queries/0_stateless/00009_array_join_subquery.sql similarity index 100% rename from dbms/tests/queries/00009_array_join_subquery.sql rename to dbms/tests/queries/0_stateless/00009_array_join_subquery.sql diff --git a/dbms/tests/queries/00010_big_array_join.reference b/dbms/tests/queries/0_stateless/00010_big_array_join.reference similarity index 100% rename from dbms/tests/queries/00010_big_array_join.reference rename to dbms/tests/queries/0_stateless/00010_big_array_join.reference diff --git a/dbms/tests/queries/00010_big_array_join.sql b/dbms/tests/queries/0_stateless/00010_big_array_join.sql similarity index 100% rename from dbms/tests/queries/00010_big_array_join.sql rename to dbms/tests/queries/0_stateless/00010_big_array_join.sql diff --git a/dbms/tests/queries/00011_array_join_alias.reference b/dbms/tests/queries/0_stateless/00011_array_join_alias.reference similarity index 100% rename from dbms/tests/queries/00011_array_join_alias.reference rename to dbms/tests/queries/0_stateless/00011_array_join_alias.reference diff --git a/dbms/tests/queries/00011_array_join_alias.sql b/dbms/tests/queries/0_stateless/00011_array_join_alias.sql similarity index 100% rename from dbms/tests/queries/00011_array_join_alias.sql rename to dbms/tests/queries/0_stateless/00011_array_join_alias.sql diff --git a/dbms/tests/queries/00012_array_join_alias_2.reference b/dbms/tests/queries/0_stateless/00012_array_join_alias_2.reference similarity index 100% rename from dbms/tests/queries/00012_array_join_alias_2.reference rename to dbms/tests/queries/0_stateless/00012_array_join_alias_2.reference diff --git a/dbms/tests/queries/00012_array_join_alias_2.sql b/dbms/tests/queries/0_stateless/00012_array_join_alias_2.sql similarity index 100% rename from dbms/tests/queries/00012_array_join_alias_2.sql rename to dbms/tests/queries/0_stateless/00012_array_join_alias_2.sql diff --git a/dbms/tests/queries/00013_1_drop_if_exists_table_with_arrays.reference b/dbms/tests/queries/0_stateless/00013_1_drop_if_exists_table_with_arrays.reference similarity index 100% rename from dbms/tests/queries/00013_1_drop_if_exists_table_with_arrays.reference rename to dbms/tests/queries/0_stateless/00013_1_drop_if_exists_table_with_arrays.reference diff --git a/dbms/tests/queries/00013_1_drop_if_exists_table_with_arrays.sql b/dbms/tests/queries/0_stateless/00013_1_drop_if_exists_table_with_arrays.sql similarity index 100% rename from dbms/tests/queries/00013_1_drop_if_exists_table_with_arrays.sql rename to dbms/tests/queries/0_stateless/00013_1_drop_if_exists_table_with_arrays.sql diff --git a/dbms/tests/queries/00013_2_create_table_with_arrays.reference b/dbms/tests/queries/0_stateless/00013_2_create_table_with_arrays.reference similarity index 100% rename from dbms/tests/queries/00013_2_create_table_with_arrays.reference rename to dbms/tests/queries/0_stateless/00013_2_create_table_with_arrays.reference diff --git a/dbms/tests/queries/00013_2_create_table_with_arrays.sql b/dbms/tests/queries/0_stateless/00013_2_create_table_with_arrays.sql similarity index 100% rename from dbms/tests/queries/00013_2_create_table_with_arrays.sql rename to dbms/tests/queries/0_stateless/00013_2_create_table_with_arrays.sql diff --git a/dbms/tests/queries/00013_3_insert_into_table_with_arrays.reference b/dbms/tests/queries/0_stateless/00013_3_insert_into_table_with_arrays.reference similarity index 100% rename from dbms/tests/queries/00013_3_insert_into_table_with_arrays.reference rename to dbms/tests/queries/0_stateless/00013_3_insert_into_table_with_arrays.reference diff --git a/dbms/tests/queries/00013_3_insert_into_table_with_arrays.sql b/dbms/tests/queries/0_stateless/00013_3_insert_into_table_with_arrays.sql similarity index 100% rename from dbms/tests/queries/00013_3_insert_into_table_with_arrays.sql rename to dbms/tests/queries/0_stateless/00013_3_insert_into_table_with_arrays.sql diff --git a/dbms/tests/queries/00013_4_select_from_table_with_arrays.reference b/dbms/tests/queries/0_stateless/00013_4_select_from_table_with_arrays.reference similarity index 100% rename from dbms/tests/queries/00013_4_select_from_table_with_arrays.reference rename to dbms/tests/queries/0_stateless/00013_4_select_from_table_with_arrays.reference diff --git a/dbms/tests/queries/00013_4_select_from_table_with_arrays.sql b/dbms/tests/queries/0_stateless/00013_4_select_from_table_with_arrays.sql similarity index 100% rename from dbms/tests/queries/00013_4_select_from_table_with_arrays.sql rename to dbms/tests/queries/0_stateless/00013_4_select_from_table_with_arrays.sql diff --git a/dbms/tests/queries/00013_5_select_from_table_with_arrays.reference b/dbms/tests/queries/0_stateless/00013_5_select_from_table_with_arrays.reference similarity index 100% rename from dbms/tests/queries/00013_5_select_from_table_with_arrays.reference rename to dbms/tests/queries/0_stateless/00013_5_select_from_table_with_arrays.reference diff --git a/dbms/tests/queries/00013_5_select_from_table_with_arrays.sql b/dbms/tests/queries/0_stateless/00013_5_select_from_table_with_arrays.sql similarity index 100% rename from dbms/tests/queries/00013_5_select_from_table_with_arrays.sql rename to dbms/tests/queries/0_stateless/00013_5_select_from_table_with_arrays.sql diff --git a/dbms/tests/queries/00013_6_select_from_table_with_arrays.reference b/dbms/tests/queries/0_stateless/00013_6_select_from_table_with_arrays.reference similarity index 100% rename from dbms/tests/queries/00013_6_select_from_table_with_arrays.reference rename to dbms/tests/queries/0_stateless/00013_6_select_from_table_with_arrays.reference diff --git a/dbms/tests/queries/00013_6_select_from_table_with_arrays.sql b/dbms/tests/queries/0_stateless/00013_6_select_from_table_with_arrays.sql similarity index 100% rename from dbms/tests/queries/00013_6_select_from_table_with_arrays.sql rename to dbms/tests/queries/0_stateless/00013_6_select_from_table_with_arrays.sql diff --git a/dbms/tests/queries/00013_7_select_from_table_with_arrays.reference b/dbms/tests/queries/0_stateless/00013_7_select_from_table_with_arrays.reference similarity index 100% rename from dbms/tests/queries/00013_7_select_from_table_with_arrays.reference rename to dbms/tests/queries/0_stateless/00013_7_select_from_table_with_arrays.reference diff --git a/dbms/tests/queries/00013_7_select_from_table_with_arrays.sql b/dbms/tests/queries/0_stateless/00013_7_select_from_table_with_arrays.sql similarity index 100% rename from dbms/tests/queries/00013_7_select_from_table_with_arrays.sql rename to dbms/tests/queries/0_stateless/00013_7_select_from_table_with_arrays.sql diff --git a/dbms/tests/queries/00013_8_select_from_table_with_arrays.reference b/dbms/tests/queries/0_stateless/00013_8_select_from_table_with_arrays.reference similarity index 100% rename from dbms/tests/queries/00013_8_select_from_table_with_arrays.reference rename to dbms/tests/queries/0_stateless/00013_8_select_from_table_with_arrays.reference diff --git a/dbms/tests/queries/00013_8_select_from_table_with_arrays.sql b/dbms/tests/queries/0_stateless/00013_8_select_from_table_with_arrays.sql similarity index 100% rename from dbms/tests/queries/00013_8_select_from_table_with_arrays.sql rename to dbms/tests/queries/0_stateless/00013_8_select_from_table_with_arrays.sql diff --git a/dbms/tests/queries/00013_9_select_from_table_with_arrays.reference b/dbms/tests/queries/0_stateless/00013_9_select_from_table_with_arrays.reference similarity index 100% rename from dbms/tests/queries/00013_9_select_from_table_with_arrays.reference rename to dbms/tests/queries/0_stateless/00013_9_select_from_table_with_arrays.reference diff --git a/dbms/tests/queries/00013_9_select_from_table_with_arrays.sql b/dbms/tests/queries/0_stateless/00013_9_select_from_table_with_arrays.sql similarity index 100% rename from dbms/tests/queries/00013_9_select_from_table_with_arrays.sql rename to dbms/tests/queries/0_stateless/00013_9_select_from_table_with_arrays.sql diff --git a/dbms/tests/queries/00013_a_select_from_table_with_arrays.reference b/dbms/tests/queries/0_stateless/00013_a_select_from_table_with_arrays.reference similarity index 100% rename from dbms/tests/queries/00013_a_select_from_table_with_arrays.reference rename to dbms/tests/queries/0_stateless/00013_a_select_from_table_with_arrays.reference diff --git a/dbms/tests/queries/00013_a_select_from_table_with_arrays.sql b/dbms/tests/queries/0_stateless/00013_a_select_from_table_with_arrays.sql similarity index 100% rename from dbms/tests/queries/00013_a_select_from_table_with_arrays.sql rename to dbms/tests/queries/0_stateless/00013_a_select_from_table_with_arrays.sql diff --git a/dbms/tests/queries/00014_1_drop_if_exists_table_with_nested.reference b/dbms/tests/queries/0_stateless/00014_1_drop_if_exists_table_with_nested.reference similarity index 100% rename from dbms/tests/queries/00014_1_drop_if_exists_table_with_nested.reference rename to dbms/tests/queries/0_stateless/00014_1_drop_if_exists_table_with_nested.reference diff --git a/dbms/tests/queries/00014_1_drop_if_exists_table_with_nested.sql b/dbms/tests/queries/0_stateless/00014_1_drop_if_exists_table_with_nested.sql similarity index 100% rename from dbms/tests/queries/00014_1_drop_if_exists_table_with_nested.sql rename to dbms/tests/queries/0_stateless/00014_1_drop_if_exists_table_with_nested.sql diff --git a/dbms/tests/queries/00014_2_create_table_with_nested.reference b/dbms/tests/queries/0_stateless/00014_2_create_table_with_nested.reference similarity index 100% rename from dbms/tests/queries/00014_2_create_table_with_nested.reference rename to dbms/tests/queries/0_stateless/00014_2_create_table_with_nested.reference diff --git a/dbms/tests/queries/00014_2_create_table_with_nested.sql b/dbms/tests/queries/0_stateless/00014_2_create_table_with_nested.sql similarity index 100% rename from dbms/tests/queries/00014_2_create_table_with_nested.sql rename to dbms/tests/queries/0_stateless/00014_2_create_table_with_nested.sql diff --git a/dbms/tests/queries/00014_3_insert_into_table_with_nested.reference b/dbms/tests/queries/0_stateless/00014_3_insert_into_table_with_nested.reference similarity index 100% rename from dbms/tests/queries/00014_3_insert_into_table_with_nested.reference rename to dbms/tests/queries/0_stateless/00014_3_insert_into_table_with_nested.reference diff --git a/dbms/tests/queries/00014_3_insert_into_table_with_nested.sql b/dbms/tests/queries/0_stateless/00014_3_insert_into_table_with_nested.sql similarity index 100% rename from dbms/tests/queries/00014_3_insert_into_table_with_nested.sql rename to dbms/tests/queries/0_stateless/00014_3_insert_into_table_with_nested.sql diff --git a/dbms/tests/queries/00014_4_select_from_table_with_nested.reference b/dbms/tests/queries/0_stateless/00014_4_select_from_table_with_nested.reference similarity index 100% rename from dbms/tests/queries/00014_4_select_from_table_with_nested.reference rename to dbms/tests/queries/0_stateless/00014_4_select_from_table_with_nested.reference diff --git a/dbms/tests/queries/00014_4_select_from_table_with_nested.sql b/dbms/tests/queries/0_stateless/00014_4_select_from_table_with_nested.sql similarity index 100% rename from dbms/tests/queries/00014_4_select_from_table_with_nested.sql rename to dbms/tests/queries/0_stateless/00014_4_select_from_table_with_nested.sql diff --git a/dbms/tests/queries/00014_5_select_from_table_with_nested.reference b/dbms/tests/queries/0_stateless/00014_5_select_from_table_with_nested.reference similarity index 100% rename from dbms/tests/queries/00014_5_select_from_table_with_nested.reference rename to dbms/tests/queries/0_stateless/00014_5_select_from_table_with_nested.reference diff --git a/dbms/tests/queries/00014_5_select_from_table_with_nested.sql b/dbms/tests/queries/0_stateless/00014_5_select_from_table_with_nested.sql similarity index 100% rename from dbms/tests/queries/00014_5_select_from_table_with_nested.sql rename to dbms/tests/queries/0_stateless/00014_5_select_from_table_with_nested.sql diff --git a/dbms/tests/queries/00014_6_select_from_table_with_nested.reference b/dbms/tests/queries/0_stateless/00014_6_select_from_table_with_nested.reference similarity index 100% rename from dbms/tests/queries/00014_6_select_from_table_with_nested.reference rename to dbms/tests/queries/0_stateless/00014_6_select_from_table_with_nested.reference diff --git a/dbms/tests/queries/00014_6_select_from_table_with_nested.sql b/dbms/tests/queries/0_stateless/00014_6_select_from_table_with_nested.sql similarity index 100% rename from dbms/tests/queries/00014_6_select_from_table_with_nested.sql rename to dbms/tests/queries/0_stateless/00014_6_select_from_table_with_nested.sql diff --git a/dbms/tests/queries/00014_7_select_from_table_with_nested.reference b/dbms/tests/queries/0_stateless/00014_7_select_from_table_with_nested.reference similarity index 100% rename from dbms/tests/queries/00014_7_select_from_table_with_nested.reference rename to dbms/tests/queries/0_stateless/00014_7_select_from_table_with_nested.reference diff --git a/dbms/tests/queries/00014_7_select_from_table_with_nested.sql b/dbms/tests/queries/0_stateless/00014_7_select_from_table_with_nested.sql similarity index 100% rename from dbms/tests/queries/00014_7_select_from_table_with_nested.sql rename to dbms/tests/queries/0_stateless/00014_7_select_from_table_with_nested.sql diff --git a/dbms/tests/queries/00014_9_select_from_table_with_nested.reference b/dbms/tests/queries/0_stateless/00014_9_select_from_table_with_nested.reference similarity index 100% rename from dbms/tests/queries/00014_9_select_from_table_with_nested.reference rename to dbms/tests/queries/0_stateless/00014_9_select_from_table_with_nested.reference diff --git a/dbms/tests/queries/00014_9_select_from_table_with_nested.sql b/dbms/tests/queries/0_stateless/00014_9_select_from_table_with_nested.sql similarity index 100% rename from dbms/tests/queries/00014_9_select_from_table_with_nested.sql rename to dbms/tests/queries/0_stateless/00014_9_select_from_table_with_nested.sql diff --git a/dbms/tests/queries/00014_a_select_from_table_with_nested.reference b/dbms/tests/queries/0_stateless/00014_a_select_from_table_with_nested.reference similarity index 100% rename from dbms/tests/queries/00014_a_select_from_table_with_nested.reference rename to dbms/tests/queries/0_stateless/00014_a_select_from_table_with_nested.reference diff --git a/dbms/tests/queries/00014_a_select_from_table_with_nested.sql b/dbms/tests/queries/0_stateless/00014_a_select_from_table_with_nested.sql similarity index 100% rename from dbms/tests/queries/00014_a_select_from_table_with_nested.sql rename to dbms/tests/queries/0_stateless/00014_a_select_from_table_with_nested.sql diff --git a/dbms/tests/queries/00014_b_select_from_table_with_nested.reference b/dbms/tests/queries/0_stateless/00014_b_select_from_table_with_nested.reference similarity index 100% rename from dbms/tests/queries/00014_b_select_from_table_with_nested.reference rename to dbms/tests/queries/0_stateless/00014_b_select_from_table_with_nested.reference diff --git a/dbms/tests/queries/00014_b_select_from_table_with_nested.sql b/dbms/tests/queries/0_stateless/00014_b_select_from_table_with_nested.sql similarity index 100% rename from dbms/tests/queries/00014_b_select_from_table_with_nested.sql rename to dbms/tests/queries/0_stateless/00014_b_select_from_table_with_nested.sql diff --git a/dbms/tests/queries/00014_c_select_from_table_with_nested.reference b/dbms/tests/queries/0_stateless/00014_c_select_from_table_with_nested.reference similarity index 100% rename from dbms/tests/queries/00014_c_select_from_table_with_nested.reference rename to dbms/tests/queries/0_stateless/00014_c_select_from_table_with_nested.reference diff --git a/dbms/tests/queries/00014_c_select_from_table_with_nested.sql b/dbms/tests/queries/0_stateless/00014_c_select_from_table_with_nested.sql similarity index 100% rename from dbms/tests/queries/00014_c_select_from_table_with_nested.sql rename to dbms/tests/queries/0_stateless/00014_c_select_from_table_with_nested.sql diff --git a/dbms/tests/queries/00015_totals_having_constants.reference b/dbms/tests/queries/0_stateless/00015_totals_having_constants.reference similarity index 100% rename from dbms/tests/queries/00015_totals_having_constants.reference rename to dbms/tests/queries/0_stateless/00015_totals_having_constants.reference diff --git a/dbms/tests/queries/00015_totals_having_constants.sql b/dbms/tests/queries/0_stateless/00015_totals_having_constants.sql similarity index 100% rename from dbms/tests/queries/00015_totals_having_constants.sql rename to dbms/tests/queries/0_stateless/00015_totals_having_constants.sql diff --git a/dbms/tests/queries/00016_totals_having_constants.reference b/dbms/tests/queries/0_stateless/00016_totals_having_constants.reference similarity index 100% rename from dbms/tests/queries/00016_totals_having_constants.reference rename to dbms/tests/queries/0_stateless/00016_totals_having_constants.reference diff --git a/dbms/tests/queries/00016_totals_having_constants.sql b/dbms/tests/queries/0_stateless/00016_totals_having_constants.sql similarity index 100% rename from dbms/tests/queries/00016_totals_having_constants.sql rename to dbms/tests/queries/0_stateless/00016_totals_having_constants.sql diff --git a/dbms/tests/queries/00017_in_subquery_with_empty_result.reference b/dbms/tests/queries/0_stateless/00017_in_subquery_with_empty_result.reference similarity index 100% rename from dbms/tests/queries/00017_in_subquery_with_empty_result.reference rename to dbms/tests/queries/0_stateless/00017_in_subquery_with_empty_result.reference diff --git a/dbms/tests/queries/00017_in_subquery_with_empty_result.sql b/dbms/tests/queries/0_stateless/00017_in_subquery_with_empty_result.sql similarity index 100% rename from dbms/tests/queries/00017_in_subquery_with_empty_result.sql rename to dbms/tests/queries/0_stateless/00017_in_subquery_with_empty_result.sql diff --git a/dbms/tests/queries/00018_distinct_in_subquery.reference b/dbms/tests/queries/0_stateless/00018_distinct_in_subquery.reference similarity index 100% rename from dbms/tests/queries/00018_distinct_in_subquery.reference rename to dbms/tests/queries/0_stateless/00018_distinct_in_subquery.reference diff --git a/dbms/tests/queries/00018_distinct_in_subquery.sql b/dbms/tests/queries/0_stateless/00018_distinct_in_subquery.sql similarity index 100% rename from dbms/tests/queries/00018_distinct_in_subquery.sql rename to dbms/tests/queries/0_stateless/00018_distinct_in_subquery.sql From 9abadce603210bc2295668e8bc1dff44a4c6f6e0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Mar 2014 06:50:39 +0400 Subject: [PATCH 14/56] dbms: updated tests [#METR-2944]. --- dbms/tests/clickhouse-test | 111 +++++++++++++++++++++---------------- 1 file changed, 62 insertions(+), 49 deletions(-) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 4c918cf30a7..3300a944d0f 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -21,58 +21,71 @@ MSG_GENERATED="${COLOR_WHITE}[ ${COLOR_UNKNOWN}GENERATED${COLOR_WHITE} ]${COLOR_ ERRORS=0 - -for query_file in $(ls $QUERIES_DIR/*.sql) +for dir in $(ls $QUERIES_DIR) do - test_name=$(basename $query_file .sql) + tests_name=$(echo $dir | sed -E 's/^[0-9_]+//') - result_file=$QUERIES_DIR/$test_name.result - error_file=$QUERIES_DIR/$test_name.error - reference_file=$QUERIES_DIR/$test_name.reference - diff_file=$QUERIES_DIR/$test_name.diff + echo + echo "Running $tests_name tests." + echo - printf "%-60s" "$test_name: " - - $CLIENT_PROGRAM < $query_file > $result_file 2> $error_file - ret_code=$? - - if [ $ret_code -ne 0 ]; then - ERRORS=$(($ERRORS + 1)) - echo -e "$MSG_FAIL - return code $ret_code" - if [ -s "$error_file" ]; then - cat $error_file - fi - # разорвано соединение с сервером - if grep -q -E "Connection refused|Attempt to read after eof" $error_file; then - exit 1; - fi - elif [ -s "$error_file" ]; then - ERRORS=$(($ERRORS + 1)) - echo -e "$MSG_FAIL - having stderror:" - cat $error_file - elif grep -q "Exception" $result_file; then - ERRORS=$(($ERRORS + 1)) - echo -e "$MSG_FAIL - having exception:" - cat $result_file - elif [ ! -e "$reference_file" ]; then - # надо сгенерировать эталонный результат - if [ "$1" == "--generate" ]; then - cp $result_file $reference_file - echo -e "$MSG_GENERATED - no reference file" - else - echo -e "$MSG_UNKNOWN - no reference file (use --generate to create)" - fi - else - diff $reference_file $result_file > $diff_file - if [ -s "$diff_file" ]; then - ERRORS=$(($ERRORS + 1)) - echo -e "$MSG_FAIL - result differs with reference:" - cat $diff_file - else - echo -e "$MSG_OK" - rm $error_file $result_file $diff_file - fi + if [[ "$tests_name" =~ "stateful" && 0 -eq $(echo "EXISTS TABLE test.hits" | $CLIENT_PROGRAM) ]]; then + echo "Won't run stateful tests because test data wasn't loaded. See README.txt." + continue fi + + for query_file in $(ls $QUERIES_DIR/$dir/*.sql) + do + test_name=$(basename $query_file .sql) + + result_file=$QUERIES_DIR/$dir/$test_name.result + error_file=$QUERIES_DIR/$dir/$test_name.error + reference_file=$QUERIES_DIR/$dir/$test_name.reference + diff_file=$QUERIES_DIR/$dir/$test_name.diff + + printf "%-60s" "$test_name: " + + $CLIENT_PROGRAM < $query_file > $result_file 2> $error_file + ret_code=$? + + if [ $ret_code -ne 0 ]; then + ERRORS=$(($ERRORS + 1)) + echo -e "$MSG_FAIL - return code $ret_code" + if [ -s "$error_file" ]; then + cat $error_file + fi + # разорвано соединение с сервером + if grep -q -E "Connection refused|Attempt to read after eof" $error_file; then + exit 1; + fi + elif [ -s "$error_file" ]; then + ERRORS=$(($ERRORS + 1)) + echo -e "$MSG_FAIL - having stderror:" + cat $error_file + elif grep -q "Exception" $result_file; then + ERRORS=$(($ERRORS + 1)) + echo -e "$MSG_FAIL - having exception:" + cat $result_file + elif [ ! -e "$reference_file" ]; then + # надо сгенерировать эталонный результат + if [ "$1" == "--generate" ]; then + cp $result_file $reference_file + echo -e "$MSG_GENERATED - no reference file" + else + echo -e "$MSG_UNKNOWN - no reference file (use --generate to create)" + fi + else + diff $reference_file $result_file > $diff_file + if [ -s "$diff_file" ]; then + ERRORS=$(($ERRORS + 1)) + echo -e "$MSG_FAIL - result differs with reference:" + cat $diff_file + else + echo -e "$MSG_OK" + rm $error_file $result_file $diff_file + fi + fi + done done @@ -82,6 +95,6 @@ if [ $ERRORS -gt 0 ]; then echo -e "${COLOR_FAIL}Having $ERRORS errors!${COLOR_RESET}" exit 1 else - echo -e "${COLOR_OK}All tests succeeded.${COLOR_RESET}" + echo -e "${COLOR_OK}All tests passed.${COLOR_RESET}" exit 0 fi From 258b14b506bfbf3fd124154b446a24f79b30f7d0 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 24 Mar 2014 17:24:47 +0400 Subject: [PATCH 15/56] Fixed segfault with table functions. [#METR-10549] --- dbms/include/DB/Storages/IStorage.h | 18 ++++++++++++------ .../DB/TableFunctions/TableFunctionMerge.h | 2 +- .../DB/TableFunctions/TableFunctionRemote.h | 2 +- 3 files changed, 14 insertions(+), 8 deletions(-) diff --git a/dbms/include/DB/Storages/IStorage.h b/dbms/include/DB/Storages/IStorage.h index 90c068aa008..01d97dde8ad 100644 --- a/dbms/include/DB/Storages/IStorage.h +++ b/dbms/include/DB/Storages/IStorage.h @@ -27,6 +27,11 @@ typedef SharedPtr BlockInputStreamPtr; typedef std::vector BlockInputStreams; +class IStorage; + +typedef std::shared_ptr StoragePtr; + + /** Хранилище. Отвечает за: * - хранение данных таблицы; * - определение, в каком файле (или не файле) хранятся данные; @@ -59,7 +64,7 @@ public: /** Не дает изменять описание таблицы (в том числе переименовывать и удалять таблицу). * Если в течение какой-то операции структура таблицы должна оставаться неизменной, нужно держать такой лок на все ее время. * Например, нужно держать такой лок на время всего запроса SELECT или INSERT и на все время слияния набора кусков - * (но между выбором кусков для слияния и их слиянием структура таблицы может измениться). + * (но между выбором кусков для слияния и их слиянием структура таблицы может измениться). * NOTE: Это лок на "чтение" описания таблицы. Чтобы изменить описание таблицы, нужно взять TableStructureWriteLock. */ class TableStructureReadLock @@ -67,13 +72,15 @@ public: private: friend class IStorage; + StoragePtr storage; /// Порядок важен. Poco::SharedPtr data_lock; Poco::SharedPtr structure_lock; - TableStructureReadLock(IStorage & storage, bool lock_structure, bool lock_data) - : data_lock(lock_data ? new Poco::ScopedReadRWLock(storage. data_lock) : nullptr), - structure_lock(lock_structure ? new Poco::ScopedReadRWLock(storage.structure_lock) : nullptr) {} + TableStructureReadLock(StoragePtr storage_, bool lock_structure, bool lock_data) + : storage(storage_), + data_lock(lock_data ? new Poco::ScopedReadRWLock(storage-> data_lock) : nullptr), + structure_lock(lock_structure ? new Poco::ScopedReadRWLock(storage->structure_lock) : nullptr) {} }; typedef Poco::SharedPtr TableStructureReadLockPtr; @@ -88,7 +95,7 @@ public: */ TableStructureReadLockPtr lockStructure(bool will_modify_data) { - TableStructureReadLockPtr res = new TableStructureReadLock(*this, true, will_modify_data); + TableStructureReadLockPtr res = new TableStructureReadLock(thisPtr(), true, will_modify_data); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); return res; @@ -277,7 +284,6 @@ private: mutable Poco::RWLock structure_lock; }; -typedef std::shared_ptr StoragePtr; typedef std::vector StorageVector; typedef IStorage::TableStructureReadLocks TableLocks; diff --git a/dbms/include/DB/TableFunctions/TableFunctionMerge.h b/dbms/include/DB/TableFunctions/TableFunctionMerge.h index b56ce106fbc..f08be8eef42 100644 --- a/dbms/include/DB/TableFunctions/TableFunctionMerge.h +++ b/dbms/include/DB/TableFunctions/TableFunctionMerge.h @@ -44,7 +44,7 @@ public: String table_name_regexp = safeGet(dynamic_cast(*args[1]).value); /// В InterpreterSelectQuery будет создан ExpressionAnalzyer, который при обработке запроса наткнется на этот Identifier. - /// Нам необходимо его пометить как имя базы данных, посколку по умолчанию стоит значение column + /// Нам необходимо его пометить как имя базы данных, поскольку по умолчанию стоит значение column dynamic_cast(*args[0]).kind = ASTIdentifier::Database; return StorageMerge::create(getName(), chooseColumns(source_database, table_name_regexp, context), source_database, table_name_regexp, context); diff --git a/dbms/include/DB/TableFunctions/TableFunctionRemote.h b/dbms/include/DB/TableFunctions/TableFunctionRemote.h index b5edb9a8a21..0239c4800a2 100644 --- a/dbms/include/DB/TableFunctions/TableFunctionRemote.h +++ b/dbms/include/DB/TableFunctions/TableFunctionRemote.h @@ -50,7 +50,7 @@ public: String password = args.size() == 5 ? safeGet(dynamic_cast(*args[4]).value) : ""; /// В InterpreterSelectQuery будет создан ExpressionAnalzyer, который при обработке запроса наткнется на эти Identifier. - /// Нам необходимо их пометить как имя базы данных и таблицы посколку по умолчанию стоит значение column + /// Нам необходимо их пометить как имя базы данных и таблицы поскольку по умолчанию стоит значение column dynamic_cast(*args[1]).kind = ASTIdentifier::Database; dynamic_cast(*args[2]).kind = ASTIdentifier::Table; From 4c92b5f2208e2d946af69efebd9b121b9c598e03 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 24 Mar 2014 20:10:14 +0400 Subject: [PATCH 16/56] Fixed segfault when some of the nested columns are missing. [#METR-10564] --- .../DB/Storages/MergeTree/MergeTreeReader.h | 58 +++++++++++++------ 1 file changed, 41 insertions(+), 17 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index f6ee8257d0b..3f0f7173e4a 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -37,6 +37,8 @@ typedef std::vector MarkRanges; */ class MergeTreeReader { + typedef std::map OffsetColumns; + public: MergeTreeReader(const String & path_, /// Путь к куску const Names & columns_names_, bool use_uncompressed_cache_, MergeTreeData & storage_) @@ -60,7 +62,6 @@ public: /// Указатели на столбцы смещений, общие для столбцов из вложенных структур данных /// Если append, все значения NULL, и offset_columns используется только для проверки, что столбец смещений уже прочитан. - typedef std::map OffsetColumns; OffsetColumns offset_columns; for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it) @@ -124,6 +125,24 @@ public: { try { + /** Для недостающих столбцов из вложенной структуры нужно создавать не столбец пустых массивов, а столбец массивов + * правильных длин. + * TODO: Если для какой-то вложенной структуры были запрошены только отсутствующие столбцы, для них вернутся пустые + * массивы, даже если в куске есть смещения для этой вложенной структуры. Это можно исправить. + */ + + /// Сначала запомним столбцы смещений для всех массивов в блоке. + OffsetColumns offset_columns; + for (size_t i = 0; i < res.columns(); ++i) + { + const ColumnWithNameAndType & column = res.getByPosition(i); + if (const ColumnArray * array = dynamic_cast(&*column.column)) + { + String offsets_name = DataTypeNested::extractNestedTableName(column.name); + offset_columns[offsets_name] = array->getOffsetsColumn(); + } + } + size_t pos = 0; /// Позиция, куда надо вставить недостающий столбец. for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it, ++pos) { @@ -133,11 +152,27 @@ public: column.name = *it; column.type = storage.getDataTypeByName(*it); - /** Нужно превратить константный столбец в полноценный, так как в части блоков (из других кусков), - * он может быть полноценным (а то интерпретатор может посчитать, что он константный везде). - */ - column.column = dynamic_cast(*column.type->createConstColumn( - res.rows(), column.type->getDefault())).convertToFullColumn(); + String offsets_name = DataTypeNested::extractNestedTableName(column.name); + if (offset_columns.count(offsets_name)) + { + ColumnPtr offsets_column = offset_columns[offsets_name]; + DataTypePtr nested_type = dynamic_cast(*column.type).getNestedType(); + size_t nested_rows = offsets_column->empty() ? 0 + : dynamic_cast(*offsets_column).getData().back(); + + ColumnPtr nested_column = dynamic_cast(*nested_type->createConstColumn( + nested_rows, nested_type->getDefault())).convertToFullColumn(); + + column.column = new ColumnArray(nested_column, offsets_column); + } + else + { + /** Нужно превратить константный столбец в полноценный, так как в части блоков (из других кусков), + * он может быть полноценным (а то интерпретатор может посчитать, что он константный везде). + */ + column.column = dynamic_cast(*column.type->createConstColumn( + res.rows(), column.type->getDefault())).convertToFullColumn(); + } res.insert(pos, column); } @@ -266,17 +301,6 @@ private: addStream(name, *type_arr->getNestedType(), level + 1); } - else if (const DataTypeNested * type_nested = dynamic_cast(&type)) - { - String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level); - String escaped_size_name = escaped_column_name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level); - - streams[size_name] = new Stream(path + escaped_size_name, uncompressed_cache, mark_cache); - - const NamesAndTypesList & columns = *type_nested->getNestedTypesList(); - for (NamesAndTypesList::const_iterator it = columns.begin(); it != columns.end(); ++it) - addStream(DataTypeNested::concatenateNestedName(name, it->first), *it->second, level + 1); - } else streams[name] = new Stream(path + escaped_column_name, uncompressed_cache, mark_cache); } From 2bc8beff045c3e74e732b9981a745b7ad221898a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Mar 2014 23:32:31 +0400 Subject: [PATCH 17/56] Added tests [#METR-10566]. --- .../queries/0_stateless/00019_quantiles_totals_distributed.sql | 1 + 1 file changed, 1 insertion(+) create mode 100644 dbms/tests/queries/0_stateless/00019_quantiles_totals_distributed.sql diff --git a/dbms/tests/queries/0_stateless/00019_quantiles_totals_distributed.sql b/dbms/tests/queries/0_stateless/00019_quantiles_totals_distributed.sql new file mode 100644 index 00000000000..c075aeecc08 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00019_quantiles_totals_distributed.sql @@ -0,0 +1 @@ +SELECT quantilesTiming(0.1, 0.5, 0.9)(materialize(dummy)) FROM remote('127.0.0.{1,2}', system, one) GROUP BY 1 WITH TOTALS From 7cde6bb6a6617663dc98dc766227f795d1cf8598 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Mar 2014 04:28:25 +0400 Subject: [PATCH 18/56] dbms: fixed error with hash tables [#METR-10571]. --- dbms/include/DB/Interpreters/ClearableHashMap.h | 14 +++++++++++++- dbms/include/DB/Interpreters/HashMap.h | 14 +++++++++++++- dbms/include/DB/Interpreters/HashSet.h | 14 +++++++++++++- 3 files changed, 39 insertions(+), 3 deletions(-) diff --git a/dbms/include/DB/Interpreters/ClearableHashMap.h b/dbms/include/DB/Interpreters/ClearableHashMap.h index c3279ed5a5a..963511a7cc8 100644 --- a/dbms/include/DB/Interpreters/ClearableHashMap.h +++ b/dbms/include/DB/Interpreters/ClearableHashMap.h @@ -86,10 +86,22 @@ private: * Элемент может остаться на месте, или переместиться в новое место "справа", * или переместиться левее по цепочке разрешения коллизий, из-за того, что элементы левее него были перемещены в новое место "справа". */ - for (size_t i = 0; i < old_size; ++i) + size_t i = 0; + for (; i < old_size; ++i) if (buf[i].version == version) reinsert(buf[i]); + /** Также имеется особый случай: + * если элемент должен был быть в конце старого буфера, [ x] + * но находится в начале из-за цепочки разрешения коллизий, [o x] + * то после ресайза, он сначала снова окажется не на своём месте, [ xo ] + * и для того, чтобы перенести его куда надо, + * надо будет после переноса всех элементов из старой половинки [ o x ] + * обработать ещё хвостик из цепочки разрешения коллизий сразу после неё [ o x ] + */ + for (; buf[i].version == version; ++i) + reinsert(buf[i]); + #ifdef DBMS_HASH_MAP_DEBUG_RESIZES watch.stop(); std::cerr << std::fixed << std::setprecision(3) diff --git a/dbms/include/DB/Interpreters/HashMap.h b/dbms/include/DB/Interpreters/HashMap.h index fe0d6db5b5c..fa04d48f435 100644 --- a/dbms/include/DB/Interpreters/HashMap.h +++ b/dbms/include/DB/Interpreters/HashMap.h @@ -181,10 +181,22 @@ private: * Элемент может остаться на месте, или переместиться в новое место "справа", * или переместиться левее по цепочке разрешения коллизий, из-за того, что элементы левее него были перемещены в новое место "справа". */ - for (size_t i = 0; i < old_size; ++i) + size_t i = 0; + for (; i < old_size; ++i) if (!ZeroTraits::check(buf[i].first)) reinsert(buf[i]); + /** Также имеется особый случай: + * если элемент должен был быть в конце старого буфера, [ x] + * но находится в начале из-за цепочки разрешения коллизий, [o x] + * то после ресайза, он сначала снова окажется не на своём месте, [ xo ] + * и для того, чтобы перенести его куда надо, + * надо будет после переноса всех элементов из старой половинки [ o x ] + * обработать ещё хвостик из цепочки разрешения коллизий сразу после неё [ o x ] + */ + for (; !ZeroTraits::check(buf[i].first); ++i) + reinsert(buf[i]); + #ifdef DBMS_HASH_MAP_DEBUG_RESIZES watch.stop(); std::cerr << std::fixed << std::setprecision(3) diff --git a/dbms/include/DB/Interpreters/HashSet.h b/dbms/include/DB/Interpreters/HashSet.h index 67f4c52d494..7856eff6072 100644 --- a/dbms/include/DB/Interpreters/HashSet.h +++ b/dbms/include/DB/Interpreters/HashSet.h @@ -87,10 +87,22 @@ private: * Элемент может остаться на месте, или переместиться в новое место "справа", * или переместиться левее по цепочке разрешения коллизий, из-за того, что элементы левее него были перемещены в новое место "справа". */ - for (size_t i = 0; i < old_size; ++i) + size_t i = 0; + for (; i < old_size; ++i) if (!ZeroTraits::check(buf[i])) reinsert(buf[i]); + /** Также имеется особый случай: + * если элемент должен был быть в конце старого буфера, [ x] + * но находится в начале из-за цепочки разрешения коллизий, [o x] + * то после ресайза, он сначала снова окажется не на своём месте, [ xo ] + * и для того, чтобы перенести его куда надо, + * надо будет после переноса всех элементов из старой половинки [ o x ] + * обработать ещё хвостик из цепочки разрешения коллизий сразу после неё [ o x ] + */ + for (; !ZeroTraits::check(buf[i]); ++i) + reinsert(buf[i]); + #ifdef DBMS_HASH_MAP_DEBUG_RESIZES watch.stop(); std::cerr << std::fixed << std::setprecision(3) From 262ffcd74b85bccdc13bd18c340fc41ed81e99ca Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 25 Mar 2014 22:16:26 +0400 Subject: [PATCH 19/56] Fixed parametric aggregate functions in totals. [#METR-10566] --- .../AggregateFunctions/AggregateFunctionIf.h | 2 +- .../AggregateFunctionQuantile.h | 4 +- .../AggregateFunctionQuantileTiming.h | 4 +- .../AggregateFunctions/IAggregateFunction.h | 5 ++- dbms/include/DB/Common/AutoArray.h | 20 +++++----- .../DB/DataTypes/DataTypeAggregateFunction.h | 22 +++++++--- dbms/include/DB/Interpreters/Aggregator.h | 1 + .../CollapsingSortedBlockInputStream.cpp | 2 +- .../SummingSortedBlockInputStream.cpp | 2 +- dbms/src/DataTypes/DataTypeFactory.cpp | 40 +++++++++++++++---- dbms/src/Interpreters/Aggregator.cpp | 2 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 3 +- .../Interpreters/InterpreterSelectQuery.cpp | 2 +- 13 files changed, 75 insertions(+), 34 deletions(-) diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionIf.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionIf.h index 296331f56bb..c9936106426 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionIf.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionIf.h @@ -47,7 +47,7 @@ public: nested_func->setArguments(nested_arguments); } - void setParameters(const Row & params) + void setParameters(const Array & params) { nested_func->setParameters(params); } diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantile.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantile.h index f9eb53d4afa..1c12f2d48bb 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantile.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantile.h @@ -56,7 +56,7 @@ public: type = argument; } - void setParameters(const Row & params) + void setParameters(const Array & params) { if (params.size() != 1) throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -130,7 +130,7 @@ public: type = argument; } - void setParameters(const Row & params) + void setParameters(const Array & params) { if (params.empty()) throw Exception("Aggregate function " + getName() + " requires at least one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h index 51107b85f83..06ae5222c41 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h @@ -511,7 +511,7 @@ public: { } - void setParameters(const Row & params) + void setParameters(const Array & params) { if (params.size() != 1) throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -570,7 +570,7 @@ public: { } - void setParameters(const Row & params) + void setParameters(const Array & params) { if (params.empty()) throw Exception("Aggregate function " + getName() + " requires at least one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); diff --git a/dbms/include/DB/AggregateFunctions/IAggregateFunction.h b/dbms/include/DB/AggregateFunctions/IAggregateFunction.h index d0c3653ad83..b6063508d7a 100644 --- a/dbms/include/DB/AggregateFunctions/IAggregateFunction.h +++ b/dbms/include/DB/AggregateFunctions/IAggregateFunction.h @@ -39,9 +39,10 @@ public: * Если параметры не предусмотрены или переданные параметры недопустимы - кинуть исключение. * Если параметры есть - необходимо вызывать перед остальными вызовами, иначе - не вызывать. */ - virtual void setParameters(const Row & params) + virtual void setParameters(const Array & params) { - throw Exception("Aggregate function " + getName() + " doesn't allow parameters.", ErrorCodes::AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS); + throw Exception("Aggregate function " + getName() + " doesn't allow parameters.", + ErrorCodes::AGGREGATE_FUNCTION_DOESNT_ALLOW_PARAMETERS); } /// Получить тип результата. diff --git a/dbms/include/DB/Common/AutoArray.h b/dbms/include/DB/Common/AutoArray.h index 43f31ad2309..454e430efdd 100644 --- a/dbms/include/DB/Common/AutoArray.h +++ b/dbms/include/DB/Common/AutoArray.h @@ -20,7 +20,7 @@ namespace DB * sizeof равен размеру одного указателя. * * Не exception-safe. - * Копирование и присваивание разрушающее: исходный объект становится пустым. + * Копирование не поддерживается. Перемещение опустошает исходный объект. * То есть, использовать этот массив во многих случаях неудобно. * * Предназначен для ситуаций, в которых создаётся много массивов одинакового небольшого размера, @@ -82,24 +82,24 @@ public: init(size_, dont_init_elems); } - /** Разрушающее копирование. + /** Премещение. */ - AutoArray(const AutoArray & src) + AutoArray(AutoArray && src) { - //std::cerr << this << " AutoArray(const AutoArray & src)" << std::endl; - + if (this == &src) + return; setEmpty(); data = src.data; - const_cast &>(src).setEmpty(); + src.setEmpty(); } - AutoArray & operator= (const AutoArray & src) + AutoArray & operator= (AutoArray && src) { - //std::cerr << this << " operator=(const AutoArray & src)" << std::endl; - + if (this == &src) + return; uninit(); data = src.data; - const_cast &>(src).setEmpty(); + src.setEmpty(); return *this; } diff --git a/dbms/include/DB/DataTypes/DataTypeAggregateFunction.h b/dbms/include/DB/DataTypes/DataTypeAggregateFunction.h index bc7ffcf8d28..e3d87a4c65c 100644 --- a/dbms/include/DB/DataTypes/DataTypeAggregateFunction.h +++ b/dbms/include/DB/DataTypes/DataTypeAggregateFunction.h @@ -11,19 +11,19 @@ namespace DB using Poco::SharedPtr; /** Тип - состояние агрегатной функции. - * Параметры типа - это агрегатная функция и типы её аргументов. + * Параметры типа - это агрегатная функция, типы её аргументов и её параметры (для параметрических агрегатных функций). */ class DataTypeAggregateFunction : public IDataType { private: AggregateFunctionPtr function; DataTypes argument_types; + Array parameters; public: - DataTypeAggregateFunction(const AggregateFunctionPtr & function_, const DataTypes & argument_types_) - : function(function_), argument_types(argument_types_) + DataTypeAggregateFunction(const AggregateFunctionPtr & function_, const DataTypes & argument_types_, const Array & parameters_) + : function(function_), argument_types(argument_types_), parameters(parameters_) { - function->setArguments(argument_types); } std::string getName() const @@ -31,6 +31,18 @@ public: std::stringstream stream; stream << "AggregateFunction(" << function->getName(); + if (!parameters.empty()) + { + stream << "("; + for (size_t i = 0; i < parameters.size(); ++i) + { + if (i) + stream << ", "; + stream << apply_visitor(DB::FieldVisitorToString(), parameters[i]); + } + stream << ")"; + } + for (DataTypes::const_iterator it = argument_types.begin(); it != argument_types.end(); ++it) stream << ", " << (*it)->getName(); @@ -38,7 +50,7 @@ public: return stream.str(); } - DataTypePtr clone() const { return new DataTypeAggregateFunction(function, argument_types); } + DataTypePtr clone() const { return new DataTypeAggregateFunction(function, argument_types, parameters); } void serializeBinary(const Field & field, WriteBuffer & ostr) const; void deserializeBinary(Field & field, ReadBuffer & istr) const; diff --git a/dbms/include/DB/Interpreters/Aggregator.h b/dbms/include/DB/Interpreters/Aggregator.h index 3cd9464fbab..b324bfbbbb3 100644 --- a/dbms/include/DB/Interpreters/Aggregator.h +++ b/dbms/include/DB/Interpreters/Aggregator.h @@ -27,6 +27,7 @@ namespace DB struct AggregateDescription { AggregateFunctionPtr function; + Array parameters; /// Параметры (параметрической) агрегатной функции. ColumnNumbers arguments; Names argument_names; /// Используются, если arguments не заданы. String column_name; /// Какое имя использовать для столбца со значениями агрегатной функции diff --git a/dbms/src/DataStreams/CollapsingSortedBlockInputStream.cpp b/dbms/src/DataStreams/CollapsingSortedBlockInputStream.cpp index 028da3692e4..2fbdeddf913 100644 --- a/dbms/src/DataStreams/CollapsingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/CollapsingSortedBlockInputStream.cpp @@ -129,7 +129,7 @@ void CollapsingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPt /// Запишем данные для предыдущего визита. insertRows(merged_columns, merged_rows); - current_key = next_key; + current_key = std::move(next_key); next_key.resize(description.size()); count_negative = 0; diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp index c49f05349f4..48b100049e0 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp @@ -90,7 +90,7 @@ void SummingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPtrs insertCurrentRow(merged_columns); } - current_key = next_key; + current_key = std::move(next_key); next_key.resize(description.size()); setRow(current_row, current); diff --git a/dbms/src/DataTypes/DataTypeFactory.cpp b/dbms/src/DataTypes/DataTypeFactory.cpp index b5b812426f5..6fe29be3efe 100644 --- a/dbms/src/DataTypes/DataTypeFactory.cpp +++ b/dbms/src/DataTypes/DataTypeFactory.cpp @@ -21,14 +21,15 @@ #include #include #include +#include namespace DB { DataTypeFactory::DataTypeFactory() - : fixed_string_regexp("^FixedString\\s*\\(\\s*(\\d+)\\s*\\)$"), - nested_regexp("^(\\w+)\\s*\\(\\s*(.+)\\s*\\)$", Poco::RegularExpression::RE_MULTILINE | Poco::RegularExpression::RE_DOTALL) + : fixed_string_regexp(R"--(^FixedString\s*\(\s*(\d+)\s*\)$)--"), + nested_regexp(R"--(^(\w+)\s*\(\s*(.+)\s*\)$)--", Poco::RegularExpression::RE_MULTILINE | Poco::RegularExpression::RE_DOTALL) { boost::assign::insert(non_parametric_data_types) ("UInt8", new DataTypeUInt8) @@ -71,6 +72,7 @@ DataTypePtr DataTypeFactory::get(const String & name) const String function_name; AggregateFunctionPtr function; DataTypes argument_types; + Array params_row; ParserExpressionList args_parser; ASTPtr args_ast; @@ -87,14 +89,38 @@ DataTypePtr DataTypeFactory::get(const String & name) const throw Exception("Data type AggregateFunction requires parameters: " "name of aggregate function and list of data types for arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - ASTs::iterator it = args_list.children.begin(); - function_name = (*it)->getColumnName(); + if (ASTFunction * parametric = dynamic_cast(&*args_list.children[0])) + { + if (parametric->parameters) + throw Exception("Unexpected level of parameters to aggregate function", ErrorCodes::SYNTAX_ERROR); + function_name = parametric->name; - for (++it; it != args_list.children.end(); ++it) - argument_types.push_back(get((*it)->getColumnName())); + ASTs & parameters = dynamic_cast(*parametric->arguments).children; + params_row.resize(parameters.size()); + + for (size_t i = 0; i < parameters.size(); ++i) + { + ASTLiteral * lit = dynamic_cast(&*parameters[i]); + if (!lit) + throw Exception("Parameters to aggregate functions must be literals", + ErrorCodes::PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS); + + params_row[i] = lit->value; + } + } + else + { + function_name = args_list.children[0]->getColumnName(); + } + + for (size_t i = 1; i < args_list.children.size(); ++i) + argument_types.push_back(get(args_list.children[i]->getColumnName())); function = AggregateFunctionFactory().get(function_name, argument_types); - return new DataTypeAggregateFunction(function, argument_types); + if (!params_row.empty()) + function->setParameters(params_row); + function->setArguments(argument_types); + return new DataTypeAggregateFunction(function, argument_types, params_row); } if (base_name == "Nested") diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 71a7b14e060..df5c9f58e34 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -86,7 +86,7 @@ void Aggregator::initialize(Block & block) for (size_t j = 0; j < arguments_size; ++j) argument_types[j] = block.getByPosition(aggregates[i].arguments[j]).type; - col.type = new DataTypeAggregateFunction(aggregates[i].function, argument_types); + col.type = new DataTypeAggregateFunction(aggregates[i].function, argument_types, aggregates[i].parameters); col.column = new ColumnAggregateFunction(aggregates[i].function); sample.insert(col); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 4d06a422031..38aaeabd661 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -1004,7 +1004,7 @@ void ExpressionAnalyzer::getAggregatesImpl(ASTPtr ast, ExpressionActions & actio if (node->parameters) { ASTs & parameters = dynamic_cast(*node->parameters).children; - Row params_row(parameters.size()); + Array params_row(parameters.size()); for (size_t i = 0; i < parameters.size(); ++i) { @@ -1015,6 +1015,7 @@ void ExpressionAnalyzer::getAggregatesImpl(ASTPtr ast, ExpressionActions & actio params_row[i] = lit->value; } + aggregate.parameters = params_row; aggregate.function->setParameters(params_row); } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 518a42c2f39..009ca1ca9a3 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -293,7 +293,7 @@ BlockInputStreamPtr InterpreterSelectQuery::execute() settings.limits.max_rows_to_group_by && settings.limits.group_by_overflow_mode == OverflowMode::ANY && settings.totals_mode != TotalsMode::AFTER_HAVING_EXCLUSIVE; - /// Нужно ли после агрегации сразу финализироыать агрегатные функции. + /// Нужно ли после агрегации сразу финализировать агрегатные функции. bool aggregate_final = need_aggregate && to_stage > QueryProcessingStage::WithMergeableState && From 755fa64369701dc61da626ab0a41526843353897 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 25 Mar 2014 22:19:08 +0400 Subject: [PATCH 20/56] Fixed build. [#METR-10566] --- dbms/include/DB/Common/AutoArray.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/include/DB/Common/AutoArray.h b/dbms/include/DB/Common/AutoArray.h index 454e430efdd..350fdb4f50a 100644 --- a/dbms/include/DB/Common/AutoArray.h +++ b/dbms/include/DB/Common/AutoArray.h @@ -96,7 +96,7 @@ public: AutoArray & operator= (AutoArray && src) { if (this == &src) - return; + return *this; uninit(); data = src.data; src.setEmpty(); From 42be4d6ec5102b021bbc02e71e94997faa8485e5 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 25 Mar 2014 23:19:22 +0400 Subject: [PATCH 21/56] Added an option for clickhouse-test to generate reference for just one test. Added reference for one test query. [#METR-10566] --- dbms/tests/clickhouse-test | 4 ++-- .../0_stateless/00019_quantiles_totals_distributed.reference | 3 +++ 2 files changed, 5 insertions(+), 2 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00019_quantiles_totals_distributed.reference diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 3300a944d0f..6911441e468 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -68,11 +68,11 @@ do cat $result_file elif [ ! -e "$reference_file" ]; then # надо сгенерировать эталонный результат - if [ "$1" == "--generate" ]; then + if [ "$1" == "--generate" ] && ( [ "$2" == "" ] || [ "$2" == "$test_name" ] ); then cp $result_file $reference_file echo -e "$MSG_GENERATED - no reference file" else - echo -e "$MSG_UNKNOWN - no reference file (use --generate to create)" + echo -e "$MSG_UNKNOWN - no reference file (use --generate [test_name] to create)" fi else diff $reference_file $result_file > $diff_file diff --git a/dbms/tests/queries/0_stateless/00019_quantiles_totals_distributed.reference b/dbms/tests/queries/0_stateless/00019_quantiles_totals_distributed.reference new file mode 100644 index 00000000000..7575632704a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00019_quantiles_totals_distributed.reference @@ -0,0 +1,3 @@ +[0,0,0] + +[0,0,0] From 82f6b4a73d46f98411829374efc2c277fb06e781 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Mar 2014 02:53:48 +0400 Subject: [PATCH 22/56] dbms: removed old test script [#METR-2944]. --- dbms/tests/test_runner.sh | 91 --------------------------------------- 1 file changed, 91 deletions(-) delete mode 100755 dbms/tests/test_runner.sh diff --git a/dbms/tests/test_runner.sh b/dbms/tests/test_runner.sh deleted file mode 100755 index 0418251a98d..00000000000 --- a/dbms/tests/test_runner.sh +++ /dev/null @@ -1,91 +0,0 @@ -#!/bin/bash -# script to run query to databases - -function usage() -{ - cat < $log - queries=("${@}") - queries_count=${#queries[@]} - - index=0 - while [ "$index" -lt "$queries_count" ]; do - query=${queries[$index]} - - if [[ $query == "" ]]; then - let "index = $index + 1" - continue - fi - - comment_re='--.*' - if [[ $query =~ $comment_re ]]; then - echo "$query" - echo - else - echo "query:" "$query" - expect -c "#!/bin/bash -#!/bin/expect - -# Set timeout -set timeout 600 - -# Get arguments -set query [lindex $argv 0] - -spawn clickhouse-client --multiline; -expect \":) \" -send \"$query;\r\"; -expect \":) \" -send \"quit\";" >> "$log" - fi - let "index = $index + 1" - done - - echo "stop time: $(date)" >> $log -} - -mapfile -t test_queries < $test_file - -execute "${test_queries[@]}" - -echo "Error list" -cat $log - -echo -echo Error list\: -cat $log | grep -iP 'error|exception' From 6aa304ba4a0ad37127e674c46875b19cd407251b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Mar 2014 03:14:51 +0400 Subject: [PATCH 23/56] dbms: added tests [#METR-10572]. --- dbms/tests/queries/0_stateless/00020_sorting_arrays.reference | 4 ++++ dbms/tests/queries/0_stateless/00020_sorting_arrays.sql | 1 + dbms/tests/queries/0_stateless/00021_sorting_arrays.reference | 4 ++++ dbms/tests/queries/0_stateless/00021_sorting_arrays.sql | 1 + 4 files changed, 10 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00020_sorting_arrays.reference create mode 100644 dbms/tests/queries/0_stateless/00020_sorting_arrays.sql create mode 100644 dbms/tests/queries/0_stateless/00021_sorting_arrays.reference create mode 100644 dbms/tests/queries/0_stateless/00021_sorting_arrays.sql diff --git a/dbms/tests/queries/0_stateless/00020_sorting_arrays.reference b/dbms/tests/queries/0_stateless/00020_sorting_arrays.reference new file mode 100644 index 00000000000..95c9078cae1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00020_sorting_arrays.reference @@ -0,0 +1,4 @@ +[1,1] +[2] +[3,4,5] +[6,7] diff --git a/dbms/tests/queries/0_stateless/00020_sorting_arrays.sql b/dbms/tests/queries/0_stateless/00020_sorting_arrays.sql new file mode 100644 index 00000000000..69ddf6c2b0d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00020_sorting_arrays.sql @@ -0,0 +1 @@ +SELECT arrayJoin([[3,4,5], [6,7], [2], [1,1]]) AS x ORDER BY x diff --git a/dbms/tests/queries/0_stateless/00021_sorting_arrays.reference b/dbms/tests/queries/0_stateless/00021_sorting_arrays.reference new file mode 100644 index 00000000000..9ff07694305 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00021_sorting_arrays.reference @@ -0,0 +1,4 @@ +[6,7] +[3,4,5] +[2] +[1,1] diff --git a/dbms/tests/queries/0_stateless/00021_sorting_arrays.sql b/dbms/tests/queries/0_stateless/00021_sorting_arrays.sql new file mode 100644 index 00000000000..e2034d94fa2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00021_sorting_arrays.sql @@ -0,0 +1 @@ +SELECT arrayJoin([[3,4,5], [6,7], [2], [1,1]]) AS x ORDER BY x DESC From 369767dde935ae5e606a1cf9e00a129eb2f7b887 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Mar 2014 03:15:14 +0400 Subject: [PATCH 24/56] dbms: test: allowed to specify many test names to generate reference result [#METR-10572]. --- dbms/tests/clickhouse-test | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 6911441e468..3fcc18f5c5a 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -21,6 +21,15 @@ MSG_GENERATED="${COLOR_WHITE}[ ${COLOR_UNKNOWN}GENERATED${COLOR_WHITE} ]${COLOR_ ERRORS=0 + +if [ "$1" == "--generate" ]; then + GENERATE=1 + shift +else + GENERATE=0 +fi + + for dir in $(ls $QUERIES_DIR) do tests_name=$(echo $dir | sed -E 's/^[0-9_]+//') @@ -68,11 +77,11 @@ do cat $result_file elif [ ! -e "$reference_file" ]; then # надо сгенерировать эталонный результат - if [ "$1" == "--generate" ] && ( [ "$2" == "" ] || [ "$2" == "$test_name" ] ); then + if [[ $GENERATE -eq 1 && ( -z "$1" || "$@" =~ "$test_name") ]]; then cp $result_file $reference_file echo -e "$MSG_GENERATED - no reference file" else - echo -e "$MSG_UNKNOWN - no reference file (use --generate [test_name] to create)" + echo -e "$MSG_UNKNOWN - no reference file (use --generate [test_name]... to create)" fi else diff $reference_file $result_file > $diff_file From 78e59c0394a291c287ecdd26f27650252939c78b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Mar 2014 03:16:20 +0400 Subject: [PATCH 25/56] dbms: fixed sorting of arrays [#METR-10572]. --- dbms/include/DB/Columns/ColumnArray.h | 39 +++++++++++---------------- 1 file changed, 15 insertions(+), 24 deletions(-) diff --git a/dbms/include/DB/Columns/ColumnArray.h b/dbms/include/DB/Columns/ColumnArray.h index 3b697510e22..97901d5daad 100644 --- a/dbms/include/DB/Columns/ColumnArray.h +++ b/dbms/include/DB/Columns/ColumnArray.h @@ -229,7 +229,7 @@ public: return res; } - int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const + int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const final { const ColumnArray & rhs = static_cast(rhs_); @@ -238,7 +238,7 @@ public: size_t rhs_size = rhs.sizeAt(m); size_t min_size = std::min(lhs_size, rhs_size); for (size_t i = 0; i < min_size; ++i) - if (int res = data->compareAt(offsetAt(n) + i, rhs.offsetAt(m) + i, *rhs.data, nan_direction_hint)) + if (int res = data.get()->compareAt(offsetAt(n) + i, rhs.offsetAt(m) + i, *rhs.data.get(), nan_direction_hint)) return res; return lhs_size < rhs_size @@ -252,50 +252,41 @@ public: struct less { const ColumnArray & parent; - const Permutation & nested_perm; - less(const ColumnArray & parent_, const Permutation & nested_perm_) : parent(parent_), nested_perm(nested_perm_) {} + less(const ColumnArray & parent_) : parent(parent_) {} + bool operator()(size_t lhs, size_t rhs) const { - size_t lhs_size = parent.sizeAt(lhs); - size_t rhs_size = parent.sizeAt(rhs); - size_t min_size = std::min(lhs_size, rhs_size); - for (size_t i = 0; i < min_size; ++i) - { - if (nested_perm[parent.offsetAt(lhs) + i] < nested_perm[parent.offsetAt(rhs) + i]) - return positive; - else if (nested_perm[parent.offsetAt(lhs) + i] > nested_perm[parent.offsetAt(rhs) + i]) - return !positive; - } - return positive == (lhs_size < rhs_size); + if (positive) + return parent.compareAt(lhs, rhs, parent, 1) < 0; + else + return parent.compareAt(lhs, rhs, parent, -1) > 0; } }; void getPermutation(bool reverse, size_t limit, Permutation & res) const { - Permutation nested_perm; - data->getPermutation(reverse, limit, nested_perm); size_t s = size(); + if (limit > s) + limit = 0; + res.resize(s); for (size_t i = 0; i < s; ++i) res[i] = i; - if (limit > s) - limit = 0; - if (limit) { if (reverse) - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this, nested_perm)); + std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this, nested_perm)); + std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); } else { if (reverse) - std::sort(res.begin(), res.end(), less(*this, nested_perm)); + std::sort(res.begin(), res.end(), less(*this)); else - std::sort(res.begin(), res.end(), less(*this, nested_perm)); + std::sort(res.begin(), res.end(), less(*this)); } } From f0f09d00fd5229f06d0abcbe7c6e60083cb81160 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Mar 2014 04:17:31 +0400 Subject: [PATCH 26/56] dbms: fixed unitialized memory read [#METR-10597]. --- dbms/include/DB/IO/CompressedReadBufferBase.h | 1 - dbms/include/DB/IO/CompressedReadBufferFromFile.h | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/include/DB/IO/CompressedReadBufferBase.h b/dbms/include/DB/IO/CompressedReadBufferBase.h index d6eb20ccf7d..b2bd8a7b60f 100644 --- a/dbms/include/DB/IO/CompressedReadBufferBase.h +++ b/dbms/include/DB/IO/CompressedReadBufferBase.h @@ -26,7 +26,6 @@ protected: /// Если в буфере compressed_in помещается целый сжатый блок - используем его. Иначе - копируем данные по кусочкам в own_compressed_buffer. PODArray own_compressed_buffer; char * compressed_buffer; - size_t size_compressed; qlz_state_decompress * qlz_state; diff --git a/dbms/include/DB/IO/CompressedReadBufferFromFile.h b/dbms/include/DB/IO/CompressedReadBufferFromFile.h index 47cd9d7bcb1..fa9e0552a57 100644 --- a/dbms/include/DB/IO/CompressedReadBufferFromFile.h +++ b/dbms/include/DB/IO/CompressedReadBufferFromFile.h @@ -19,7 +19,7 @@ private: * - size_compressed содержит сжатый размер этого блока. */ ReadBufferFromFile file_in; - size_t size_compressed; + size_t size_compressed = 0; bool nextImpl() { From 651e593bb2f1d6b2593a50bd9145f6277d591f48 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Mar 2014 04:34:00 +0400 Subject: [PATCH 27/56] dbms: fixed error with aggregate function quantileTiming [#METR-10597]. --- .../DB/AggregateFunctions/AggregateFunctionQuantileTiming.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h index 06ae5222c41..86b8cd78389 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h @@ -324,11 +324,13 @@ private: void toLarge() { - large = new detail::QuantileTimingLarge; + /// На время копирования данных из tiny, устанавливать значение large ещё нельзя (иначе оно перезатрёт часть данных). + detail::QuantileTimingLarge * tmp_large = new detail::QuantileTimingLarge; for (size_t i = 0; i < tiny.count; ++i) - large->insert(tiny.elems[i]); + tmp_large->insert(tiny.elems[i]); + large = tmp_large; tiny.count = TINY_MAX_ELEMS + 1; } From 4042069b3eb35dbfa51a4524a9feb800c946bacf Mon Sep 17 00:00:00 2001 From: Pavel Kartavyy Date: Wed, 26 Mar 2014 18:08:00 +0400 Subject: [PATCH 28/56] fixed build --- dbms/src/Common/tests/auto_array.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/tests/auto_array.cpp b/dbms/src/Common/tests/auto_array.cpp index fd0aff7ea76..3d8ae6fa9a0 100644 --- a/dbms/src/Common/tests/auto_array.cpp +++ b/dbms/src/Common/tests/auto_array.cpp @@ -55,7 +55,7 @@ int main(int argc, char ** argv) std::cerr << std::endl; - Arr arr2 = arr; + Arr arr2 = std::move(arr); std::cerr << arr.size() << ", " << arr2.size() << std::endl; From e060784ecca1120396f5d4aea2888ec01ae69e72 Mon Sep 17 00:00:00 2001 From: Pavel Kartavyy Date: Wed, 26 Mar 2014 18:45:50 +0400 Subject: [PATCH 29/56] fixed build --- dbms/src/Common/tests/auto_array.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Common/tests/auto_array.cpp b/dbms/src/Common/tests/auto_array.cpp index 3d8ae6fa9a0..43c8c359f9f 100644 --- a/dbms/src/Common/tests/auto_array.cpp +++ b/dbms/src/Common/tests/auto_array.cpp @@ -79,7 +79,7 @@ int main(int argc, char ** argv) for (size_t j = 0; j < n; ++j) key[j] = DB::toString(rand()); - map[key] = "Hello, world! " + DB::toString(i); + map[std::move(key)] = "Hello, world! " + DB::toString(i); } for (Map::const_iterator it = map.begin(); it != map.end(); ++it) @@ -94,7 +94,7 @@ int main(int argc, char ** argv) std::cerr << std::endl; - Map map2 = map; + Map map2 = std::move(map); for (Map::const_iterator it = map2.begin(); it != map2.end(); ++it) { @@ -123,7 +123,7 @@ int main(int argc, char ** argv) for (size_t j = 0; j < n; ++j) key[j] = DB::toString(rand()); - vec.push_back(key); + vec.push_back(std::move(key)); } for (Vec::const_iterator it = vec.begin(); it != vec.end(); ++it) @@ -136,7 +136,7 @@ int main(int argc, char ** argv) std::cerr << std::endl; - Vec vec2 = vec; + Vec vec2 = std::move(vec); for (Vec::const_iterator it = vec2.begin(); it != vec2.end(); ++it) { @@ -224,7 +224,7 @@ int main(int argc, char ** argv) arr2[i] = "Goodbye, world! " + DB::toString(i); } - arr2 = arr1; + arr2 = std::move(arr1); arr1.resize(n); std::cerr From 966995b011e3112f23190f7f5d2f15ef1642e194 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Mar 2014 22:29:30 +0400 Subject: [PATCH 30/56] Fixed build [#METR-2807]. --- dbms/src/Interpreters/tests/hash_map.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Interpreters/tests/hash_map.cpp b/dbms/src/Interpreters/tests/hash_map.cpp index 1bac5fb5f3a..684df7bc3fd 100644 --- a/dbms/src/Interpreters/tests/hash_map.cpp +++ b/dbms/src/Interpreters/tests/hash_map.cpp @@ -48,7 +48,7 @@ * Но в этом тесте осталось нечто похожее на старый сценарий использования хэш-таблиц при агрегации. */ -#define USE_AUTO_ARRAY 1 +#define USE_AUTO_ARRAY 0 int main(int argc, char ** argv) @@ -131,7 +131,7 @@ int main(int argc, char ** argv) map.emplace(data[i], it, inserted); if (inserted) { - new(&it->second) Value(value); + new(&it->second) Value(std::move(value)); INIT; } } @@ -155,7 +155,7 @@ int main(int argc, char ** argv) std::unordered_map >::iterator it; for (size_t i = 0; i < n; ++i) { - it = map.insert(std::make_pair(data[i], value)).first; + it = map.insert(std::make_pair(data[i], std::move(value))).first; INIT; } @@ -176,7 +176,7 @@ int main(int argc, char ** argv) map.set_empty_key(-1ULL); for (size_t i = 0; i < n; ++i) { - it = map.insert(std::make_pair(data[i], value)).first; + it = map.insert(std::make_pair(data[i], std::move(value))).first; INIT; } @@ -196,7 +196,7 @@ int main(int argc, char ** argv) google::sparse_hash_map >::iterator it; for (size_t i = 0; i < n; ++i) { - map.insert(std::make_pair(data[i], value)); + map.insert(std::make_pair(data[i], std::move(value))); INIT; } From 64251e04d1545c439ea6c1c9dcc04b9ddea9d48e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Mar 2014 02:23:54 +0400 Subject: [PATCH 31/56] dbms: added tests [#METR-8757] [#METR-9119]. --- .../0_stateless/00022_func_higher_order_and_constants.reference | 1 + .../0_stateless/00022_func_higher_order_and_constants.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00022_func_higher_order_and_constants.reference create mode 100644 dbms/tests/queries/0_stateless/00022_func_higher_order_and_constants.sql diff --git a/dbms/tests/queries/0_stateless/00022_func_higher_order_and_constants.reference b/dbms/tests/queries/0_stateless/00022_func_higher_order_and_constants.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00022_func_higher_order_and_constants.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00022_func_higher_order_and_constants.sql b/dbms/tests/queries/0_stateless/00022_func_higher_order_and_constants.sql new file mode 100644 index 00000000000..c2831e52763 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00022_func_higher_order_and_constants.sql @@ -0,0 +1 @@ +select arrayExists(x -> position(x, 'a') > 0, ['a']) From 8bbee1f1c18ec0818226fdb5c9260cf2cc6bc933 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 26 Mar 2014 23:44:51 +0400 Subject: [PATCH 32/56] dbms: Added HashingWriteBuffer and a unit test. [#METR-10202] --- dbms/include/DB/IO/HashingWriteBuffer.h | 97 ++++++++++++++++++++ dbms/src/IO/tests/hashing_write_buffer.cpp | 102 +++++++++++++++++++++ 2 files changed, 199 insertions(+) create mode 100644 dbms/include/DB/IO/HashingWriteBuffer.h create mode 100644 dbms/src/IO/tests/hashing_write_buffer.cpp diff --git a/dbms/include/DB/IO/HashingWriteBuffer.h b/dbms/include/DB/IO/HashingWriteBuffer.h new file mode 100644 index 00000000000..b25d50aeaf9 --- /dev/null +++ b/dbms/include/DB/IO/HashingWriteBuffer.h @@ -0,0 +1,97 @@ +#pragma once + +#include +#include +#include + +#define DBMS_DEFAULT_HASHING_BLOCK_SIZE 2048ULL + + +namespace DB +{ + +/** Вычисляет хеш от записываемых данных и передает их в указанный WriteBuffer. + * В качестве основного буфера используется буфер вложенного WriteBuffer. + */ +class HashingWriteBuffer : public BufferWithOwnMemory +{ +private: + WriteBuffer & out; + + size_t block_size; + size_t block_pos; + uint128 state; + + void append(Position data) + { + state = CityHash128WithSeed(data, block_size, state); + } + + void nextImpl() override + { + size_t len = offset(); + + if (len) + { + Position data = working_buffer.begin(); + + if (block_pos + len < block_size) + { + memcpy(&memory[block_pos], data, len); + block_pos += len; + } + else + { + if (block_pos) + { + size_t n = block_size - block_pos; + memcpy(&memory[block_pos], data, n); + append(&memory[0]); + len -= n; + data += n; + block_pos = 0; + } + + while (len >= block_size) + { + append(data); + len -= block_size; + data += block_size; + } + + if (len) + { + memcpy(&memory[0], data, len); + block_pos = len; + } + } + } + + out.position() = pos; + out.next(); + working_buffer = out.buffer(); + } + +public: + HashingWriteBuffer( + WriteBuffer & out_, + size_t block_size_ = DBMS_DEFAULT_HASHING_BLOCK_SIZE) + : BufferWithOwnMemory(block_size_), out(out_), block_size(block_size_), block_pos(0) + { + out.next(); /// Если до нас в out что-то уже писали, не дадим остаткам этих данных повлиять на хеш. + working_buffer = out.buffer(); + pos = working_buffer.begin(); + state = uint128(0, 0); + } + + uint128 getHash() + { + next(); + if (block_pos) + return CityHash128WithSeed(&memory[0], block_pos, state); + else + return state; + } +}; + +} diff --git a/dbms/src/IO/tests/hashing_write_buffer.cpp b/dbms/src/IO/tests/hashing_write_buffer.cpp new file mode 100644 index 00000000000..1143ae759af --- /dev/null +++ b/dbms/src/IO/tests/hashing_write_buffer.cpp @@ -0,0 +1,102 @@ +#include +#include + +#define FAIL(msg) { std::cout << msg; exit(1); } + + +uint128 referenceHash(char * data, size_t len) +{ + const size_t block_size = DBMS_DEFAULT_HASHING_BLOCK_SIZE; + uint128 state(0, 0); + size_t pos; + + for (pos = 0; pos + block_size <= len; pos += block_size) + { + state = CityHash128WithSeed(data + pos, block_size, state); + } + + if (pos < len) + state = CityHash128WithSeed(data + pos, len - pos, state); + + return state; +} + + +void test(size_t data_size) +{ + std::vector vec(data_size); + char * data = &vec[0]; + + for (size_t i = 0; i < data_size; ++i) + data[i] = rand() & 255; + + uint128 reference = referenceHash(data, data_size); + + DB::WriteBufferFromFile sink("/dev/null", 1 << 16); + + { + DB::HashingWriteBuffer buf(sink); + + for (size_t pos = 0; pos < data_size;) + { + size_t len = std::min(static_cast(rand() % 10000 + 1), data_size - pos); + buf.write(data + pos, len); + buf.next(); + pos += len; + } + + if (buf.getHash() != reference) + FAIL("failed on data size " << data_size << " writing random chunks of up to 10000 bytes"); + } + + { + DB::HashingWriteBuffer buf(sink); + + for (size_t pos = 0; pos < data_size;) + { + size_t len = std::min(static_cast(rand() % 5 + 1), data_size - pos); + buf.write(data + pos, len); + buf.next(); + pos += len; + } + + if (buf.getHash() != reference) + FAIL("failed on data size " << data_size << " writing random chunks of up to 5 bytes"); + } + + { + DB::HashingWriteBuffer buf(sink); + + for (size_t pos = 0; pos < data_size;) + { + size_t len = std::min(static_cast(2048 + rand() % 3 - 1), data_size - pos); + buf.write(data + pos, len); + buf.next(); + pos += len; + } + + if (buf.getHash() != reference) + FAIL("failed on data size " << data_size << " writing random chunks of 2048 +-1 bytes"); + } + + { + DB::HashingWriteBuffer buf(sink); + + buf.write(data, data_size); + + if (buf.getHash() != reference) + FAIL("failed on data size " << data_size << " writing all at once"); + } +} + +int main() +{ + test(5); + test(100); + test(2048); + test(2049); + test(100000); + test(1 << 17); + + return 0; +} From 45dcf66dd8b9fc259c3849c66e7440ebce8b56e7 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 27 Mar 2014 12:40:35 +0400 Subject: [PATCH 33/56] Added test query. [#METR-10615] --- .../queries/0_stateless/00023_agg_select_agg_subquery.reference | 1 + dbms/tests/queries/0_stateless/00023_agg_select_agg_subquery.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00023_agg_select_agg_subquery.reference create mode 100644 dbms/tests/queries/0_stateless/00023_agg_select_agg_subquery.sql diff --git a/dbms/tests/queries/0_stateless/00023_agg_select_agg_subquery.reference b/dbms/tests/queries/0_stateless/00023_agg_select_agg_subquery.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00023_agg_select_agg_subquery.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00023_agg_select_agg_subquery.sql b/dbms/tests/queries/0_stateless/00023_agg_select_agg_subquery.sql new file mode 100644 index 00000000000..398b4ff8b89 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00023_agg_select_agg_subquery.sql @@ -0,0 +1 @@ +SELECT count() FROM (SELECT sum(1), sum(2)) From db3c061396bb1ee8a06df8c13f90d43cba5eb595 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 27 Mar 2014 15:29:40 +0400 Subject: [PATCH 34/56] Added checksums to data parts. Not writing them yet. [#METR-10202] --- dbms/include/DB/Core/ErrorCodes.h | 3 + dbms/include/DB/IO/ReadHelpers.h | 1 + .../DB/Storages/MergeTree/MergeTreeData.h | 37 +++++++++ dbms/src/IO/ReadHelpers.cpp | 6 ++ dbms/src/Storages/MergeTree/MergeTreeData.cpp | 77 +++++++++++++++++++ 5 files changed, 124 insertions(+) diff --git a/dbms/include/DB/Core/ErrorCodes.h b/dbms/include/DB/Core/ErrorCodes.h index b875e465ba4..a5a36a09f18 100644 --- a/dbms/include/DB/Core/ErrorCodes.h +++ b/dbms/include/DB/Core/ErrorCodes.h @@ -227,6 +227,9 @@ namespace ErrorCodes CLIENT_HAS_CONNECTED_TO_WRONG_PORT, TABLE_IS_DROPPED, DATABASE_NOT_EMPTY, + NO_FILE_IN_DATA_PART, + UNEXPECTED_FILE_IN_DATA_PART, + BAD_SIZE_OF_FILE_IN_DATA_PART, POCO_EXCEPTION = 1000, STD_EXCEPTION, diff --git a/dbms/include/DB/IO/ReadHelpers.h b/dbms/include/DB/IO/ReadHelpers.h index cb336d7194b..d45fadb78ff 100644 --- a/dbms/include/DB/IO/ReadHelpers.h +++ b/dbms/include/DB/IO/ReadHelpers.h @@ -114,6 +114,7 @@ inline void readChar(char & x, ReadBuffer & buf) } void assertString(const char * s, ReadBuffer & buf); +void assertEOF(ReadBuffer & buf); inline void readBoolText(bool & x, ReadBuffer & buf) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeData.h b/dbms/include/DB/Storages/MergeTree/MergeTreeData.h index 593eef78765..dc7af53d186 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeData.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeData.h @@ -33,6 +33,7 @@ namespace DB * Структура файлов: * / min-date _ max-date _ min-id _ max-id _ level / - директория с куском. * Внутри директории с куском: + * checksums.txt - список файлов с их размерами и контрольными суммами. * primary.idx - индексный файл. * Column.bin - данные столбца * Column.mrk - засечки, указывающие, откуда начинать чтение, чтобы пропустить n * k строк. @@ -99,6 +100,28 @@ public: /// Описание куска с данными. struct DataPart { + /** Контрольные суммы всех не временных файлов. + * Для сжатых файлов хранятся чексумма и размер разжатых данных, чтобы не зависеть от способа сжатия. + */ + struct Checksums + { + struct Checksum + { + size_t size; + uint128 hash; + }; + + typedef std::map FileChecksums; + FileChecksums file_checksums; + + /// Проверяет, что множество столбцов и их контрольные суммы совпадают. Если нет - бросает исключение. + void check(const Checksums & rhs) const; + + /// Сериализует и десериализует в человекочитаемом виде. + void readText(ReadBuffer & in); + void writeText(WriteBuffer & out) const; + }; + DataPart(MergeTreeData & storage_) : storage(storage_), size_in_bytes(0) {} MergeTreeData & storage; @@ -121,6 +144,8 @@ public: typedef std::vector Index; Index index; + Checksums checksums; + /// NOTE можно загружать засечки тоже в оперативку /// Вычисляем сумарный размер всей директории со всеми файлами @@ -204,6 +229,18 @@ public: size_in_bytes = calcTotalSize(storage.full_path + name + "/"); } + + /// Прочитать контрольные суммы, если есть. + bool loadChecksums() + { + String path = storage.full_path + name + "/checksums.txt"; + if (!Poco::File(path).exists()) + return false; + ReadBufferFromFile file(path, std::min(static_cast(DBMS_DEFAULT_BUFFER_SIZE), Poco::File(path).getSize())); + checksums.readText(file); + assertEOF(file); + return true; + } }; typedef std::shared_ptr MutableDataPartPtr; diff --git a/dbms/src/IO/ReadHelpers.cpp b/dbms/src/IO/ReadHelpers.cpp index dcb42e86d54..dac6048f6d7 100644 --- a/dbms/src/IO/ReadHelpers.cpp +++ b/dbms/src/IO/ReadHelpers.cpp @@ -36,6 +36,12 @@ void assertString(const char * s, ReadBuffer & buf) } } +void assertEOF(ReadBuffer & buf) +{ + if (!buf.eof()) + throwAtAssertionFailed("eof", buf); +} + void readString(String & s, ReadBuffer & buf) { s = ""; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 580504feb0e..4199283126f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -212,6 +212,7 @@ void MergeTreeData::loadDataParts() try { part->loadIndex(); + part->loadChecksums(); } catch (...) { @@ -687,4 +688,80 @@ MergeTreeData::DataParts MergeTreeData::getDataParts() return data_parts; } + +void MergeTreeData::DataPart::Checksums::check(const Checksums & rhs) const +{ + for (const auto & it : rhs.file_checksums) + { + const String & name = it.first; + + if (!file_checksums.count(name)) + throw Exception("Unexpected file " + name + " in data part", ErrorCodes::UNEXPECTED_FILE_IN_DATA_PART); + } + + for (const auto & it : file_checksums) + { + const String & name = it.first; + + auto jt = rhs.file_checksums.find(name); + if (jt == rhs.file_checksums.end()) + throw Exception("No file " + name + " in data part", ErrorCodes::NO_FILE_IN_DATA_PART); + + const Checksum & expected = it.second; + const Checksum & found = jt->second; + + if (expected.size != found.size) + throw Exception("Unexpected size of file " + name + " in data part", ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART); + + if (expected.hash != found.hash) + throw Exception("Checksum mismatch for file " + name + " in data part", ErrorCodes::CHECKSUM_DOESNT_MATCH); + } +} + +void MergeTreeData::DataPart::Checksums::readText(ReadBuffer & in) +{ + file_checksums.clear(); + size_t count; + + DB::assertString("checksums format version: 1\n", in); + DB::readText(count, in); + DB::assertString(" files:\n", in); + + for (size_t i = 0; i < count; ++i) + { + String name; + Checksum sum; + + DB::readString(name, in); + DB::assertString("\n\tsize: ", in); + DB::readText(sum.size, in); + DB::assertString("\n\thash: ", in); + DB::readText(sum.hash.first, in); + DB::assertString(" ", in); + DB::readText(sum.hash.second, in); + DB::assertString("\n", in); + + file_checksums.insert(std::make_pair(name, sum)); + } +} + +void MergeTreeData::DataPart::Checksums::writeText(WriteBuffer & out) const +{ + DB::writeString("checksums format version: 1\n", out); + DB::writeText(file_checksums.size(), out); + DB::writeString(" files:\n", out); + + for (const auto & it : file_checksums) + { + DB::writeString(it.first, out); + DB::writeString("\n\tsize: ", out); + DB::writeText(it.second.size, out); + DB::writeString("\n\thash: ", out); + DB::writeText(it.second.hash.first, out); + DB::writeString(" ", out); + DB::writeText(it.second.hash.second, out); + DB::writeString("\n", out); + } +} + } From 2b7cdfd950dcfc741264910dada662f745aabfc8 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 27 Mar 2014 15:30:54 +0400 Subject: [PATCH 35/56] Merge --- dbms/src/Storages/StorageMergeTree.cpp | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 5e2b2dfd549..fe3d5be0ffe 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -134,6 +134,8 @@ void StorageMergeTree::mergeThread(bool while_can, bool aggressive) { while (!shutdown_called) { + auto structure_lock = lockStructure(false); + /// Удаляем старые куски. На случай, если в слиянии что-то сломано, и из следующего блока вылетит исключение. data.clearOldParts(); @@ -161,17 +163,13 @@ void StorageMergeTree::mergeThread(bool while_can, bool aggressive) } } - { - auto structure_lock = lockStructure(false); - if (!merger.selectPartsToMerge(parts, disk_space, false, aggressive, only_small, can_merge) && - !merger.selectPartsToMerge(parts, disk_space, true, aggressive, only_small, can_merge)) - break; - } + if (!merger.selectPartsToMerge(parts, disk_space, false, aggressive, only_small, can_merge) && + !merger.selectPartsToMerge(parts, disk_space, true, aggressive, only_small, can_merge)) + break; merging_tagger = new CurrentlyMergingPartsTagger(parts, merger.estimateDiskSpaceForMerge(parts), *this); } - auto structure_lock = lockStructure(true); merger.mergeParts(merging_tagger->parts); } From b48bc12739301210862fe31c16529ed8187276cc Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 27 Mar 2014 15:32:41 +0400 Subject: [PATCH 36/56] Merge --- .../Storages/MergeTree/MergeTreeDataWriter.h | 6 +- .../MergeTree/MergeTreeDataWriter.cpp | 81 ++++++++++--------- 2 files changed, 44 insertions(+), 43 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeDataWriter.h b/dbms/include/DB/Storages/MergeTree/MergeTreeDataWriter.h index 6e720bfd791..2220456ac33 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeDataWriter.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeDataWriter.h @@ -34,12 +34,12 @@ public: MergeTreeDataWriter(MergeTreeData & data_) : data(data_), log(&Logger::get("MergeTreeDataWriter")), flags(O_TRUNC | O_CREAT | O_WRONLY) {} /** Разбивает блок на блоки, каждый из которых нужно записать в отдельный кусок. - * (читай: разбивает строки по месяцам) + * (читай: разбивает строки по месяцам) * Работает детерминированно: если отдать на вход такой же блок, на выходе получатся такие же блоки в таком же порядке. */ BlocksWithDateIntervals splitBlockIntoParts(const Block & block); - /** Все строки должны относиться к одному месяцу. Возвращает название временного куска. + /** Все строки должны относиться к одному месяцу. * temp_index - значение left и right для нового куска. Можно будет изменить при переименовании. * Возвращает кусок с именем, начинающимся с tmp_, еще не добавленный в MergeTreeData. */ @@ -56,7 +56,7 @@ private: /// Записать данные одного столбца. void writeData(const String & path, const String & name, const IDataType & type, const IColumn & column, - OffsetColumns & offset_columns, size_t level = 0); + OffsetColumns & offset_columns, MergeTreeData::DataPart::Checksums & checksums, size_t level = 0); }; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp index a7c1550c7ee..e2ebfaab235 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -104,12 +105,15 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa /// Наконец-то можно писать данные на диск. LOG_TRACE(log, "Writing index."); - /// Сначала пишем индекс. Индекс содержит значение PK для каждой index_granularity строки. + MergeTreeData::DataPart::Checksums checksums; MergeTreeData::DataPart::Index index_vec; + + /// Сначала пишем индекс. Индекс содержит значение PK для каждой index_granularity строки. index_vec.reserve(part_size * sort_descr.size()); { - WriteBufferFromFile index(part_tmp_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, flags); + WriteBufferFromFile index_file(part_tmp_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, flags); + HashingWriteBuffer index(index_file); typedef std::vector PrimaryColumns; PrimaryColumns primary_columns; @@ -130,6 +134,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa } index.next(); + checksums.file_checksums["primary.idx"].size = index.count(); + checksums.file_checksums["primary.idx"].hash = index.getHash(); } LOG_TRACE(log, "Writing data."); @@ -140,7 +146,14 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa for (size_t i = 0; i < columns; ++i) { const ColumnWithNameAndType & column = block.getByPosition(i); - writeData(part_tmp_path, column.name, *column.type, *column.column, offset_columns); + writeData(part_tmp_path, column.name, *column.type, *column.column, offset_columns, checksums); + } + + /// Запишем файл с чексуммами. + { + WriteBufferFromFile checksums_file(part_tmp_path + "checksums.txt", 1024, flags); + checksums.writeText(checksums_file); + checksums_file.next(); } MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared(data); @@ -155,12 +168,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa new_data_part->left_month = date_lut.toFirstDayNumOfMonth(new_data_part->left_date); new_data_part->right_month = date_lut.toFirstDayNumOfMonth(new_data_part->right_date); new_data_part->index.swap(index_vec); + new_data_part->checksums = checksums; return new_data_part; } void MergeTreeDataWriter::writeData(const String & path, const String & name, const IDataType & type, const IColumn & column, - OffsetColumns & offset_columns, size_t level) + OffsetColumns & offset_columns, MergeTreeData::DataPart::Checksums & checksums, size_t level) { String escaped_column_name = escapeForFileName(name); size_t size = column.size(); @@ -174,15 +188,17 @@ void MergeTreeDataWriter::writeData(const String & path, const String & name, co { offset_columns.insert(size_name); - WriteBufferFromFile plain(path + size_name + ".bin", DBMS_DEFAULT_BUFFER_SIZE, flags); - WriteBufferFromFile marks(path + size_name + ".mrk", 4096, flags); - CompressedWriteBuffer compressed(plain); + WriteBufferFromFile plain_file(path + size_name + ".bin", DBMS_DEFAULT_BUFFER_SIZE, flags); + WriteBufferFromFile marks_file(path + size_name + ".mrk", 4096, flags); + CompressedWriteBuffer compressed_file(plain_file); + HashingWriteBuffer marks(marks_file); + HashingWriteBuffer compressed(compressed_file); size_t prev_mark = 0; while (prev_mark < size) { /// Каждая засечка - это: (смещение в файле до начала сжатого блока, смещение внутри блока) - writeIntBinary(plain.count(), marks); + writeIntBinary(plain_file.count(), marks); writeIntBinary(compressed.offset(), marks); type_arr->serializeOffsets(column, compressed, prev_mark, data.index_granularity); @@ -192,45 +208,26 @@ void MergeTreeDataWriter::writeData(const String & path, const String & name, co } compressed.next(); - plain.next(); + plain_file.next(); marks.next(); + checksums.file_checksums[size_name + ".bin"].size = compressed.count(); + checksums.file_checksums[size_name + ".bin"].hash = compressed.getHash(); + checksums.file_checksums[size_name + ".mrk"].size = marks.count(); + checksums.file_checksums[size_name + ".mrk"].hash = marks.getHash(); } } - if (const DataTypeNested * type_nested = dynamic_cast(&type)) - { - String size_name = escaped_column_name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level); - WriteBufferFromFile plain(path + size_name + ".bin", DBMS_DEFAULT_BUFFER_SIZE, flags); - WriteBufferFromFile marks(path + size_name + ".mrk", 4096, flags); - CompressedWriteBuffer compressed(plain); + { + WriteBufferFromFile plain_file(path + escaped_column_name + ".bin", DBMS_DEFAULT_BUFFER_SIZE, flags); + WriteBufferFromFile marks_file(path + escaped_column_name + ".mrk", 4096, flags); + CompressedWriteBuffer compressed_file(plain_file); + HashingWriteBuffer marks(marks_file); + HashingWriteBuffer compressed(compressed_file); size_t prev_mark = 0; while (prev_mark < size) { - /// Каждая засечка - это: (смещение в файле до начала сжатого блока, смещение внутри блока) - writeIntBinary(plain.count(), marks); - writeIntBinary(compressed.offset(), marks); - - type_nested->serializeOffsets(column, compressed, prev_mark, data.index_granularity); - prev_mark += data.index_granularity; - - compressed.nextIfAtEnd(); /// Чтобы вместо засечек, указывающих на конец сжатого блока, были засечки, указывающие на начало следующего. - } - - compressed.next(); - plain.next(); - marks.next(); - } - - { - WriteBufferFromFile plain(path + escaped_column_name + ".bin", DBMS_DEFAULT_BUFFER_SIZE, flags); - WriteBufferFromFile marks(path + escaped_column_name + ".mrk", 4096, flags); - CompressedWriteBuffer compressed(plain); - - size_t prev_mark = 0; - while (prev_mark < size) - { - writeIntBinary(plain.count(), marks); + writeIntBinary(plain_file.count(), marks); writeIntBinary(compressed.offset(), marks); type.serializeBinary(column, compressed, prev_mark, data.index_granularity); @@ -240,8 +237,12 @@ void MergeTreeDataWriter::writeData(const String & path, const String & name, co } compressed.next(); - plain.next(); + plain_file.next(); marks.next(); + checksums.file_checksums[escaped_column_name + ".bin"].size = compressed.count(); + checksums.file_checksums[escaped_column_name + ".bin"].hash = compressed.getHash(); + checksums.file_checksums[escaped_column_name + ".mrk"].size = marks.count(); + checksums.file_checksums[escaped_column_name + ".mrk"].hash = marks.getHash(); } } From 0c59630ddc07d3cf797e74a08eb4639293d98de6 Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Tue, 4 Mar 2014 19:31:56 +0400 Subject: [PATCH 37/56] temporary tables: support of sending temporary talbes from client to server via tcp client, all previous versions should work fine with new one [METR-10071] --- dbms/include/DB/Client/Connection.h | 1 + dbms/include/DB/Core/Defines.h | 1 + dbms/include/DB/Interpreters/Context.h | 4 +- dbms/src/Client/Client.cpp | 3 ++ dbms/src/Client/Connection.cpp | 20 +++++++++ dbms/src/Interpreters/Context.cpp | 36 ++++++++++++++- dbms/src/Server/TCPHandler.cpp | 62 +++++++++++++++++++++----- dbms/src/Server/TCPHandler.h | 5 ++- 8 files changed, 116 insertions(+), 16 deletions(-) diff --git a/dbms/include/DB/Client/Connection.h b/dbms/include/DB/Client/Connection.h index 2793e0b2e7c..a2f542eb0d5 100644 --- a/dbms/include/DB/Client/Connection.h +++ b/dbms/include/DB/Client/Connection.h @@ -84,6 +84,7 @@ public: void sendCancel(); void sendData(const Block & block); + void sendTemporaryTables(); /// Проверить, если ли данные, которые можно прочитать. bool poll(size_t timeout_microseconds = 0); diff --git a/dbms/include/DB/Core/Defines.h b/dbms/include/DB/Core/Defines.h index 487c322a726..b4908d5f7a6 100644 --- a/dbms/include/DB/Core/Defines.h +++ b/dbms/include/DB/Core/Defines.h @@ -40,3 +40,4 @@ #define DBMS_MIN_REVISION_WITH_USER_PASSWORD 34482 #define DBMS_MIN_REVISION_WITH_TOTALS_EXTREMES 35265 #define DBMS_MIN_REVISION_WITH_STRING_QUERY_ID 39002 +#define DBMS_MIN_REVISION_WITH_TEMPRORY_TABLES 50092 diff --git a/dbms/include/DB/Interpreters/Context.h b/dbms/include/DB/Interpreters/Context.h index cc71891d2d3..91aadb4c27f 100644 --- a/dbms/include/DB/Interpreters/Context.h +++ b/dbms/include/DB/Interpreters/Context.h @@ -172,7 +172,7 @@ private: String default_format; /// Формат, используемый, если сервер сам форматирует данные, и если в запросе не задан FORMAT. /// То есть, используется в HTTP-интерфейсе. Может быть не задан - тогда используется некоторый глобальный формат по-умолчанию. - + Tables temporary_tables; /// Временные таблицы. Context * session_context; /// Контекст сессии или NULL, если его нет. (Возможно, равен this.) Context * global_context; /// Глобальный контекст или NULL, если его нет. (Возможно, равен this.) @@ -209,8 +209,10 @@ public: void assertDatabaseExists(const String & database_name) const; void assertDatabaseDoesntExist(const String & database_name) const; + StoragePtr tryGetTemporaryTable(const String & table_name) const; StoragePtr getTable(const String & database_name, const String & table_name) const; StoragePtr tryGetTable(const String & database_name, const String & table_name) const; + void addTemporaryTable(const String & table_name, StoragePtr storage); void addTable(const String & database_name, const String & table_name, StoragePtr table); void addDatabase(const String & database_name); diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index 9f42db41403..cb4e89912d0 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -432,6 +432,7 @@ private: void processOrdinaryQuery() { connection->sendQuery(query, "", QueryProcessingStage::Complete); + connection->sendTemporaryTables(); receiveResult(); } @@ -449,6 +450,7 @@ private: throw Exception("No data to insert", ErrorCodes::NO_DATA_TO_INSERT); connection->sendQuery(query_without_data, "", QueryProcessingStage::Complete); + connection->sendTemporaryTables(); /// Получим структуру таблицы Block sample = receiveSampleBlock(); @@ -857,6 +859,7 @@ private: int main(int argc, char ** argv) { DB::Client client; +// client.stopOptionsProcessing(); client.init(argc, argv); return client.run(); } diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 30c492cf068..f84b9204073 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -258,6 +258,10 @@ void Connection::sendData(const Block & block) } writeVarUInt(Protocol::Client::Data, *out); + + if (server_revision >= DBMS_MIN_REVISION_WITH_TEMPRORY_TABLES) + writeStringBinary("", *out); + block.checkNestedArraysOffsets(); block_out->write(block); maybe_compressed_out->next(); @@ -265,6 +269,17 @@ void Connection::sendData(const Block & block) } +void Connection::sendTemporaryTables() +{ + /// Если работаем со старым сервером, то никакой информации не отправляем + if (server_revision < DBMS_MIN_REVISION_WITH_TEMPRORY_TABLES) + return; + + /// Отправляем пустой блок, символизируя конец передачи данных + sendData(Block()); +} + + bool Connection::poll(size_t timeout_microseconds) { return static_cast(*in).poll(timeout_microseconds); @@ -336,6 +351,11 @@ Block Connection::receiveData() initBlockInput(); + String temporary_table_name; + + if (server_revision >= DBMS_MIN_REVISION_WITH_TEMPRORY_TABLES) + readStringBinary(temporary_table_name, *in); + /// Прочитать из сети один блок return block_in->read(); } diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 1fcc148ff9d..cb88a2eaf96 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -161,23 +161,47 @@ void Context::assertDatabaseDoesntExist(const String & database_name) const } +StoragePtr Context::tryGetTemporaryTable(const String & table_name) const +{ + Poco::ScopedLock lock(shared->mutex); + + Tables::const_iterator jt; + if (temporary_tables.end() == (jt = temporary_tables.find(table_name))) + return StoragePtr(); + + return jt->second; +} + + StoragePtr Context::getTable(const String & database_name, const String & table_name) const { Poco::ScopedLock lock(shared->mutex); + Databases::const_iterator it; + Tables::const_iterator jt; + + if (database_name.empty()) + { + StoragePtr res; + if (res = tryGetTemporaryTable(table_name)) + return res; + if (res = session_context->tryGetTemporaryTable(table_name)) + return res; + if (res = global_context->tryGetTemporaryTable(table_name)) + return res; + } String db = database_name.empty() ? current_database : database_name; - Databases::const_iterator it; if (shared->databases.end() == (it = shared->databases.find(db))) throw Exception("Database " + db + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE); - Tables::const_iterator jt; if (it->second.end() == (jt = it->second.find(table_name))) throw Exception("Table " + db + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); return jt->second; } + StoragePtr Context::tryGetTable(const String & database_name, const String & table_name) const { Poco::ScopedLock lock(shared->mutex); @@ -196,6 +220,14 @@ StoragePtr Context::tryGetTable(const String & database_name, const String & tab } +void Context::addTemporaryTable(const String & table_name, StoragePtr storage) +{ + if (temporary_tables.end() != temporary_tables.find(table_name)) + throw Exception("Temporary table " + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); + temporary_tables[table_name] = storage; +} + + void Context::addTable(const String & database_name, const String & table_name, StoragePtr table) { Poco::ScopedLock lock(shared->mutex); diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index 4aeed0f31a4..7f1234ffe7a 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -22,6 +22,8 @@ #include #include +#include + #include "TCPHandler.h" @@ -112,13 +114,21 @@ void TCPHandler::runImpl() if (!receivePacket()) continue; + /// Получить блоки временных таблиц + if (client_revision >= DBMS_MIN_REVISION_WITH_TEMPRORY_TABLES) + readData(global_settings); + /// Обрабатываем Query - + state.io = executeQuery(state.query, query_context, false, state.stage); + + if (state.io.out) + state.is_insert = true; + after_check_cancelled.restart(); after_send_progress.restart(); /// Запрос требует приёма данных от клиента? - if (state.io.out) + if (state.is_insert) processInsertQuery(global_settings); else processOrdinaryQuery(); @@ -203,13 +213,8 @@ void TCPHandler::runImpl() } -void TCPHandler::processInsertQuery(const Settings & global_settings) +void TCPHandler::readData(const Settings & global_settings) { - /// Отправляем клиенту блок - структура таблицы. - Block block = state.io.out_sample; - sendData(block); - - state.io.out->writePrefix(); while (1) { /// Ждём пакета от клиента. При этом, каждые POLL_INTERVAL сек. проверяем, не требуется ли завершить работу. @@ -220,9 +225,22 @@ void TCPHandler::processInsertQuery(const Settings & global_settings) if (Daemon::instance().isCancelled() || in->eof()) return; + std::cerr << "Receiving packet" << std::endl; + if (!receivePacket()) break; } +} + + +void TCPHandler::processInsertQuery(const Settings & global_settings) +{ + /// Отправляем клиенту блок - структура таблицы. + Block block = state.io.out_sample; + sendData(block); + + state.io.out->writePrefix(); + readData(global_settings); state.io.out->writeSuffix(); } @@ -518,19 +536,37 @@ void TCPHandler::receiveQuery() LOG_DEBUG(log, "Query ID: " << state.query_id); LOG_DEBUG(log, "Query: " << state.query); LOG_DEBUG(log, "Requested stage: " << QueryProcessingStage::toString(stage)); - - state.io = executeQuery(state.query, query_context, false, state.stage); } bool TCPHandler::receiveData() { initBlockInput(); - - /// Прочитать из сети один блок и засунуть его в state.io.out (данные для INSERT-а) + + /// Имя временной таблицы для записи данных, по умолчанию пустая строка + String temporary_table_name; + if (client_revision >= DBMS_MIN_REVISION_WITH_TEMPRORY_TABLES) + readStringBinary(temporary_table_name, *in); + + /// Прочитать из сети один блок и записать его Block block = state.block_in->read(); if (block) { + /// Если запрос на вставку, то данные нужно писать напрямую в state.io.out. + /// Иначе пишем блоки во временную таблицу temporary_table_name. + if (!state.is_insert) + { + StoragePtr storage; + /// Если такой таблицы не существовало, создаем ее. + if (!(storage = query_context.tryGetTemporaryTable(temporary_table_name))) + { + NamesAndTypesListPtr columns = new NamesAndTypesList(block.getColumnsList()); + storage = StorageMemory::create(temporary_table_name, columns); + query_context.addTemporaryTable(temporary_table_name, storage); + } + /// Данные будем писать напрямую в таблицу. + state.io.out = storage->write(ASTPtr()); + } state.io.out->write(block); return true; } @@ -614,6 +650,8 @@ void TCPHandler::sendData(Block & block) initBlockOutput(); writeVarUInt(Protocol::Server::Data, *out); + if (client_revision >= DBMS_MIN_REVISION_WITH_TEMPRORY_TABLES) + writeStringBinary("", *out); state.block_out->write(block); state.maybe_compressed_out->next(); diff --git a/dbms/src/Server/TCPHandler.h b/dbms/src/Server/TCPHandler.h index 5aeedd29719..83928aed0bf 100644 --- a/dbms/src/Server/TCPHandler.h +++ b/dbms/src/Server/TCPHandler.h @@ -45,6 +45,8 @@ struct QueryState bool is_empty; /// Данные были отправлены. bool sent_all_data; + /// Запрос на вставку или нет. + bool is_insert; /// Для вывода прогресса - разница после предыдущей отправки прогресса. volatile size_t rows_processed; @@ -52,7 +54,7 @@ struct QueryState QueryState() : query_id(""), stage(QueryProcessingStage::Complete), compression(Protocol::Compression::Disable), - is_cancelled(false), is_empty(true), sent_all_data(false), rows_processed(0), bytes_processed(0) {} + is_cancelled(false), is_empty(true), sent_all_data(false), is_insert(false), rows_processed(0), bytes_processed(0) {} void reset() { @@ -107,6 +109,7 @@ private: bool receivePacket(); void receiveQuery(); bool receiveData(); + void readData(const Settings & global_settings); /// Обработать запрос INSERT void processInsertQuery(const Settings & global_settings); From 92658ff05a7f8099fd2789dd72b2de3b39b08e18 Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Wed, 5 Mar 2014 17:48:45 +0400 Subject: [PATCH 38/56] client: comand line parsing changed from poco to boost::program_options [METR-10071] --- dbms/src/Client/Client.cpp | 91 ++++++++++++++++---------------------- 1 file changed, 37 insertions(+), 54 deletions(-) diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index cb4e89912d0..8b054b0dc45 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -15,6 +15,7 @@ #include #include +#include #include #include @@ -790,66 +791,49 @@ private: if (is_interactive && !written_first_block) std::cout << "Ok." << std::endl; } - - void defineOptions(Poco::Util::OptionSet & options) +public: + void init(int argc, char ** argv) { - Poco::Util::Application::defineOptions(options); + /// Останавливаем внутреннюю обработку командной строки + stopOptionsProcessing(); - options.addOption( - Poco::Util::Option("config-file", "c") - .required(false) - .repeatable(false) - .argument("") - .binding("config-file")); + /// Перечисляем опции командной строки + boost::program_options::options_description desc("Allowed options"); + desc.add_options() + ("config-file,c", boost::program_options::value (), "config-file") + ("host,h", boost::program_options::value ()->default_value("localhost"), "host") + ("port,p", boost::program_options::value ()->default_value(9000), "port") + ("user,u", boost::program_options::value (), "user") + ("password,p", boost::program_options::value (), "password") + ("query,q", boost::program_options::value (), "query") + ("database,d", boost::program_options::value (), "database") + ("multiline,m", "multiline") + ; - options.addOption( - Poco::Util::Option("host", "h") - .required(false) - .repeatable(false) - .argument("") - .binding("host")); + /// Парсим командную строку + boost::program_options::variables_map options; + boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); - options.addOption( - Poco::Util::Option("port", "") - .required(false) - .repeatable(false) - .argument("") - .binding("port")); + /// Сохраняем полученные значение во внутренний конфиг + if (options.count("config-file")) + config().setString("config-file", options["config-file"].as()); + if (options.count("host")) + config().setString("host", options["host"].as()); + if (options.count("query")) + config().setString("query", options["query"].as()); + if (options.count("database")) + config().setString("database", options["database"].as()); - options.addOption( - Poco::Util::Option("user", "u") - .required(false) - .repeatable(false) - .argument("") - .binding("user")); + if (options.count("port")) + config().setInt("port", options["port"].as()); + if (options.count("user")) + config().setInt("user", options["user"].as()); + if (options.count("password")) + config().setInt("password", options["password"].as()); - options.addOption( - Poco::Util::Option("password", "") - .required(false) - .repeatable(false) - .argument("") - .binding("password")); - - options.addOption( - Poco::Util::Option("query", "e") - .required(false) - .repeatable(false) - .argument("") - .binding("query")); - - options.addOption( - Poco::Util::Option("database", "d") - .required(false) - .repeatable(false) - .argument("") - .binding("database")); - - options.addOption( - Poco::Util::Option("multiline", "m") - .required(false) - .repeatable(false) - .binding("multiline")); + if (options.count("multiline")) + config().setBool("multiline", true); } }; @@ -859,7 +843,6 @@ private: int main(int argc, char ** argv) { DB::Client client; -// client.stopOptionsProcessing(); client.init(argc, argv); return client.run(); } From 3fb930ef53d510dd73d3d042ed35520ed854aba5 Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Wed, 5 Mar 2014 19:26:43 +0400 Subject: [PATCH 39/56] client: support external tables descriptions in command line arguments [METR-10071] --- dbms/src/Client/Client.cpp | 131 +++++++++++++++++++++++++++++++++++-- 1 file changed, 124 insertions(+), 7 deletions(-) diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index 8b054b0dc45..f6609eb3360 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -58,6 +58,93 @@ namespace DB using Poco::SharedPtr; +class ExternalTable +{ +public: + std::string file; + std::string name; + std::string format; + std::vector > structure; + + void write() + { + std::cerr << "file " << file << std::endl; + std::cerr << "name " << name << std::endl; + std::cerr << "format " << format << std::endl; + std::cerr << "structure: \n"; + for (size_t i = 0; i < structure.size(); ++i) + std::cerr << "\t" << structure[i].first << " " << structure[i].second << std::endl; + } + + ExternalTable(const boost::program_options::variables_map & external_options) + { + if (external_options.count("file")) + file = external_options["file"].as(); + else + throw Exception("File field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); + + if (external_options.count("name")) + name = external_options["name"].as(); + else + throw Exception("Name field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); + + if (external_options.count("format")) + format = external_options["format"].as(); + else + throw Exception("format field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); + + if (external_options.count("structure")) + { + std::vector temp = external_options["structure"].as>(); + + std::string argument; + for (size_t i = 0; i < temp.size(); ++i) + argument = argument + temp[i] + " "; + std::vector vals = split(argument, " ,"); + + if (vals.size() & 1) + throw Exception("Odd number of attributes in section structure", ErrorCodes::BAD_ARGUMENTS); + + for (size_t i = 0; i < vals.size(); i += 2) + structure.push_back(std::make_pair(vals[i], vals[i+1])); + } + else if (external_options.count("types")) + { + std::vector temp = external_options["types"].as>(); + std::string argument; + for (size_t i = 0; i < temp.size(); ++i) + argument = argument + temp[i] + " "; + std::vector vals = split(argument, " ,"); + + for (size_t i = 0; i < vals.size(); ++i) + structure.push_back(std::make_pair("_" + toString(i + 1), vals[i])); + } + else + throw Exception("Neither structure nor types have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); + } + + static std::vector split(const std::string & s, const std::string &d) + { + std::vector res; + std::string now; + for (size_t i = 0; i < s.size(); ++i) + { + if (d.find(s[i]) != std::string::npos) + { + if (!now.empty()) + res.push_back(now); + now = ""; + continue; + } + now += s[i]; + } + if (!now.empty()) + res.push_back(now); + return res; + } +}; + + class Client : public Poco::Util::Application { public: @@ -795,12 +882,13 @@ private: public: void init(int argc, char ** argv) { + /// Останавливаем внутреннюю обработку командной строки stopOptionsProcessing(); - /// Перечисляем опции командной строки - boost::program_options::options_description desc("Allowed options"); - desc.add_options() + /// Перечисляем основные опции командной строки относящиеся к функциональности клиента + boost::program_options::options_description main_description("Main options"); + main_description.add_options() ("config-file,c", boost::program_options::value (), "config-file") ("host,h", boost::program_options::value ()->default_value("localhost"), "host") ("port,p", boost::program_options::value ()->default_value(9000), "port") @@ -811,11 +899,40 @@ public: ("multiline,m", "multiline") ; - /// Парсим командную строку - boost::program_options::variables_map options; - boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); + /// Перечисляем опции командной строки относящиеся к внешним таблицам + boost::program_options::options_description external_description("Main options"); + external_description.add_options() + ("file", boost::program_options::value (), "data file or - for stdin") + ("name", boost::program_options::value ()->default_value("_data"), "name of the table") + ("format", boost::program_options::value ()->default_value("TabSeparated"), "data format") + ("structure", boost::program_options::value> ()->multitoken(), "structure") + ("types", boost::program_options::value> ()->multitoken(), "types") + ; - /// Сохраняем полученные значение во внутренний конфиг + std::vector positions; + + positions.push_back(0); + for (int i = 0; i < argc; ++i) + if (std::string(argv[i]) == "--external") + positions.push_back(i); + positions.push_back(argc); + + /// Парсим основные опции командной строки + boost::program_options::variables_map options; + boost::program_options::store(boost::program_options::parse_command_line(positions[1] - positions[0], argv, main_description), options); + + std::vector external_tables; + + for (size_t i = 1; i + 1 < positions.size(); ++i) + { + boost::program_options::variables_map external_options; + boost::program_options::store(boost::program_options::parse_command_line( + positions[i+1] - positions[i], &argv[positions[i]], external_description), external_options); + external_tables.push_back(ExternalTable(external_options)); +// external_tables.back().write(); + } + + /// Сохраняем полученные данные во внутренний конфиг if (options.count("config-file")) config().setString("config-file", options["config-file"].as()); if (options.count("host")) From 4ffc3fbdf7e8b320b96e937d46fe79c0918e281e Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Thu, 6 Mar 2014 15:37:30 +0400 Subject: [PATCH 40/56] client: try catch for External table exceptions, couple of fixes [METR-10071] --- dbms/src/Client/Client.cpp | 30 ++++++++++++++++++++---------- 1 file changed, 20 insertions(+), 10 deletions(-) diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index f6609eb3360..64527a1fac3 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -81,17 +81,17 @@ public: if (external_options.count("file")) file = external_options["file"].as(); else - throw Exception("File field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); + throw Exception("--file field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); if (external_options.count("name")) name = external_options["name"].as(); else - throw Exception("Name field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); + throw Exception("--name field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); if (external_options.count("format")) format = external_options["format"].as(); else - throw Exception("format field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); + throw Exception("--format field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); if (external_options.count("structure")) { @@ -120,7 +120,7 @@ public: structure.push_back(std::make_pair("_" + toString(i + 1), vals[i])); } else - throw Exception("Neither structure nor types have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); + throw Exception("Neither --structure nor --types have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); } static std::vector split(const std::string & s, const std::string &d) @@ -199,6 +199,9 @@ private: size_t written_progress_chars; bool written_first_block; + /// Информация о внешних таблицах + std::vector external_tables; + void initialize(Poco::Util::Application & self) { @@ -912,8 +915,8 @@ public: std::vector positions; positions.push_back(0); - for (int i = 0; i < argc; ++i) - if (std::string(argv[i]) == "--external") + for (int i = 1; i < argc; ++i) + if (strcmp(argv[i], "--external") == 0) positions.push_back(i); positions.push_back(argc); @@ -921,15 +924,22 @@ public: boost::program_options::variables_map options; boost::program_options::store(boost::program_options::parse_command_line(positions[1] - positions[0], argv, main_description), options); - std::vector external_tables; - for (size_t i = 1; i + 1 < positions.size(); ++i) { boost::program_options::variables_map external_options; boost::program_options::store(boost::program_options::parse_command_line( positions[i+1] - positions[i], &argv[positions[i]], external_description), external_options); - external_tables.push_back(ExternalTable(external_options)); -// external_tables.back().write(); + try + { + external_tables.push_back(ExternalTable(external_options)); + } + catch (const Exception & e) + { + std::string text = e.displayText(); + std::cerr << "Code: " << e.code() << ". " << text << std::endl << std::endl; + std::cerr << "Table #" << i << std::endl; + exit(e.code()); + } } /// Сохраняем полученные данные во внутренний конфиг From a17a57c4253a32d848cc53f874d45a1b612bc67a Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Thu, 6 Mar 2014 18:02:20 +0400 Subject: [PATCH 41/56] client: external data sending from client to server [METR-10071] --- dbms/include/DB/Client/Connection.h | 5 ++- dbms/src/Client/Client.cpp | 44 ++++++++++++++++++- dbms/src/Client/Connection.cpp | 15 +++++-- dbms/src/Interpreters/Context.cpp | 11 +++++ .../Interpreters/InterpreterSelectQuery.cpp | 17 ++++--- 5 files changed, 78 insertions(+), 14 deletions(-) diff --git a/dbms/include/DB/Client/Connection.h b/dbms/include/DB/Client/Connection.h index a2f542eb0d5..648df36b904 100644 --- a/dbms/include/DB/Client/Connection.h +++ b/dbms/include/DB/Client/Connection.h @@ -24,6 +24,7 @@ namespace DB using Poco::SharedPtr; +typedef std::pair ExternalTableData; /** Соединение с сервером БД для использования в клиенте. * Как использовать - см. Core/Protocol.h @@ -83,8 +84,8 @@ public: const Settings * settings = NULL); void sendCancel(); - void sendData(const Block & block); - void sendTemporaryTables(); + void sendData(const Block & block, const String & name = ""); + void sendExternalTables(std::vector & data); /// Проверить, если ли данные, которые можно прочитать. bool poll(size_t timeout_microseconds = 0); diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index 64527a1fac3..ad3fa6bd413 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -65,6 +65,37 @@ public: std::string name; std::string format; std::vector > structure; + ReadBuffer *read_buffer; + Block sample_block; + + void initReadBuffer() + { + /// stdin + if (file == "-") + throw Exception("stdin as file is not supported yet", ErrorCodes::BAD_ARGUMENTS); + read_buffer = new ReadBufferFromFile(file); + } + + void initSampleBlock(const Context &context) + { + for (size_t i = 0; i < structure.size(); ++i) + { + ColumnWithNameAndType column; + column.name = structure[i].first; + column.type = context.getDataTypeFactory().get(structure[i].second); + column.column = column.type->createColumn(); + sample_block.insert(column); + } + } + + ExternalTableData getData(const Context &context) + { + initReadBuffer(); + initSampleBlock(context); + ExternalTableData res = std::make_pair(new AsynchronousBlockInputStream(context.getFormatFactory().getInput( + format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE, context.getDataTypeFactory())), name); + return res; + } void write() { @@ -519,11 +550,20 @@ private: } + void sendExternalTables() + { + std::vector data; + for (size_t i = 0; i < external_tables.size(); ++i) + data.push_back(external_tables[i].getData(context)); + connection->sendExternalTables(data); + } + + /// Обработать запрос, который не требует передачи блоков данных на сервер. void processOrdinaryQuery() { connection->sendQuery(query, "", QueryProcessingStage::Complete); - connection->sendTemporaryTables(); + sendExternalTables(); receiveResult(); } @@ -541,7 +581,7 @@ private: throw Exception("No data to insert", ErrorCodes::NO_DATA_TO_INSERT); connection->sendQuery(query_without_data, "", QueryProcessingStage::Complete); - connection->sendTemporaryTables(); + sendExternalTables(); /// Получим структуру таблицы Block sample = receiveSampleBlock(); diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index f84b9204073..0e9692cff00 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -243,7 +243,7 @@ void Connection::sendCancel() } -void Connection::sendData(const Block & block) +void Connection::sendData(const Block & block, const String & name) { //LOG_TRACE(log, "Sending data (" << getServerAddress() << ")"); @@ -260,7 +260,7 @@ void Connection::sendData(const Block & block) writeVarUInt(Protocol::Client::Data, *out); if (server_revision >= DBMS_MIN_REVISION_WITH_TEMPRORY_TABLES) - writeStringBinary("", *out); + writeStringBinary(name, *out); block.checkNestedArraysOffsets(); block_out->write(block); @@ -268,13 +268,20 @@ void Connection::sendData(const Block & block) out->next(); } - -void Connection::sendTemporaryTables() +void Connection::sendExternalTables(std::vector & data) { /// Если работаем со старым сервером, то никакой информации не отправляем if (server_revision < DBMS_MIN_REVISION_WITH_TEMPRORY_TABLES) return; + for (size_t i = 0; i < data.size(); ++i) + { + data[i].first->readPrefix(); + while(Block block = data[i].first->read()) + sendData(block, data[i].second); + data[i].first->readSuffix(); + } + /// Отправляем пустой блок, символизируя конец передачи данных sendData(Block()); } diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index cb88a2eaf96..43473d8c7e3 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -164,6 +164,7 @@ void Context::assertDatabaseDoesntExist(const String & database_name) const StoragePtr Context::tryGetTemporaryTable(const String & table_name) const { Poco::ScopedLock lock(shared->mutex); + std::cerr << temporary_tables.size() << " " << table_name << std::endl; Tables::const_iterator jt; if (temporary_tables.end() == (jt = temporary_tables.find(table_name))) @@ -206,6 +207,16 @@ StoragePtr Context::tryGetTable(const String & database_name, const String & tab { Poco::ScopedLock lock(shared->mutex); + if (database_name.empty()) + { + StoragePtr res; + if (res = tryGetTemporaryTable(table_name)) + return res; + if (res = session_context->tryGetTemporaryTable(table_name)) + return res; + if (res = global_context->tryGetTemporaryTable(table_name)) + return res; + } String db = database_name.empty() ? current_database : database_name; Databases::const_iterator it; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 009ca1ca9a3..0284f4538d0 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -126,18 +126,23 @@ void InterpreterSelectQuery::getDatabaseAndTableNames(String & database_name, St /** Если таблица не указана - используем таблицу system.one. * Если база данных не указана - используем текущую базу данных. */ + if (query.database) + database_name = dynamic_cast(*query.database).name; + if (query.table) + table_name = dynamic_cast(*query.table).name; + if (!query.table) { database_name = "system"; table_name = "one"; } else if (!query.database) - database_name = context.getCurrentDatabase(); - - if (query.database) - database_name = dynamic_cast(*query.database).name; - if (query.table) - table_name = dynamic_cast(*query.table).name; + { + if (context.tryGetTable("", table_name)) + database_name = ""; + else + database_name = context.getCurrentDatabase(); + } } From 27f081a382e80ed56ec0b86dcd040df36505a53f Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Wed, 12 Mar 2014 17:14:16 +0400 Subject: [PATCH 42/56] dbms: create temporary table query, external tables from stdin [METR-10071] --- dbms/include/DB/Parsers/ASTCreateQuery.h | 5 +-- dbms/src/Client/Client.cpp | 19 ++++++++--- dbms/src/Interpreters/Context.cpp | 11 +++--- .../Interpreters/InterpreterCreateQuery.cpp | 34 ++++++++++++++----- dbms/src/Parsers/ParserCreateQuery.cpp | 9 +++++ 5 files changed, 56 insertions(+), 22 deletions(-) diff --git a/dbms/include/DB/Parsers/ASTCreateQuery.h b/dbms/include/DB/Parsers/ASTCreateQuery.h index bedd7f89d02..7f35bd5ad0a 100644 --- a/dbms/include/DB/Parsers/ASTCreateQuery.h +++ b/dbms/include/DB/Parsers/ASTCreateQuery.h @@ -18,6 +18,7 @@ public: bool is_view; bool is_materialized_view; bool is_populate; + bool is_temporary; String database; String table; ASTPtr columns; @@ -27,8 +28,8 @@ public: String as_table; ASTPtr select; - ASTCreateQuery() : attach(false), if_not_exists(false), is_view(false), is_materialized_view(false), is_populate(false) {} - ASTCreateQuery(StringRange range_) : IAST(range_), attach(false), if_not_exists(false), is_view(false), is_materialized_view(false),is_populate(false) {} + ASTCreateQuery() : attach(false), if_not_exists(false), is_view(false), is_materialized_view(false), is_populate(false), is_temporary(false) {} + ASTCreateQuery(StringRange range_) : IAST(range_), attach(false), if_not_exists(false), is_view(false), is_materialized_view(false), is_populate(false), is_temporary(false) {} /** Получить текст, который идентифицирует этот элемент. */ String getID() const { return (attach ? "AttachQuery_" : "CreateQuery_") + database + "_" + table; }; diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index ad3fa6bd413..ef22e193bf1 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -35,6 +35,7 @@ #include #include #include +#include #include @@ -70,10 +71,10 @@ public: void initReadBuffer() { - /// stdin if (file == "-") - throw Exception("stdin as file is not supported yet", ErrorCodes::BAD_ARGUMENTS); - read_buffer = new ReadBufferFromFile(file); + read_buffer = new ReadBufferFromIStream(std::cin); + else + read_buffer = new ReadBufferFromFile(file); } void initSampleBlock(const Context &context) @@ -552,6 +553,9 @@ private: void sendExternalTables() { + const ASTSelectQuery * select = dynamic_cast(&*parsed_query); + if (!select && !external_tables.empty()) + throw Exception("External tables could be sent only with select query", ErrorCodes::BAD_ARGUMENTS); std::vector data; for (size_t i = 0; i < external_tables.size(); ++i) data.push_back(external_tables[i].getData(context)); @@ -964,6 +968,7 @@ public: boost::program_options::variables_map options; boost::program_options::store(boost::program_options::parse_command_line(positions[1] - positions[0], argv, main_description), options); + size_t stdin_count = 0; for (size_t i = 1; i + 1 < positions.size(); ++i) { boost::program_options::variables_map external_options; @@ -972,12 +977,16 @@ public: try { external_tables.push_back(ExternalTable(external_options)); + if (external_tables.back().file == "-") + stdin_count ++; + if (stdin_count > 1) + throw Exception("Two or more external tables has stdin (-) set as --file field", ErrorCodes::BAD_ARGUMENTS); } catch (const Exception & e) { std::string text = e.displayText(); - std::cerr << "Code: " << e.code() << ". " << text << std::endl << std::endl; - std::cerr << "Table #" << i << std::endl; + std::cerr << "Code: " << e.code() << ". " << text << std::endl; + std::cerr << "Table #" << i << std::endl << std::endl; exit(e.code()); } } diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 43473d8c7e3..50a2e81972a 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -164,7 +164,6 @@ void Context::assertDatabaseDoesntExist(const String & database_name) const StoragePtr Context::tryGetTemporaryTable(const String & table_name) const { Poco::ScopedLock lock(shared->mutex); - std::cerr << temporary_tables.size() << " " << table_name << std::endl; Tables::const_iterator jt; if (temporary_tables.end() == (jt = temporary_tables.find(table_name))) @@ -186,9 +185,9 @@ StoragePtr Context::getTable(const String & database_name, const String & table_ StoragePtr res; if (res = tryGetTemporaryTable(table_name)) return res; - if (res = session_context->tryGetTemporaryTable(table_name)) + if (session_context && (res = session_context->tryGetTemporaryTable(table_name))) return res; - if (res = global_context->tryGetTemporaryTable(table_name)) + if (global_context && (res = global_context->tryGetTemporaryTable(table_name))) return res; } String db = database_name.empty() ? current_database : database_name; @@ -206,15 +205,15 @@ StoragePtr Context::getTable(const String & database_name, const String & table_ StoragePtr Context::tryGetTable(const String & database_name, const String & table_name) const { Poco::ScopedLock lock(shared->mutex); - + if (database_name.empty()) { StoragePtr res; if (res = tryGetTemporaryTable(table_name)) return res; - if (res = session_context->tryGetTemporaryTable(table_name)) + if (session_context && (res = session_context->tryGetTemporaryTable(table_name))) return res; - if (res = global_context->tryGetTemporaryTable(table_name)) + if (global_context && (res = global_context->tryGetTemporaryTable(table_name))) return res; } String db = database_name.empty() ? current_database : database_name; diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 07913765133..9fa064b2631 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -97,14 +97,17 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists) { Poco::ScopedLock lock(context.getMutex()); - context.assertDatabaseExists(database_name); - - if (context.isTableExist(database_name, table_name)) + if (!create.is_temporary) { - if (create.if_not_exists) - return context.getTable(database_name, table_name); - else - throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); + context.assertDatabaseExists(database_name); + + if (context.isTableExist(database_name, table_name)) + { + if (create.if_not_exists) + return context.getTable(database_name, table_name); + else + throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); + } } /// Получаем список столбцов @@ -170,6 +173,13 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists) storage_name = as_storage->getName(); create.storage = dynamic_cast(*context.getCreateQuery(as_database_name, as_table_name)).storage; } + else if (create.is_temporary) + { + storage_name = "Memory"; + ASTFunction * func = new ASTFunction(); + func->name = storage_name; + create.storage = func; + } else if (create.is_view) { storage_name = "View"; @@ -191,7 +201,7 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists) storage_name, data_path, table_name, database_name, context.getGlobalContext(), query_ptr, columns, create.attach); /// Проверка наличия метаданных таблицы на диске и создание метаданных - if (!assume_metadata_exists) + if (!assume_metadata_exists && !create.is_temporary) { if (Poco::File(metadata_path).exists()) { @@ -225,7 +235,13 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists) } } - context.addTable(database_name, table_name, res); + if (create.is_temporary) + { + res->drop_on_destroy = true; + context.getSessionContext().addTemporaryTable(table_name, res); + } + else + context.addTable(database_name, table_name, res); } /// Если запрос CREATE SELECT, то вставим в таблицу данные diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index 23b95fd6f28..19fa637e666 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -168,6 +168,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char ParserWhiteSpaceOrComments ws; ParserString s_create("CREATE", true, true); + ParserString s_temporary("TEMPORARY", true, true); ParserString s_attach("ATTACH", true, true); ParserString s_table("TABLE", true, true); ParserString s_database("DATABASE", true, true); @@ -199,6 +200,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char bool is_view = false; bool is_materialized_view = false; bool is_populate = false; + bool is_temporary = false; ws.ignore(pos, end); @@ -212,6 +214,12 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char ws.ignore(pos, end); + if (s_temporary.ignore(pos, end, expected)) + { + is_temporary = true; + ws.ignore(pos, end); + } + if (s_database.ignore(pos, end, expected)) { ws.ignore(pos, end); @@ -403,6 +411,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, const char query->is_view = is_view; query->is_materialized_view = is_materialized_view; query->is_populate = is_populate; + query->is_temporary = is_temporary; if (database) query->database = dynamic_cast(*database).name; From cf256736062b18b2a3f40b61351f805079f57e5d Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Wed, 12 Mar 2014 18:15:35 +0400 Subject: [PATCH 43/56] expression analyzer: support "IN t" = "in select * from t" [METR-10071] --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 30 ++++++++++++++++++-- 1 file changed, 27 insertions(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 38aaeabd661..793bb377357 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -397,6 +398,9 @@ void ExpressionAnalyzer::normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_as current_asts.insert(ast); replaced = true; } + if (node->name == "in" || node->name == "notIn") + if (ASTIdentifier * right = dynamic_cast(&*node->arguments->children[1])) + right->kind = ASTIdentifier::Table; } else if (ASTIdentifier * node = dynamic_cast(&*ast)) { @@ -508,7 +512,7 @@ void ExpressionAnalyzer::normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_as void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block) { /** Нужно преобразовать правый аргумент в множество. - * Это может быть значение, перечисление значений или подзапрос. + * Это может быть имя таблицы, значение, перечисление значений или подзапрос. * Перечисление значений парсится как функция tuple. */ IAST & args = *node->arguments; @@ -517,7 +521,8 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block) if (dynamic_cast(&*arg)) return; - if (dynamic_cast(&*arg)) + /// Если подзапрос или имя таблицы для селекта + if (dynamic_cast(&*arg) || dynamic_cast(&*arg)) { /// Получаем поток блоков для подзапроса, отдаем его множеству, и кладём это множество на место подзапроса. ASTSet * ast_set = new ASTSet(arg->getColumnName()); @@ -541,7 +546,26 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block) subquery_settings.extremes = 0; subquery_context.setSettings(subquery_settings); - InterpreterSelectQuery interpreter(arg->children[0], subquery_context, QueryProcessingStage::Complete, subquery_depth + 1); + ASTPtr subquery; + if (ASTIdentifier * table = dynamic_cast(&*arg)) + { + ParserSelectQuery parser; + + String query = "SELECT * FROM " + table->name; + const char * begin = query.data(); + const char * end = begin + query.size(); + const char * pos = begin; + const char * expected = ""; + + bool parse_res = parser.parse(pos, end, subquery, expected); + if (!parse_res) + throw Exception("Error in parsing select query while creating set for table " + table->name + ".", + ErrorCodes::LOGICAL_ERROR); + } + else + subquery = arg->children[0]; + + InterpreterSelectQuery interpreter(subquery, subquery_context, QueryProcessingStage::Complete, subquery_depth + 1); ast_set->set = new Set(settings.limits); ast_set->set->setSource(interpreter.execute()); sets_with_subqueries[ast_set->getColumnName()] = ast_set->set; From 386f6d0da5dd677b779b4c43aa7c12b3ada4fcd5 Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Wed, 12 Mar 2014 19:37:44 +0400 Subject: [PATCH 44/56] connection: bug-fix: sending query with pending data [METR-10071] --- dbms/include/DB/Client/Connection.h | 2 +- dbms/src/Client/Client.cpp | 4 ++-- dbms/src/Client/Connection.cpp | 6 +++++- 3 files changed, 8 insertions(+), 4 deletions(-) diff --git a/dbms/include/DB/Client/Connection.h b/dbms/include/DB/Client/Connection.h index 648df36b904..bb130f42b35 100644 --- a/dbms/include/DB/Client/Connection.h +++ b/dbms/include/DB/Client/Connection.h @@ -81,7 +81,7 @@ public: String getServerAddress() const; void sendQuery(const String & query, const String & query_id_ = "", UInt64 stage = QueryProcessingStage::Complete, - const Settings * settings = NULL); + const Settings * settings = NULL, bool with_pending_data = false); void sendCancel(); void sendData(const Block & block, const String & name = ""); diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index ef22e193bf1..9719dc17adf 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -566,7 +566,7 @@ private: /// Обработать запрос, который не требует передачи блоков данных на сервер. void processOrdinaryQuery() { - connection->sendQuery(query, "", QueryProcessingStage::Complete); + connection->sendQuery(query, "", QueryProcessingStage::Complete, NULL, true); sendExternalTables(); receiveResult(); } @@ -584,7 +584,7 @@ private: if ((is_interactive && !parsed_insert_query.data) || (stdin_is_not_tty && std_in.eof())) throw Exception("No data to insert", ErrorCodes::NO_DATA_TO_INSERT); - connection->sendQuery(query_without_data, "", QueryProcessingStage::Complete); + connection->sendQuery(query_without_data, "", QueryProcessingStage::Complete, NULL, true); sendExternalTables(); /// Получим структуру таблицы diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 0e9692cff00..3e5136294aa 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -196,7 +196,7 @@ bool Connection::ping() } -void Connection::sendQuery(const String & query, const String & query_id_, UInt64 stage, const Settings * settings) +void Connection::sendQuery(const String & query, const String & query_id_, UInt64 stage, const Settings * settings, bool with_pending_data) { forceConnected(); @@ -231,6 +231,10 @@ void Connection::sendQuery(const String & query, const String & query_id_, UInt6 maybe_compressed_out = NULL; block_in = NULL; block_out = NULL; + + /// Отправляем пустой блок, символизируя конец передачи данных + if (!with_pending_data) + sendData(Block()); } From 6f85a1269285c9c8b6e709eb65e3270c4b2c7518 Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Thu, 13 Mar 2014 19:00:06 +0400 Subject: [PATCH 45/56] dbms: fixed variable names, temporary tables are now external tables, in distributed query external table now always will be send to remote servers [METR-10071] --- dbms/include/DB/Client/Connection.h | 7 +++- dbms/include/DB/Core/Defines.h | 2 +- .../DB/DataStreams/RemoteBlockInputStream.h | 22 +++++++++-- dbms/include/DB/Interpreters/Context.h | 8 ++-- dbms/include/DB/Storages/IStorage.h | 5 +++ dbms/include/DB/Storages/ITableDeclaration.h | 4 ++ dbms/include/DB/Storages/StorageDistributed.h | 3 ++ dbms/src/Client/Connection.cpp | 12 +++--- dbms/src/Interpreters/Context.cpp | 39 +++++++++++++------ .../Interpreters/InterpreterCreateQuery.cpp | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 2 + dbms/src/Server/TCPHandler.cpp | 18 ++++----- dbms/src/Storages/ITableDeclaration.cpp | 12 +++++- dbms/src/Storages/StorageDistributed.cpp | 4 ++ 14 files changed, 102 insertions(+), 38 deletions(-) diff --git a/dbms/include/DB/Client/Connection.h b/dbms/include/DB/Client/Connection.h index bb130f42b35..5c52a14f141 100644 --- a/dbms/include/DB/Client/Connection.h +++ b/dbms/include/DB/Client/Connection.h @@ -4,8 +4,8 @@ #include -#include #include +#include #include #include #include @@ -24,7 +24,10 @@ namespace DB using Poco::SharedPtr; +/// Поток блоков читающих из таблицы и ее имя typedef std::pair ExternalTableData; +/// Вектор пар, описывающих таблицы +typedef std::vector ExternalTablesData; /** Соединение с сервером БД для использования в клиенте. * Как использовать - см. Core/Protocol.h @@ -85,7 +88,7 @@ public: void sendCancel(); void sendData(const Block & block, const String & name = ""); - void sendExternalTables(std::vector & data); + void sendExternalTables(ExternalTablesData & data); /// Проверить, если ли данные, которые можно прочитать. bool poll(size_t timeout_microseconds = 0); diff --git a/dbms/include/DB/Core/Defines.h b/dbms/include/DB/Core/Defines.h index b4908d5f7a6..a87875c0ee8 100644 --- a/dbms/include/DB/Core/Defines.h +++ b/dbms/include/DB/Core/Defines.h @@ -40,4 +40,4 @@ #define DBMS_MIN_REVISION_WITH_USER_PASSWORD 34482 #define DBMS_MIN_REVISION_WITH_TOTALS_EXTREMES 35265 #define DBMS_MIN_REVISION_WITH_STRING_QUERY_ID 39002 -#define DBMS_MIN_REVISION_WITH_TEMPRORY_TABLES 50092 +#define DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES 50092 diff --git a/dbms/include/DB/DataStreams/RemoteBlockInputStream.h b/dbms/include/DB/DataStreams/RemoteBlockInputStream.h index 587222d6cc5..753685e6607 100644 --- a/dbms/include/DB/DataStreams/RemoteBlockInputStream.h +++ b/dbms/include/DB/DataStreams/RemoteBlockInputStream.h @@ -48,9 +48,10 @@ public: } RemoteBlockInputStream(ConnectionPool::Entry pool_entry_, const String & query_, const Settings * settings_, - const String & _host_column_, const String & _port_column_, QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete) + const String & _host_column_, const String & _port_column_, const Tables & external_tables_, QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete) : pool_entry(pool_entry_), connection(*pool_entry), query(query_), _host_column(_host_column_), - _port_column(_port_column_), stage(stage_), sent_query(false), finished(false), was_cancelled(false), + _port_column(_port_column_), external_tables(external_tables_), stage(stage_), sent_query(false), finished(false), + was_cancelled(false), got_exception_from_server(false), log(&Logger::get("RemoteBlockInputStream (" + connection.getServerAddress() + ")")) { if (settings_) @@ -122,11 +123,25 @@ protected: } } + void sendExternalTables() + { + ExternalTablesData res; + Tables::const_iterator it; + for (it = external_tables.begin(); it != external_tables.end(); it ++) + { + StoragePtr cur = it->second; + QueryProcessingStage::Enum stage = QueryProcessingStage::Complete; + res.push_back(std::make_pair(cur->read(cur->getColumnNamesList(), ASTPtr(), settings, stage, DEFAULT_BLOCK_SIZE, 1)[0], it->first)); + } + connection.sendExternalTables(res); + } + Block readImpl() { if (!sent_query) { - connection.sendQuery(query, "", stage, send_settings ? &settings : NULL); + connection.sendQuery(query, "", stage, send_settings ? &settings : NULL, true); + sendExternalTables(); sent_query = true; } @@ -253,6 +268,7 @@ private: String _host_column; /// Имя столбца, куда записать номер порта (Например "_port"). Пустая строка, если записывать не надо. String _port_column; + Tables external_tables; QueryProcessingStage::Enum stage; /// Отправили запрос (это делается перед получением первого блока). diff --git a/dbms/include/DB/Interpreters/Context.h b/dbms/include/DB/Interpreters/Context.h index 91aadb4c27f..0bf42ab6f3b 100644 --- a/dbms/include/DB/Interpreters/Context.h +++ b/dbms/include/DB/Interpreters/Context.h @@ -48,7 +48,6 @@ typedef std::pair DatabaseAndTableName; typedef std::map > ViewDependencies; typedef std::vector Dependencies; - /** Набор известных объектов, которые могут быть использованы в запросе. * Разделяемая часть. Порядок членов (порядок их уничтожения) очень важен. */ @@ -172,7 +171,7 @@ private: String default_format; /// Формат, используемый, если сервер сам форматирует данные, и если в запросе не задан FORMAT. /// То есть, используется в HTTP-интерфейсе. Может быть не задан - тогда используется некоторый глобальный формат по-умолчанию. - Tables temporary_tables; /// Временные таблицы. + Tables external_tables; /// Временные таблицы. Context * session_context; /// Контекст сессии или NULL, если его нет. (Возможно, равен this.) Context * global_context; /// Глобальный контекст или NULL, если его нет. (Возможно, равен this.) @@ -209,10 +208,11 @@ public: void assertDatabaseExists(const String & database_name) const; void assertDatabaseDoesntExist(const String & database_name) const; - StoragePtr tryGetTemporaryTable(const String & table_name) const; + Tables getExternalTables() const; + StoragePtr tryGetExternalTable(const String & table_name) const; StoragePtr getTable(const String & database_name, const String & table_name) const; StoragePtr tryGetTable(const String & database_name, const String & table_name) const; - void addTemporaryTable(const String & table_name, StoragePtr storage); + void addExternalTable(const String & table_name, StoragePtr storage); void addTable(const String & database_name, const String & table_name, StoragePtr table); void addDatabase(const String & database_name); diff --git a/dbms/include/DB/Storages/IStorage.h b/dbms/include/DB/Storages/IStorage.h index 01d97dde8ad..8b1f50065cf 100644 --- a/dbms/include/DB/Storages/IStorage.h +++ b/dbms/include/DB/Storages/IStorage.h @@ -49,6 +49,11 @@ public: */ virtual bool isRemote() const { return false; } + virtual void storeExternalTables(const std::map & tables_) + { + throw Exception("Method storeExternalTables is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + /** Возвращает true, если хранилище поддерживает запросы с секцией SAMPLE. */ virtual bool supportsSampling() const { return false; } diff --git a/dbms/include/DB/Storages/ITableDeclaration.h b/dbms/include/DB/Storages/ITableDeclaration.h index 1307d1cc5e0..cdc6d1539bd 100644 --- a/dbms/include/DB/Storages/ITableDeclaration.h +++ b/dbms/include/DB/Storages/ITableDeclaration.h @@ -25,6 +25,10 @@ public: */ virtual const NamesAndTypesList & getColumnsList() const = 0; + /** Получить список имён столбцов таблицы, только невиртуальные. + */ + virtual Names getColumnNamesList() const; + /** Получить описание реального (невиртуального) столбца по его имени. */ virtual NameAndTypePair getRealColumn(const String & column_name) const; diff --git a/dbms/include/DB/Storages/StorageDistributed.h b/dbms/include/DB/Storages/StorageDistributed.h index a856c8c0412..443629370b5 100644 --- a/dbms/include/DB/Storages/StorageDistributed.h +++ b/dbms/include/DB/Storages/StorageDistributed.h @@ -49,6 +49,7 @@ public: bool hasColumn(const String &column_name) const; bool isRemote() const { return true; } + void storeExternalTables(const Tables & tables_) { external_tables = tables_; } BlockInputStreams read( const Names & column_names, @@ -92,6 +93,8 @@ private: const Context & context; + Tables external_tables; + /// Используется только, если таблица должна владеть объектом Cluster, которым больше никто не владеет - для реализации TableFunctionRemote. SharedPtr owned_cluster; diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 3e5136294aa..8a57ea33fb6 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -263,7 +263,7 @@ void Connection::sendData(const Block & block, const String & name) writeVarUInt(Protocol::Client::Data, *out); - if (server_revision >= DBMS_MIN_REVISION_WITH_TEMPRORY_TABLES) + if (server_revision >= DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES) writeStringBinary(name, *out); block.checkNestedArraysOffsets(); @@ -272,10 +272,10 @@ void Connection::sendData(const Block & block, const String & name) out->next(); } -void Connection::sendExternalTables(std::vector & data) +void Connection::sendExternalTables(ExternalTablesData & data) { /// Если работаем со старым сервером, то никакой информации не отправляем - if (server_revision < DBMS_MIN_REVISION_WITH_TEMPRORY_TABLES) + if (server_revision < DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES) return; for (size_t i = 0; i < data.size(); ++i) @@ -362,10 +362,10 @@ Block Connection::receiveData() initBlockInput(); - String temporary_table_name; + String external_table_name; - if (server_revision >= DBMS_MIN_REVISION_WITH_TEMPRORY_TABLES) - readStringBinary(temporary_table_name, *in); + if (server_revision >= DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES) + readStringBinary(external_table_name, *in); /// Прочитать из сети один блок return block_in->read(); diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 50a2e81972a..b39263188fa 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -161,12 +161,29 @@ void Context::assertDatabaseDoesntExist(const String & database_name) const } -StoragePtr Context::tryGetTemporaryTable(const String & table_name) const +Tables Context::getExternalTables() const +{ + Tables res = external_tables; + if (session_context && session_context != this) + { + Tables buf = session_context->getExternalTables(); + res.insert(buf.begin(), buf.end()); + } + else if (global_context && global_context != this) + { + Tables buf = global_context->getExternalTables(); + res.insert(buf.begin(), buf.end()); + } + return res; +} + + +StoragePtr Context::tryGetExternalTable(const String & table_name) const { Poco::ScopedLock lock(shared->mutex); Tables::const_iterator jt; - if (temporary_tables.end() == (jt = temporary_tables.find(table_name))) + if (external_tables.end() == (jt = external_tables.find(table_name))) return StoragePtr(); return jt->second; @@ -183,11 +200,11 @@ StoragePtr Context::getTable(const String & database_name, const String & table_ if (database_name.empty()) { StoragePtr res; - if (res = tryGetTemporaryTable(table_name)) + if (res = tryGetExternalTable(table_name)) return res; - if (session_context && (res = session_context->tryGetTemporaryTable(table_name))) + if (session_context && (res = session_context->tryGetExternalTable(table_name))) return res; - if (global_context && (res = global_context->tryGetTemporaryTable(table_name))) + if (global_context && (res = global_context->tryGetExternalTable(table_name))) return res; } String db = database_name.empty() ? current_database : database_name; @@ -209,11 +226,11 @@ StoragePtr Context::tryGetTable(const String & database_name, const String & tab if (database_name.empty()) { StoragePtr res; - if (res = tryGetTemporaryTable(table_name)) + if (res = tryGetExternalTable(table_name)) return res; - if (session_context && (res = session_context->tryGetTemporaryTable(table_name))) + if (session_context && (res = session_context->tryGetExternalTable(table_name))) return res; - if (global_context && (res = global_context->tryGetTemporaryTable(table_name))) + if (global_context && (res = global_context->tryGetExternalTable(table_name))) return res; } String db = database_name.empty() ? current_database : database_name; @@ -230,11 +247,11 @@ StoragePtr Context::tryGetTable(const String & database_name, const String & tab } -void Context::addTemporaryTable(const String & table_name, StoragePtr storage) +void Context::addExternalTable(const String & table_name, StoragePtr storage) { - if (temporary_tables.end() != temporary_tables.find(table_name)) + if (external_tables.end() != external_tables.find(table_name)) throw Exception("Temporary table " + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); - temporary_tables[table_name] = storage; + external_tables[table_name] = storage; } diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 9fa064b2631..6a05b4127e2 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -238,7 +238,7 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists) if (create.is_temporary) { res->drop_on_destroy = true; - context.getSessionContext().addTemporaryTable(table_name, res); + context.getSessionContext().addExternalTable(table_name, res); } else context.addTable(database_name, table_name, res); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 0284f4538d0..ca0f8741d78 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -516,6 +516,8 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(BlockInpu /// Инициализируем изначальные потоки данных, на которые накладываются преобразования запроса. Таблица или подзапрос? if (!interpreter_subquery) { + if (storage->isRemote()) + storage->storeExternalTables(context.getExternalTables()); streams = storage->read(required_columns, query_ptr, settings_for_storage, from_stage, settings.max_block_size, settings.max_threads); for (auto stream : streams) { diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index 7f1234ffe7a..b14dee0605b 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -115,7 +115,7 @@ void TCPHandler::runImpl() continue; /// Получить блоки временных таблиц - if (client_revision >= DBMS_MIN_REVISION_WITH_TEMPRORY_TABLES) + if (client_revision >= DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES) readData(global_settings); /// Обрабатываем Query @@ -544,25 +544,25 @@ bool TCPHandler::receiveData() initBlockInput(); /// Имя временной таблицы для записи данных, по умолчанию пустая строка - String temporary_table_name; - if (client_revision >= DBMS_MIN_REVISION_WITH_TEMPRORY_TABLES) - readStringBinary(temporary_table_name, *in); + String external_table_name; + if (client_revision >= DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES) + readStringBinary(external_table_name, *in); /// Прочитать из сети один блок и записать его Block block = state.block_in->read(); if (block) { /// Если запрос на вставку, то данные нужно писать напрямую в state.io.out. - /// Иначе пишем блоки во временную таблицу temporary_table_name. + /// Иначе пишем блоки во временную таблицу external_table_name. if (!state.is_insert) { StoragePtr storage; /// Если такой таблицы не существовало, создаем ее. - if (!(storage = query_context.tryGetTemporaryTable(temporary_table_name))) + if (!(storage = query_context.tryGetExternalTable(external_table_name))) { NamesAndTypesListPtr columns = new NamesAndTypesList(block.getColumnsList()); - storage = StorageMemory::create(temporary_table_name, columns); - query_context.addTemporaryTable(temporary_table_name, storage); + storage = StorageMemory::create(external_table_name, columns); + query_context.addExternalTable(external_table_name, storage); } /// Данные будем писать напрямую в таблицу. state.io.out = storage->write(ASTPtr()); @@ -650,7 +650,7 @@ void TCPHandler::sendData(Block & block) initBlockOutput(); writeVarUInt(Protocol::Server::Data, *out); - if (client_revision >= DBMS_MIN_REVISION_WITH_TEMPRORY_TABLES) + if (client_revision >= DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES) writeStringBinary("", *out); state.block_out->write(block); diff --git a/dbms/src/Storages/ITableDeclaration.cpp b/dbms/src/Storages/ITableDeclaration.cpp index 3de3049fa62..9145faced2f 100644 --- a/dbms/src/Storages/ITableDeclaration.cpp +++ b/dbms/src/Storages/ITableDeclaration.cpp @@ -10,7 +10,7 @@ namespace DB { -bool ITableDeclaration::hasRealColumn(const String &column_name) const +bool IStorage::hasRealColumn(const String &column_name) const { const NamesAndTypesList & real_columns = getColumnsList(); for (auto & it : real_columns) @@ -20,6 +20,16 @@ bool ITableDeclaration::hasRealColumn(const String &column_name) const } +Names ITableDeclaration::getColumnNamesList() const +{ + const NamesAndTypesList & real_columns = getColumnsList(); + Names res; + for (auto & it : real_columns) + res.push_back(it.first); + return res; +} + + NameAndTypePair ITableDeclaration::getRealColumn(const String &column_name) const { const NamesAndTypesList & real_columns = getColumnsList(); diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 51a7646276b..9c79a68e332 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -210,6 +210,7 @@ BlockInputStreams StorageDistributed::read( &new_settings, need_host_column ? _host_column_name : "", need_port_column ? _port_column_name : "", + external_tables, processed_stage); if (processed_stage == QueryProcessingStage::WithMergeableState || columns_to_remove.empty()) @@ -228,6 +229,8 @@ BlockInputStreams StorageDistributed::read( /// Добавляем запросы к локальному ClickHouse DB::Context new_context = context; new_context.setSettings(new_settings); + for (auto & it : external_tables) + new_context.addExternalTable(it.first, it.second); for(size_t i = 0; i < cluster.getLocalNodesNum(); ++i) { @@ -238,6 +241,7 @@ BlockInputStreams StorageDistributed::read( res.push_back(new RemoveColumnsBlockInputStream(interpreter.execute(), columns_to_remove)); } } + external_tables.clear(); return res; } From f0aae9604ab89554d7b4eff62fb531009287ae55 Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Fri, 14 Mar 2014 18:52:48 +0400 Subject: [PATCH 46/56] dbms: Global in function, sexternal tables transfers to remote servers [METR-10071] --- .../DB/Functions/FunctionsMiscellaneous.h | 8 +- .../DB/Interpreters/ExpressionAnalyzer.h | 11 +- dbms/src/Functions/FunctionFactory.cpp | 6 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 104 +++++++++++++++++- .../Interpreters/InterpreterSelectQuery.cpp | 5 + dbms/src/Parsers/ExpressionListParsers.cpp | 2 + dbms/src/Server/TCPHandler.cpp | 2 - 7 files changed, 129 insertions(+), 9 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsMiscellaneous.h b/dbms/include/DB/Functions/FunctionsMiscellaneous.h index 0fcfc72bd76..438819c2d5e 100644 --- a/dbms/include/DB/Functions/FunctionsMiscellaneous.h +++ b/dbms/include/DB/Functions/FunctionsMiscellaneous.h @@ -285,14 +285,18 @@ class FunctionIn : public IFunction { private: bool negative; + bool global; public: - FunctionIn(bool negative_ = false) : negative(negative_) {} + FunctionIn(bool negative_ = false, bool global_ = false) : negative(negative_), global(global_) {} /// Получить имя функции. String getName() const { - return negative ? "notIn" : "in"; + if (global) + return negative ? "globalNotIn" : "globalIn"; + else + return negative ? "notIn" : "in"; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. diff --git a/dbms/include/DB/Interpreters/ExpressionAnalyzer.h b/dbms/include/DB/Interpreters/ExpressionAnalyzer.h index a16810e0a8f..0923bd6776d 100644 --- a/dbms/include/DB/Interpreters/ExpressionAnalyzer.h +++ b/dbms/include/DB/Interpreters/ExpressionAnalyzer.h @@ -64,6 +64,8 @@ public: * chain.finalize(); */ + void processGlobalOperations(); + /// До агрегации: bool appendArrayJoin(ExpressionActionsChain & chain); bool appendWhere(ExpressionActionsChain & chain); @@ -95,6 +97,8 @@ public: /// Если ast - запрос SELECT, получает имена (алиасы) и типы столбцов из секции SELECT. Block getSelectSampleBlock(); + std::vector external_tables; + private: typedef std::set NamesSet; @@ -124,6 +128,8 @@ private: AggregateDescriptions aggregate_descriptions; std::map sets_with_subqueries; + + std::vector global_nodes; typedef std::map Aliases; Aliases aliases; @@ -257,10 +263,13 @@ private: */ void normalizeTree(); void normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOfASTs & current_asts, std::string current_alias, bool in_sign_rewritten); + + void findGlobalFunctions(ASTPtr & ast); /// Превратить перечисление значений или подзапрос в ASTSet. node - функция in или notIn. void makeSet(ASTFunction * node, const Block & sample_block); - + void addExternalStorage(ASTFunction * node, size_t & name_id); + void getArrayJoinedColumns(); void getArrayJoinedColumnsImpl(ASTPtr ast); void addMultipleArrayJoinAction(ExpressionActions & actions); diff --git a/dbms/src/Functions/FunctionFactory.cpp b/dbms/src/Functions/FunctionFactory.cpp index eb48b6efa45..1ed0f4cabd7 100644 --- a/dbms/src/Functions/FunctionFactory.cpp +++ b/dbms/src/Functions/FunctionFactory.cpp @@ -191,8 +191,10 @@ FunctionPtr FunctionFactory::get( else if (name == "tuple") return new FunctionTuple; else if (name == "tupleElement") return new FunctionTupleElement; - else if (name == "in") return new FunctionIn; - else if (name == "notIn") return new FunctionIn(true); + else if (name == "in") return new FunctionIn(false, false); + else if (name == "notIn") return new FunctionIn(true, false); + else if (name == "globalIn") return new FunctionIn(false, true); + else if (name == "globalNotIn") return new FunctionIn(true, true); else if (name == "array") return new FunctionArray; else if (name == "arrayElement") return new FunctionArrayElement; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 793bb377357..8a96268adfb 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -23,6 +23,11 @@ #include #include +#include + +#include + +#include namespace DB @@ -398,7 +403,7 @@ void ExpressionAnalyzer::normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_as current_asts.insert(ast); replaced = true; } - if (node->name == "in" || node->name == "notIn") + if (node->name == "in" || node->name == "notIn" || node->name == "globalIn" || node->name == "globalNotIn") if (ASTIdentifier * right = dynamic_cast(&*node->arguments->children[1])) right->kind = ASTIdentifier::Table; } @@ -509,6 +514,87 @@ void ExpressionAnalyzer::normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_as } +void ExpressionAnalyzer::findGlobalFunctions(ASTPtr & ast) +{ + /// Рекурсивные вызовы. Не опускаемся в подзапросы. + for (ASTs::iterator it = ast->children.begin(); it != ast->children.end(); ++it) + if (!dynamic_cast(&**it)) + findGlobalFunctions(*it); + + if (ASTFunction * node = dynamic_cast(&*ast)) + { + if (node->name == "globalIn" || node->name == "globalNotIn") + { + global_nodes.push_back(ast); + } + } +} + + +void ExpressionAnalyzer::addExternalStorage(ASTFunction * node, size_t & name_id) +{ + IAST & args = *node->arguments; + ASTPtr & arg = args.children[1]; + StoragePtr external_storage = StoragePtr(); + + /// Если подзапрос или имя таблицы для селекта + if (dynamic_cast(&*arg) || dynamic_cast(&*arg)) + { + /** Для подзапроса в секции IN не действуют ограничения на максимальный размер результата. + * Так как результат этого поздапроса - ещё не результат всего запроса. + * Вместо этого работают ограничения max_rows_in_set, max_bytes_in_set, set_overflow_mode. + */ + Context subquery_context = context; + Settings subquery_settings = context.getSettings(); + subquery_settings.limits.max_result_rows = 0; + subquery_settings.limits.max_result_bytes = 0; + /// Вычисление extremes не имеет смысла и не нужно (если его делать, то в результате всего запроса могут взяться extremes подзапроса). + subquery_settings.extremes = 0; + subquery_context.setSettings(subquery_settings); + + ASTPtr subquery; + if (ASTIdentifier * table = dynamic_cast(&*arg)) + { + ParserSelectQuery parser; + + if (context.tryGetExternalTable(table->name)) + return; + + String query = "SELECT * FROM " + table->name; + const char * begin = query.data(); + const char * end = begin + query.size(); + const char * pos = begin; + const char * expected = ""; + + bool parse_res = parser.parse(pos, end, subquery, expected); + if (!parse_res) + throw Exception("Error in parsing select query while creating set for table " + table->name + ".", + ErrorCodes::LOGICAL_ERROR); + } + else + subquery = arg->children[0]; + + InterpreterSelectQuery interpreter(subquery, subquery_context, QueryProcessingStage::Complete, subquery_depth + 1); + + Block sample = interpreter.getSampleBlock(); + NamesAndTypesListPtr columns = new NamesAndTypesList(sample.getColumnsList()); + + String external_table_name = "_table" + toString(name_id++); + external_storage = StorageMemory::create(external_table_name, columns); + BlockOutputStreamPtr output = external_storage->write(ASTPtr()); + copyData(*interpreter.execute(), *output); + + ASTIdentifier * ast_ident = new ASTIdentifier(); + ast_ident->kind = ASTIdentifier::Table; + ast_ident->name = external_storage->getTableName(); + arg = ast_ident; + external_tables.push_back(external_storage); + } + else + throw Exception("Global in (notIn) supports only select data.", ErrorCodes::BAD_ARGUMENTS); +} + + void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block) { /** Нужно преобразовать правый аргумент в множество. @@ -566,6 +652,7 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block) subquery = arg->children[0]; InterpreterSelectQuery interpreter(subquery, subquery_context, QueryProcessingStage::Complete, subquery_depth + 1); + ast_set->set = new Set(settings.limits); ast_set->set->setSource(interpreter.execute()); sets_with_subqueries[ast_set->getColumnName()] = ast_set->set; @@ -797,7 +884,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl if (node->kind == ASTFunction::FUNCTION) { - if (node->name == "in" || node->name == "notIn") + if (node->name == "in" || node->name == "notIn" || node->name == "globalIn" || node->name == "globalNotIn") { if (!no_subqueries) { @@ -1092,6 +1179,19 @@ void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActions & actions) actions.add(ExpressionActions::Action::arrayJoin(result_columns)); } +void ExpressionAnalyzer::processGlobalOperations() +{ + findGlobalFunctions(ast); + + size_t id = 1; + for (size_t i = 0; i < global_nodes.size(); ++i) + { + String external_table_name = "_data"; + while (context.tryGetExternalTable(external_table_name + toString(id))) + id ++; + addExternalStorage(dynamic_cast(&*global_nodes[i]), id); + } +} bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain) { diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index ca0f8741d78..b410896ce4c 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -78,6 +78,11 @@ void InterpreterSelectQuery::init(BlockInputStreamPtr input_, const NamesAndType query_analyzer = new ExpressionAnalyzer(query_ptr, context, storage, subquery_depth); + query_analyzer->processGlobalOperations(); + + for (auto & it : query_analyzer->external_tables) + context.addExternalTable(it->getTableName(), it); + if (input_) streams.push_back(input_); } diff --git a/dbms/src/Parsers/ExpressionListParsers.cpp b/dbms/src/Parsers/ExpressionListParsers.cpp index ee71f89a92b..60b78c56846 100644 --- a/dbms/src/Parsers/ExpressionListParsers.cpp +++ b/dbms/src/Parsers/ExpressionListParsers.cpp @@ -47,6 +47,8 @@ const char * ParserComparisonExpression::operators[] = "NOT LIKE", "notLike", "IN", "in", "NOT IN", "notIn", + "GLOBAL IN", "globalIn", + "GLOBAL NOT IN","globalNotIn", nullptr }; diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index b14dee0605b..a67d0fe8c36 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -225,8 +225,6 @@ void TCPHandler::readData(const Settings & global_settings) if (Daemon::instance().isCancelled() || in->eof()) return; - std::cerr << "Receiving packet" << std::endl; - if (!receivePacket()) break; } From 003c7b30cdf118e4b079134273c4762929bb7274 Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Fri, 14 Mar 2014 19:42:30 +0400 Subject: [PATCH 47/56] dbms: fixes and more comments on 'Global in' and external tables behavior [METR-10071] --- dbms/include/DB/Client/Connection.h | 5 ++++- .../DB/DataStreams/RemoteBlockInputStream.h | 4 +++- .../DB/Interpreters/ExpressionAnalyzer.h | 10 +++++----- dbms/include/DB/Storages/StorageDistributed.h | 3 +++ dbms/src/Client/Client.cpp | 17 ++++++++++++----- dbms/src/Client/Connection.cpp | 2 +- dbms/src/Interpreters/Context.cpp | 2 ++ dbms/src/Interpreters/ExpressionAnalyzer.cpp | 8 +++++--- .../src/Interpreters/InterpreterSelectQuery.cpp | 2 ++ 9 files changed, 37 insertions(+), 16 deletions(-) diff --git a/dbms/include/DB/Client/Connection.h b/dbms/include/DB/Client/Connection.h index 5c52a14f141..d5ab6afe10a 100644 --- a/dbms/include/DB/Client/Connection.h +++ b/dbms/include/DB/Client/Connection.h @@ -83,12 +83,15 @@ public: /// Адрес сервера - для сообщений в логе и в эксепшенах. String getServerAddress() const; + /// Если последний флаг true, то затем необходимо вызвать sendExternalTablesData void sendQuery(const String & query, const String & query_id_ = "", UInt64 stage = QueryProcessingStage::Complete, const Settings * settings = NULL, bool with_pending_data = false); void sendCancel(); + /// Отправить блок данных, на сервере сохранить во временную таблицу name void sendData(const Block & block, const String & name = ""); - void sendExternalTables(ExternalTablesData & data); + /// Отправить все содержимое внешних таблиц + void sendExternalTablesData(ExternalTablesData & data); /// Проверить, если ли данные, которые можно прочитать. bool poll(size_t timeout_microseconds = 0); diff --git a/dbms/include/DB/DataStreams/RemoteBlockInputStream.h b/dbms/include/DB/DataStreams/RemoteBlockInputStream.h index 753685e6607..e1a816acc82 100644 --- a/dbms/include/DB/DataStreams/RemoteBlockInputStream.h +++ b/dbms/include/DB/DataStreams/RemoteBlockInputStream.h @@ -123,6 +123,7 @@ protected: } } + /// Отправить на удаленные сервера все временные таблицы void sendExternalTables() { ExternalTablesData res; @@ -133,7 +134,7 @@ protected: QueryProcessingStage::Enum stage = QueryProcessingStage::Complete; res.push_back(std::make_pair(cur->read(cur->getColumnNamesList(), ASTPtr(), settings, stage, DEFAULT_BLOCK_SIZE, 1)[0], it->first)); } - connection.sendExternalTables(res); + connection.sendExternalTablesData(res); } Block readImpl() @@ -268,6 +269,7 @@ private: String _host_column; /// Имя столбца, куда записать номер порта (Например "_port"). Пустая строка, если записывать не надо. String _port_column; + /// Временные таблицы, которые необходимо переслать на удаленные сервера. Tables external_tables; QueryProcessingStage::Enum stage; diff --git a/dbms/include/DB/Interpreters/ExpressionAnalyzer.h b/dbms/include/DB/Interpreters/ExpressionAnalyzer.h index 0923bd6776d..41008e12c75 100644 --- a/dbms/include/DB/Interpreters/ExpressionAnalyzer.h +++ b/dbms/include/DB/Interpreters/ExpressionAnalyzer.h @@ -97,8 +97,8 @@ public: /// Если ast - запрос SELECT, получает имена (алиасы) и типы столбцов из секции SELECT. Block getSelectSampleBlock(); + /// Все новые временные таблицы, полученные при выполнении подзапросов Global In std::vector external_tables; - private: typedef std::set NamesSet; @@ -128,9 +128,6 @@ private: AggregateDescriptions aggregate_descriptions; std::map sets_with_subqueries; - - std::vector global_nodes; - typedef std::map Aliases; Aliases aliases; @@ -264,10 +261,13 @@ private: void normalizeTree(); void normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_asts, SetOfASTs & current_asts, std::string current_alias, bool in_sign_rewritten); - void findGlobalFunctions(ASTPtr & ast); + /// Обходит запрос и сохраняет найденные глобальные функции (например Global in) + void findGlobalFunctions(ASTPtr & ast, std::vector & global_nodes); /// Превратить перечисление значений или подзапрос в ASTSet. node - функция in или notIn. void makeSet(ASTFunction * node, const Block & sample_block); + /// Выполнить подзапрос в секции global in и запомнить результат во временную таблицу типа memory + /// Все новые временные таблицы хранятся в переменной external_tables void addExternalStorage(ASTFunction * node, size_t & name_id); void getArrayJoinedColumns(); diff --git a/dbms/include/DB/Storages/StorageDistributed.h b/dbms/include/DB/Storages/StorageDistributed.h index 443629370b5..5640bb532f0 100644 --- a/dbms/include/DB/Storages/StorageDistributed.h +++ b/dbms/include/DB/Storages/StorageDistributed.h @@ -49,6 +49,7 @@ public: bool hasColumn(const String &column_name) const; bool isRemote() const { return true; } + /// Сохранить временные таблицы, чтобы при следующем вызове метода read переслать их на удаленные сервера void storeExternalTables(const Tables & tables_) { external_tables = tables_; } BlockInputStreams read( @@ -93,6 +94,8 @@ private: const Context & context; + /// Временные таблицы, которые необходимо отправить на сервер. Переменная очищается после каждого вызова метода read + /// Для подготовки к отправке нужно использовтаь метод storeExternalTables Tables external_tables; /// Используется только, если таблица должна владеть объектом Cluster, которым больше никто не владеет - для реализации TableFunctionRemote. diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index 9719dc17adf..7081141ad4f 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -58,14 +58,17 @@ namespace DB using Poco::SharedPtr; - +/// Описание внешней таблицы class ExternalTable { public: - std::string file; - std::string name; - std::string format; + std::string file; /// Файл с данными или '-' если stdin + std::string name; /// Имя таблицы + std::string format; /// Название формата хранения данных + + /// Описание структуры таблицы: (имя столбца, имя типа данных) std::vector > structure; + ReadBuffer *read_buffer; Block sample_block; @@ -98,6 +101,7 @@ public: return res; } + /// Функция для отладочного вывода информации void write() { std::cerr << "file " << file << std::endl; @@ -108,6 +112,7 @@ public: std::cerr << "\t" << structure[i].first << " " << structure[i].second << std::endl; } + /// Извлечение параметров из variables_map, которая строится по командной строке ExternalTable(const boost::program_options::variables_map & external_options) { if (external_options.count("file")) @@ -551,15 +556,17 @@ private: } + /// Преобразовать внешние таблицы к ExternalTableData и переслать через connection void sendExternalTables() { const ASTSelectQuery * select = dynamic_cast(&*parsed_query); if (!select && !external_tables.empty()) throw Exception("External tables could be sent only with select query", ErrorCodes::BAD_ARGUMENTS); + std::vector data; for (size_t i = 0; i < external_tables.size(); ++i) data.push_back(external_tables[i].getData(context)); - connection->sendExternalTables(data); + connection->sendExternalTablesData(data); } diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 8a57ea33fb6..f7e5af1db9d 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -272,7 +272,7 @@ void Connection::sendData(const Block & block, const String & name) out->next(); } -void Connection::sendExternalTables(ExternalTablesData & data) +void Connection::sendExternalTablesData(ExternalTablesData & data) { /// Если работаем со старым сервером, то никакой информации не отправляем if (server_revision < DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES) diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index b39263188fa..67aa10b9f62 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -163,6 +163,8 @@ void Context::assertDatabaseDoesntExist(const String & database_name) const Tables Context::getExternalTables() const { + Poco::ScopedLock lock(shared->mutex); + Tables res = external_tables; if (session_context && session_context != this) { diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 8a96268adfb..32b4543048f 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -403,6 +403,7 @@ void ExpressionAnalyzer::normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_as current_asts.insert(ast); replaced = true; } + /// может быть указано in t, где t - таблица, что равносильно select * from t. if (node->name == "in" || node->name == "notIn" || node->name == "globalIn" || node->name == "globalNotIn") if (ASTIdentifier * right = dynamic_cast(&*node->arguments->children[1])) right->kind = ASTIdentifier::Table; @@ -514,12 +515,12 @@ void ExpressionAnalyzer::normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_as } -void ExpressionAnalyzer::findGlobalFunctions(ASTPtr & ast) +void ExpressionAnalyzer::findGlobalFunctions(ASTPtr & ast, std::vector & global_nodes) { /// Рекурсивные вызовы. Не опускаемся в подзапросы. for (ASTs::iterator it = ast->children.begin(); it != ast->children.end(); ++it) if (!dynamic_cast(&**it)) - findGlobalFunctions(*it); + findGlobalFunctions(*it, global_nodes); if (ASTFunction * node = dynamic_cast(&*ast)) { @@ -1181,7 +1182,8 @@ void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActions & actions) void ExpressionAnalyzer::processGlobalOperations() { - findGlobalFunctions(ast); + std::vector global_nodes; + findGlobalFunctions(ast, global_nodes); size_t id = 1; for (size_t i = 0; i < global_nodes.size(); ++i) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index b410896ce4c..78caf91142b 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -78,8 +78,10 @@ void InterpreterSelectQuery::init(BlockInputStreamPtr input_, const NamesAndType query_analyzer = new ExpressionAnalyzer(query_ptr, context, storage, subquery_depth); + /// Выполняем все Global in подзапросы, результаты будут сохранены в query_analyzer->external_tables query_analyzer->processGlobalOperations(); + /// Сохраняем в query context новые временные таблицы for (auto & it : query_analyzer->external_tables) context.addExternalTable(it->getTableName(), it); From 9ecf7b3f795647c02f5983e29b7b359ed9a61693 Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Wed, 19 Mar 2014 16:52:14 +0400 Subject: [PATCH 48/56] defines: fixed DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES to correct value [METR-10071] --- dbms/include/DB/Core/Defines.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/include/DB/Core/Defines.h b/dbms/include/DB/Core/Defines.h index a87875c0ee8..2212b58811d 100644 --- a/dbms/include/DB/Core/Defines.h +++ b/dbms/include/DB/Core/Defines.h @@ -40,4 +40,4 @@ #define DBMS_MIN_REVISION_WITH_USER_PASSWORD 34482 #define DBMS_MIN_REVISION_WITH_TOTALS_EXTREMES 35265 #define DBMS_MIN_REVISION_WITH_STRING_QUERY_ID 39002 -#define DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES 50092 +#define DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES 50237 From c9c2f671f23be856284dd4803bb873b154fc843a Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Wed, 19 Mar 2014 17:03:29 +0400 Subject: [PATCH 49/56] connection: fixed bug with sending extra block to old-versioned server [METR-10071] --- dbms/src/Client/Connection.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index f7e5af1db9d..2887af93e72 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -232,8 +232,8 @@ void Connection::sendQuery(const String & query, const String & query_id_, UInt6 block_in = NULL; block_out = NULL; - /// Отправляем пустой блок, символизируя конец передачи данных - if (!with_pending_data) + /// Если версия сервера достаточно новая и стоит флаг, отправляем пустой блок, символизируя конец передачи данных. + if (server_revision > DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES && !with_pending_data) sendData(Block()); } From a4d499086000e45fc2b64f5eab1a88e53d7d65d4 Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Wed, 19 Mar 2014 19:07:29 +0400 Subject: [PATCH 50/56] dbms: fixed problem with sending empty tables, few more fixes [METR-10071] --- dbms/include/DB/DataStreams/RemoteBlockInputStream.h | 6 +++++- dbms/include/DB/Storages/StorageMemory.h | 2 ++ dbms/src/Server/TCPHandler.cpp | 5 +++-- dbms/src/Storages/StorageDistributed.cpp | 5 +++-- 4 files changed, 13 insertions(+), 5 deletions(-) diff --git a/dbms/include/DB/DataStreams/RemoteBlockInputStream.h b/dbms/include/DB/DataStreams/RemoteBlockInputStream.h index e1a816acc82..5828d2f64f6 100644 --- a/dbms/include/DB/DataStreams/RemoteBlockInputStream.h +++ b/dbms/include/DB/DataStreams/RemoteBlockInputStream.h @@ -132,7 +132,11 @@ protected: { StoragePtr cur = it->second; QueryProcessingStage::Enum stage = QueryProcessingStage::Complete; - res.push_back(std::make_pair(cur->read(cur->getColumnNamesList(), ASTPtr(), settings, stage, DEFAULT_BLOCK_SIZE, 1)[0], it->first)); + DB::BlockInputStreams input = cur->read(cur->getColumnNamesList(), ASTPtr(), settings, stage, DEFAULT_BLOCK_SIZE, 1); + if (input.size() == 0) + res.push_back(std::make_pair(new OneBlockInputStream(cur->getSampleBlock()), it->first)); + else + res.push_back(std::make_pair(input[0], it->first)); } connection.sendExternalTablesData(res); } diff --git a/dbms/include/DB/Storages/StorageMemory.h b/dbms/include/DB/Storages/StorageMemory.h index e6ab8d97b4b..10f46a4bec1 100644 --- a/dbms/include/DB/Storages/StorageMemory.h +++ b/dbms/include/DB/Storages/StorageMemory.h @@ -69,6 +69,8 @@ public: const NamesAndTypesList & getColumnsList() const { return *columns; } + size_t getSize() const { return data.size(); } + BlockInputStreams read( const Names & column_names, ASTPtr query, diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index a67d0fe8c36..3a566864985 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -548,7 +548,7 @@ bool TCPHandler::receiveData() /// Прочитать из сети один блок и записать его Block block = state.block_in->read(); - if (block) + if (block || !external_table_name.empty()) { /// Если запрос на вставку, то данные нужно писать напрямую в state.io.out. /// Иначе пишем блоки во временную таблицу external_table_name. @@ -565,7 +565,8 @@ bool TCPHandler::receiveData() /// Данные будем писать напрямую в таблицу. state.io.out = storage->write(ASTPtr()); } - state.io.out->write(block); + if (block) + state.io.out->write(block); return true; } else diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 9c79a68e332..d60c2f80b64 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -219,7 +219,7 @@ BlockInputStreams StorageDistributed::read( res.push_back(new RemoveColumnsBlockInputStream(temp, columns_to_remove)); } - if (all_inclusive || values.find(std::make_pair("localhost", clickhouse_port)) != values.end()) + if (cluster.getLocalNodesNum() > 0 && (all_inclusive || values.find(std::make_pair("localhost", clickhouse_port)) != values.end())) { ASTPtr modified_query_ast = remakeQuery( query, @@ -230,7 +230,8 @@ BlockInputStreams StorageDistributed::read( DB::Context new_context = context; new_context.setSettings(new_settings); for (auto & it : external_tables) - new_context.addExternalTable(it.first, it.second); + if (!new_context.tryGetExternalTable(it.first)) + new_context.addExternalTable(it.first, it.second); for(size_t i = 0; i < cluster.getLocalNodesNum(); ++i) { From eda2a1590bd8fec37081c215b760e50238dd7768 Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Wed, 19 Mar 2014 19:50:04 +0400 Subject: [PATCH 51/56] dbms: fixed different names of new external storages in expression analyzer [METR-10071] --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 32b4543048f..1b1f0774af5 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -580,7 +580,7 @@ void ExpressionAnalyzer::addExternalStorage(ASTFunction * node, size_t & name_id Block sample = interpreter.getSampleBlock(); NamesAndTypesListPtr columns = new NamesAndTypesList(sample.getColumnsList()); - String external_table_name = "_table" + toString(name_id++); + String external_table_name = "_data" + toString(name_id++); external_storage = StorageMemory::create(external_table_name, columns); BlockOutputStreamPtr output = external_storage->write(ASTPtr()); copyData(*interpreter.execute(), *output); From 0898ccfeb4cb8a4b41ca4cd972b28d4ab3412d35 Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Wed, 26 Mar 2014 17:06:10 +0400 Subject: [PATCH 52/56] dbms: fixed merging errors [METR-10071] --- dbms/src/Interpreters/InterpreterCreateQuery.cpp | 2 +- dbms/src/Storages/ITableDeclaration.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 6a05b4127e2..ee2cd8d004a 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -237,7 +237,7 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists) if (create.is_temporary) { - res->drop_on_destroy = true; + res->is_dropped = true; context.getSessionContext().addExternalTable(table_name, res); } else diff --git a/dbms/src/Storages/ITableDeclaration.cpp b/dbms/src/Storages/ITableDeclaration.cpp index 9145faced2f..abf21a3ccbf 100644 --- a/dbms/src/Storages/ITableDeclaration.cpp +++ b/dbms/src/Storages/ITableDeclaration.cpp @@ -10,7 +10,7 @@ namespace DB { -bool IStorage::hasRealColumn(const String &column_name) const +bool ITableDeclaration::hasRealColumn(const String &column_name) const { const NamesAndTypesList & real_columns = getColumnsList(); for (auto & it : real_columns) From 5237f8b6ca9318516723af4668c5679c7e7ee40e Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Thu, 27 Mar 2014 15:30:08 +0400 Subject: [PATCH 53/56] dbms: fixed cancelling input stream in tcphandler [METR-10071] --- dbms/include/DB/Core/Defines.h | 2 +- dbms/src/Client/Connection.cpp | 2 +- dbms/src/Server/TCPHandler.cpp | 7 ++++++- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/dbms/include/DB/Core/Defines.h b/dbms/include/DB/Core/Defines.h index 2212b58811d..025bb85e44c 100644 --- a/dbms/include/DB/Core/Defines.h +++ b/dbms/include/DB/Core/Defines.h @@ -40,4 +40,4 @@ #define DBMS_MIN_REVISION_WITH_USER_PASSWORD 34482 #define DBMS_MIN_REVISION_WITH_TOTALS_EXTREMES 35265 #define DBMS_MIN_REVISION_WITH_STRING_QUERY_ID 39002 -#define DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES 50237 +#define DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES 50263 diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 2887af93e72..df4b933769e 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -233,7 +233,7 @@ void Connection::sendQuery(const String & query, const String & query_id_, UInt6 block_out = NULL; /// Если версия сервера достаточно новая и стоит флаг, отправляем пустой блок, символизируя конец передачи данных. - if (server_revision > DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES && !with_pending_data) + if (server_revision >= DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES && !with_pending_data) sendData(Block()); } diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index 3a566864985..06e4c2d910d 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -118,6 +118,10 @@ void TCPHandler::runImpl() if (client_revision >= DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES) readData(global_settings); + /// Пересоздаем, поскольку получая данные внешних таблиц, мы получили пустой блок. + /// Из-за этого весь stream помечен как cancelled + state.block_in = BlockInputStreamPtr(); + /// Обрабатываем Query state.io = executeQuery(state.query, query_context, false, state.stage); @@ -548,7 +552,8 @@ bool TCPHandler::receiveData() /// Прочитать из сети один блок и записать его Block block = state.block_in->read(); - if (block || !external_table_name.empty()) + + if (block) { /// Если запрос на вставку, то данные нужно писать напрямую в state.io.out. /// Иначе пишем блоки во временную таблицу external_table_name. From 6a3043e74978e171acce2a8b1d4e7c56c4b74f69 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 27 Mar 2014 16:32:37 +0400 Subject: [PATCH 54/56] Deduplicated code for merge tree writing; previous commit is useless. [#METR-10202] --- .../Storages/MergeTree/MergeTreeDataWriter.h | 10 +- .../MergeTree/MergedBlockOutputStream.h | 95 +++--------- .../MergeTree/MergeTreeDataMerger.cpp | 12 +- .../MergeTree/MergeTreeDataWriter.cpp | 138 +----------------- 4 files changed, 40 insertions(+), 215 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeDataWriter.h b/dbms/include/DB/Storages/MergeTree/MergeTreeDataWriter.h index 2220456ac33..ebeb19f374f 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeDataWriter.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeDataWriter.h @@ -31,7 +31,7 @@ typedef std::list BlocksWithDateIntervals; class MergeTreeDataWriter { public: - MergeTreeDataWriter(MergeTreeData & data_) : data(data_), log(&Logger::get("MergeTreeDataWriter")), flags(O_TRUNC | O_CREAT | O_WRONLY) {} + MergeTreeDataWriter(MergeTreeData & data_) : data(data_), log(&Logger::get("MergeTreeDataWriter")) {} /** Разбивает блок на блоки, каждый из которых нужно записать в отдельный кусок. * (читай: разбивает строки по месяцам) @@ -49,14 +49,6 @@ private: MergeTreeData & data; Logger * log; - - const int flags; - - typedef std::set OffsetColumns; - - /// Записать данные одного столбца. - void writeData(const String & path, const String & name, const IDataType & type, const IColumn & column, - OffsetColumns & offset_columns, MergeTreeData::DataPart::Checksums & checksums, size_t level = 0); }; } diff --git a/dbms/include/DB/Storages/MergeTree/MergedBlockOutputStream.h b/dbms/include/DB/Storages/MergeTree/MergedBlockOutputStream.h index 6a745460985..a243813778e 100644 --- a/dbms/include/DB/Storages/MergeTree/MergedBlockOutputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergedBlockOutputStream.h @@ -4,6 +4,9 @@ #include #include +#include +#include +#include namespace DB @@ -20,9 +23,9 @@ protected: struct ColumnStream { ColumnStream(const String & data_path, const std::string & marks_path) : - plain(data_path, DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY), - compressed(plain), - marks(marks_path, 4096, O_TRUNC | O_CREAT | O_WRONLY) {} + plain(data_path, DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY), + compressed(plain), + marks(marks_path, 4096, O_TRUNC | O_CREAT | O_WRONLY) {} WriteBufferFromFile plain; CompressedWriteBuffer compressed; @@ -66,19 +69,6 @@ protected: addStream(path, name, *type_arr->getNestedType(), level + 1); } - else if (const DataTypeNested * type_nested = dynamic_cast(&type)) - { - String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level); - String escaped_size_name = escaped_column_name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level); - - column_streams[size_name] = new ColumnStream( - path + escaped_size_name + ".bin", - path + escaped_size_name + ".mrk"); - - const NamesAndTypesList & columns = *type_nested->getNestedTypesList(); - for (NamesAndTypesList::const_iterator it = columns.begin(); it != columns.end(); ++it) - addStream(path, DataTypeNested::concatenateNestedName(name, it->first), *it->second, level + 1); - } else column_streams[name] = new ColumnStream( path + escaped_column_name + ".bin", @@ -126,34 +116,6 @@ protected: } } } - else if (const DataTypeNested * type_nested = dynamic_cast(&type)) - { - String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level); - - ColumnStream & stream = *column_streams[size_name]; - - size_t prev_mark = 0; - while (prev_mark < size) - { - size_t limit = 0; - - /// Если есть index_offset, то первая засечка идёт не сразу, а после этого количества строк. - if (prev_mark == 0 && index_offset != 0) - { - limit = index_offset; - } - else - { - limit = storage.index_granularity; - writeIntBinary(stream.plain.count(), stream.marks); - writeIntBinary(stream.compressed.offset(), stream.marks); - } - - type_nested->serializeOffsets(column, stream.compressed, prev_mark, limit); - stream.compressed.nextIfAtEnd(); /// Чтобы вместо засечек, указывающих на конец сжатого блока, были засечки, указывающие на начало следующего. - prev_mark += limit; - } - } { ColumnStream & stream = *column_streams[name]; @@ -190,30 +152,21 @@ protected: size_t index_offset; }; -/** Для записи куска, полученного слиянием нескольких других. - * Данные уже отсортированы, относятся к одному месяцу, и пишутся в один кускок. +/** Для записи одного куска. Данные уже отсортированы, относятся к одному месяцу, и пишутся в один кускок. */ class MergedBlockOutputStream : public IMergedBlockOutputStream { public: - MergedBlockOutputStream(MergeTreeData & storage_, - UInt16 min_date, UInt16 max_date, UInt64 min_part_id, UInt64 max_part_id, UInt32 level) - : IMergedBlockOutputStream(storage_), marks_count(0) + MergedBlockOutputStream(MergeTreeData & storage_, String part_path_, const NamesAndTypesList & columns_list_) + : IMergedBlockOutputStream(storage_), columns_list(columns_list_), part_path(part_path_), marks_count(0) { - part_name = storage.getPartName( - DayNum_t(min_date), DayNum_t(max_date), - min_part_id, max_part_id, level); + Poco::File(part_path).createDirectories(); - part_tmp_path = storage.getFullPath() + "tmp_" + part_name + "/"; - part_res_path = storage.getFullPath() + part_name + "/"; - - Poco::File(part_tmp_path).createDirectories(); - - index_stream = new WriteBufferFromFile(part_tmp_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY); + index_stream = new WriteBufferFromFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY); columns_list = storage.getColumnsList(); for (const auto & it : columns_list) - addStream(part_tmp_path, it.first, *it.second); + addStream(part_path, it.first, *it.second); } void write(const Block & block) @@ -234,7 +187,8 @@ public: { for (PrimaryColumns::const_iterator it = primary_columns.begin(); it != primary_columns.end(); ++it) { - (*it)->type->serializeBinary((*(*it)->column)[i], *index_stream); + index_vec.push_back((*(*it)->column)[i]); + (*it)->type->serializeBinary(index_vec.back(), *index_stream); } ++marks_count; @@ -268,15 +222,13 @@ public: if (marks_count == 0) { /// Кусок пустой - все записи удалились. - Poco::File(part_tmp_path).remove(true); + Poco::File(part_path).remove(true); } - else - { - /// Переименовываем кусок. - Poco::File(part_tmp_path).renameTo(part_res_path); + } - /// А добавление нового куска в набор (и удаление исходных кусков) сделает вызывающая сторона. - } + MergeTreeData::DataPart::Index & getIndex() + { + return index_vec; } /// Сколько засечек уже записано. @@ -287,13 +239,12 @@ public: private: NamesAndTypesList columns_list; + String part_path; - String part_name; - String part_tmp_path; - String part_res_path; size_t marks_count; - + SharedPtr index_stream; + MergeTreeData::DataPart::Index index_vec; }; typedef Poco::SharedPtr MergedBlockOutputStreamPtr; @@ -315,7 +266,7 @@ public: for (size_t i = 0; i < block.columns(); ++i) { addStream(part_path, block.getByPosition(i).name, - *block.getByPosition(i).type, 0, prefix + block.getByPosition(i).name); + *block.getByPosition(i).type, 0, prefix + block.getByPosition(i).name); } initialized = true; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp index 160b4dc2b95..f10faef52ca 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp @@ -292,8 +292,10 @@ String MergeTreeDataMerger::mergeParts(const MergeTreeData::DataPartsVector & pa throw Exception("Unknown mode of operation for MergeTreeData: " + toString(data.mode), ErrorCodes::LOGICAL_ERROR); } - MergedBlockOutputStreamPtr to = new MergedBlockOutputStream(data, - new_data_part->left_date, new_data_part->right_date, new_data_part->left, new_data_part->right, new_data_part->level); + String new_part_tmp_path = data.getFullPath() + "tmp_" + new_data_part->name + "/"; + String new_part_res_path = data.getFullPath() + new_data_part->name + "/"; + + MergedBlockOutputStreamPtr to = new MergedBlockOutputStream(data, new_part_tmp_path, data.getColumnsList()); merged_stream->readPrefix(); to->writePrefix(); @@ -311,6 +313,8 @@ String MergeTreeDataMerger::mergeParts(const MergeTreeData::DataPartsVector & pa merged_stream->readSuffix(); to->writeSuffix(); + new_data_part->index.swap(to->getIndex()); + /// В обычном режиме строчки не могут удалиться при мердже. if (0 == to->marksCount() && data.mode == MergeTreeData::Ordinary) throw Exception("Empty part after merge", ErrorCodes::LOGICAL_ERROR); @@ -324,8 +328,8 @@ String MergeTreeDataMerger::mergeParts(const MergeTreeData::DataPartsVector & pa return ""; } - /// NOTE Только что записанный индекс заново считывается с диска. Можно было бы формировать его сразу при записи. - new_data_part->loadIndex(); + /// Переименовываем кусок. + Poco::File(new_part_tmp_path).renameTo(new_part_res_path); /// Добавляем новый кусок в набор. data.replaceParts(parts, new_data_part); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp index e2ebfaab235..c1e6051e7bc 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -78,9 +79,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa DateLUTSingleton & date_lut = DateLUTSingleton::instance(); - size_t rows = block.rows(); - size_t columns = block.columns(); - size_t part_size = (rows + data.index_granularity - 1) / data.index_granularity; + size_t part_size = (block.rows() + data.index_granularity - 1) / data.index_granularity; String tmp_part_name = "tmp_" + data.getPartName( DayNum_t(min_date), DayNum_t(max_date), @@ -102,59 +101,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa /// Сортируем. stableSortBlock(block, sort_descr); - /// Наконец-то можно писать данные на диск. - LOG_TRACE(log, "Writing index."); + MergedBlockOutputStream out(data, part_tmp_path, block.getColumnsList()); + out.getIndex().reserve(part_size * sort_descr.size()); - MergeTreeData::DataPart::Checksums checksums; - MergeTreeData::DataPart::Index index_vec; - - /// Сначала пишем индекс. Индекс содержит значение PK для каждой index_granularity строки. - index_vec.reserve(part_size * sort_descr.size()); - - { - WriteBufferFromFile index_file(part_tmp_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, flags); - HashingWriteBuffer index(index_file); - - typedef std::vector PrimaryColumns; - PrimaryColumns primary_columns; - - for (size_t i = 0, size = sort_descr.size(); i < size; ++i) - primary_columns.push_back( - !sort_descr[i].column_name.empty() - ? &block.getByName(sort_descr[i].column_name) - : &block.getByPosition(sort_descr[i].column_number)); - - for (size_t i = 0; i < rows; i += data.index_granularity) - { - for (PrimaryColumns::const_iterator it = primary_columns.begin(); it != primary_columns.end(); ++it) - { - index_vec.push_back((*(*it)->column)[i]); - (*it)->type->serializeBinary(index_vec.back(), index); - } - } - - index.next(); - checksums.file_checksums["primary.idx"].size = index.count(); - checksums.file_checksums["primary.idx"].hash = index.getHash(); - } - - LOG_TRACE(log, "Writing data."); - - /// Множество записанных столбцов со смещениями, чтобы не писать общие для вложенных структур столбцы несколько раз - OffsetColumns offset_columns; - - for (size_t i = 0; i < columns; ++i) - { - const ColumnWithNameAndType & column = block.getByPosition(i); - writeData(part_tmp_path, column.name, *column.type, *column.column, offset_columns, checksums); - } - - /// Запишем файл с чексуммами. - { - WriteBufferFromFile checksums_file(part_tmp_path + "checksums.txt", 1024, flags); - checksums.writeText(checksums_file); - checksums_file.next(); - } + out.writePrefix(); + out.write(block); + out.writeSuffix(); MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared(data); new_data_part->left_date = DayNum_t(min_date); @@ -167,83 +119,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa new_data_part->modification_time = time(0); new_data_part->left_month = date_lut.toFirstDayNumOfMonth(new_data_part->left_date); new_data_part->right_month = date_lut.toFirstDayNumOfMonth(new_data_part->right_date); - new_data_part->index.swap(index_vec); - new_data_part->checksums = checksums; + new_data_part->index.swap(out.getIndex()); return new_data_part; } -void MergeTreeDataWriter::writeData(const String & path, const String & name, const IDataType & type, const IColumn & column, - OffsetColumns & offset_columns, MergeTreeData::DataPart::Checksums & checksums, size_t level) -{ - String escaped_column_name = escapeForFileName(name); - size_t size = column.size(); - - /// Для массивов требуется сначала сериализовать размеры, а потом значения. - if (const DataTypeArray * type_arr = dynamic_cast(&type)) - { - String size_name = escapeForFileName(DataTypeNested::extractNestedTableName(name)) - + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level); - if (offset_columns.count(size_name) == 0) - { - offset_columns.insert(size_name); - - WriteBufferFromFile plain_file(path + size_name + ".bin", DBMS_DEFAULT_BUFFER_SIZE, flags); - WriteBufferFromFile marks_file(path + size_name + ".mrk", 4096, flags); - CompressedWriteBuffer compressed_file(plain_file); - HashingWriteBuffer marks(marks_file); - HashingWriteBuffer compressed(compressed_file); - - size_t prev_mark = 0; - while (prev_mark < size) - { - /// Каждая засечка - это: (смещение в файле до начала сжатого блока, смещение внутри блока) - writeIntBinary(plain_file.count(), marks); - writeIntBinary(compressed.offset(), marks); - - type_arr->serializeOffsets(column, compressed, prev_mark, data.index_granularity); - prev_mark += data.index_granularity; - - compressed.nextIfAtEnd(); /// Чтобы вместо засечек, указывающих на конец сжатого блока, были засечки, указывающие на начало следующего. - } - - compressed.next(); - plain_file.next(); - marks.next(); - checksums.file_checksums[size_name + ".bin"].size = compressed.count(); - checksums.file_checksums[size_name + ".bin"].hash = compressed.getHash(); - checksums.file_checksums[size_name + ".mrk"].size = marks.count(); - checksums.file_checksums[size_name + ".mrk"].hash = marks.getHash(); - } - } - - { - WriteBufferFromFile plain_file(path + escaped_column_name + ".bin", DBMS_DEFAULT_BUFFER_SIZE, flags); - WriteBufferFromFile marks_file(path + escaped_column_name + ".mrk", 4096, flags); - CompressedWriteBuffer compressed_file(plain_file); - HashingWriteBuffer marks(marks_file); - HashingWriteBuffer compressed(compressed_file); - - size_t prev_mark = 0; - while (prev_mark < size) - { - writeIntBinary(plain_file.count(), marks); - writeIntBinary(compressed.offset(), marks); - - type.serializeBinary(column, compressed, prev_mark, data.index_granularity); - prev_mark += data.index_granularity; - - compressed.nextIfAtEnd(); /// Чтобы вместо засечек, указывающих на конец сжатого блока, были засечки, указывающие на начало следующего. - } - - compressed.next(); - plain_file.next(); - marks.next(); - checksums.file_checksums[escaped_column_name + ".bin"].size = compressed.count(); - checksums.file_checksums[escaped_column_name + ".bin"].hash = compressed.getHash(); - checksums.file_checksums[escaped_column_name + ".mrk"].size = marks.count(); - checksums.file_checksums[escaped_column_name + ".mrk"].hash = marks.getHash(); - } -} - } From e735568723972b9fff49dd9122dbde707c84c154 Mon Sep 17 00:00:00 2001 From: Sergey Fedorov Date: Thu, 27 Mar 2014 16:48:09 +0400 Subject: [PATCH 55/56] dbms: aggregate function array [METR-10366] --- .../AggregateFunctionArray.h | 120 ++++++++++++++++++ .../AggregateFunctionFactory.cpp | 22 +++- 2 files changed, 141 insertions(+), 1 deletion(-) create mode 100644 dbms/include/DB/AggregateFunctions/AggregateFunctionArray.h diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionArray.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionArray.h new file mode 100644 index 00000000000..6e40bd22306 --- /dev/null +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionArray.h @@ -0,0 +1,120 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + + +/** Не агрегатная функция, а адаптер агрегатных функций, + * который любую агрегатную функцию agg(x) делает агрегатной функцией вида aggArray(x). + * Адаптированная агрегатная функция вычисляет вложенную агрегатную функцию для каждого элемента массива. + */ +class AggregateFunctionArray : public IAggregateFunction +{ +private: + AggregateFunctionPtr nested_func_owner; + IAggregateFunction * nested_func; + int num_agruments; + +public: + AggregateFunctionArray(AggregateFunctionPtr nested_) : nested_func_owner(nested_), nested_func(nested_func_owner.get()) {} + + String getName() const + { + return nested_func->getName() + "Array"; + } + + DataTypePtr getReturnType() const + { + return nested_func->getReturnType(); + } + + void setArguments(const DataTypes & arguments) + { + num_agruments = arguments.size(); + + DataTypes nested_arguments; + for (int i = 0; i < num_agruments; ++i) + { + if (const DataTypeArray * array = dynamic_cast(&*arguments[i])) + nested_arguments.push_back(array->getNestedType()); + else + throw Exception("Illegal type " + arguments[i]->getName() + " of argument #" + toString(i + 1) + " for aggregate function " + getName() + ". Must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + nested_func->setArguments(nested_arguments); + } + + void setParameters(const Array & params) + { + nested_func->setParameters(params); + } + + void create(AggregateDataPtr place) const + { + nested_func->create(place); + } + + void destroy(AggregateDataPtr place) const + { + nested_func->destroy(place); + } + + bool hasTrivialDestructor() const + { + return nested_func->hasTrivialDestructor(); + } + + size_t sizeOfData() const + { + return nested_func->sizeOfData(); + } + + size_t alignOfData() const + { + return nested_func->alignOfData(); + } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const + { + const IColumn ** nested = new const IColumn*[num_agruments]; + std::vector column_ptrs; + for (int i = 0; i < num_agruments; ++i) + { + ColumnPtr single_value_column = dynamic_cast(*columns[i]).cut(row_num, 1); + column_ptrs.push_back(single_value_column); + nested[i] = dynamic_cast(*single_value_column).getDataPtr().get(); + } + for (int i = 0; i < num_agruments; ++i) + if (nested[i]->size() != nested[0]->size()) + throw Exception("All arrays must be of the same size. Aggregate function " + getName(), ErrorCodes::BAD_ARGUMENTS); + for (size_t i = 0; i < nested[0]->size(); ++i) + nested_func->add(place, nested, i); + } + + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const + { + nested_func->merge(place, rhs); + } + + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const + { + nested_func->serialize(place, buf); + } + + void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const + { + nested_func->deserializeMerge(place, buf); + } + + void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const + { + nested_func->insertResultInto(place, to); + } +}; + +} diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index f7dc1c31bb5..9efa8b65fad 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include @@ -317,9 +318,25 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da /// Для агрегатных функций вида aggIf, где agg - имя другой агрегатной функции. DataTypes nested_dt = argument_types; nested_dt.pop_back(); - AggregateFunctionPtr nested = get(String(name.data(), name.size() - 2), nested_dt); + AggregateFunctionPtr nested = get(String(name.data(), name.size() - 2), nested_dt, recursion_level + 1); return new AggregateFunctionIf(nested); } + else if (recursion_level <= 1 && name.size() >= 6 && name.substr(name.size()-5) == "Array") + { + /// Для агрегатных функций вида aggArray, где agg - имя другой агрегатной функции. + size_t num_agruments = argument_types.size(); + + DataTypes nested_arguments; + for (size_t i = 0; i < num_agruments; ++i) + { + if (const DataTypeArray * array = dynamic_cast(&*argument_types[i])) + nested_arguments.push_back(array->getNestedType()); + else + throw Exception("Illegal type " + argument_types[i]->getName() + " of argument #" + toString(i + 1) + " for aggregate function " + name + ". Must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + AggregateFunctionPtr nested = get(String(name.data(), name.size() - 5), nested_arguments, recursion_level + 1); + return new AggregateFunctionArray(nested); + } else throw Exception("Unknown aggregate function " + name, ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION); } @@ -369,6 +386,9 @@ bool AggregateFunctionFactory::isAggregateFunctionName(const String & name, int /// Для агрегатных функций вида aggIf, где agg - имя другой агрегатной функции. if (recursion_level == 0 && name.size() >= 3 && name[name.size() - 2] == 'I' && name[name.size() - 1] == 'f') return isAggregateFunctionName(String(name.data(), name.size() - 2), 1); + /// Для агрегатных функций вида aggArray, где agg - имя другой агрегатной функции. + if (recursion_level <= 1 && name.size() >= 6 && name.substr(name.size()-5) == "Array") + return isAggregateFunctionName(String(name.data(), name.size() - 5), 1); return false; } From a8f4ed62788d03ffb146db1e66b793cfb8d1faf2 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 27 Mar 2014 21:30:04 +0400 Subject: [PATCH 56/56] Merge --- .../DB/Storages/MergeTree/MergeTreeData.h | 7 +- .../MergeTree/MergedBlockOutputStream.h | 90 +++++++++++++---- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 96 ++++++++++++------- .../MergeTree/MergeTreeDataMerger.cpp | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 3 +- 5 files changed, 140 insertions(+), 58 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeData.h b/dbms/include/DB/Storages/MergeTree/MergeTreeData.h index dc7af53d186..616d7392697 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeData.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeData.h @@ -112,7 +112,7 @@ public: }; typedef std::map FileChecksums; - FileChecksums file_checksums; + FileChecksums files; /// Проверяет, что множество столбцов и их контрольные суммы совпадают. Если нет - бросает исключение. void check(const Checksums & rhs) const; @@ -120,6 +120,11 @@ public: /// Сериализует и десериализует в человекочитаемом виде. void readText(ReadBuffer & in); void writeText(WriteBuffer & out) const; + + bool empty() const + { + return files.empty(); + } }; DataPart(MergeTreeData & storage_) : storage(storage_), size_in_bytes(0) {} diff --git a/dbms/include/DB/Storages/MergeTree/MergedBlockOutputStream.h b/dbms/include/DB/Storages/MergeTree/MergedBlockOutputStream.h index a243813778e..50756c2c2c5 100644 --- a/dbms/include/DB/Storages/MergeTree/MergedBlockOutputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergedBlockOutputStream.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -22,26 +23,41 @@ protected: typedef std::set OffsetColumns; struct ColumnStream { - ColumnStream(const String & data_path, const std::string & marks_path) : - plain(data_path, DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY), - compressed(plain), - marks(marks_path, 4096, O_TRUNC | O_CREAT | O_WRONLY) {} + ColumnStream(const String & escaped_column_name_, const String & data_path, const std::string & marks_path) : + escaped_column_name(escaped_column_name_), + plain_file(data_path, DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY), + compressed_buf(plain_file), + marks_file(marks_path, 4096, O_TRUNC | O_CREAT | O_WRONLY), + compressed(compressed_buf), marks(marks_file) {} - WriteBufferFromFile plain; - CompressedWriteBuffer compressed; - WriteBufferFromFile marks; + String escaped_column_name; + WriteBufferFromFile plain_file; + CompressedWriteBuffer compressed_buf; + WriteBufferFromFile marks_file; + HashingWriteBuffer compressed; + HashingWriteBuffer marks; void finalize() { compressed.next(); - plain.next(); + plain_file.next(); marks.next(); } void sync() { - plain.sync(); - marks.sync(); + plain_file.sync(); + marks_file.sync(); + } + + void addToChecksums(MergeTreeData::DataPart::Checksums & checksums, String name = "") + { + if (name == "") + name = escaped_column_name; + checksums.files[name + ".bin"].size = compressed.count(); + checksums.files[name + ".bin"].hash = compressed.getHash(); + checksums.files[name + ".mrk"].size = marks.count(); + checksums.files[name + ".mrk"].hash = marks.getHash(); } }; @@ -64,6 +80,7 @@ protected: + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level); column_streams[size_name] = new ColumnStream( + escaped_size_name, path + escaped_size_name + ".bin", path + escaped_size_name + ".mrk"); @@ -71,6 +88,7 @@ protected: } else column_streams[name] = new ColumnStream( + escaped_column_name, path + escaped_column_name + ".bin", path + escaped_column_name + ".mrk"); } @@ -106,7 +124,7 @@ protected: else { limit = storage.index_granularity; - writeIntBinary(stream.plain.count(), stream.marks); + writeIntBinary(stream.plain_file.count(), stream.marks); writeIntBinary(stream.compressed.offset(), stream.marks); } @@ -133,7 +151,7 @@ protected: else { limit = storage.index_granularity; - writeIntBinary(stream.plain.count(), stream.marks); + writeIntBinary(stream.plain_file.count(), stream.marks); writeIntBinary(stream.compressed.offset(), stream.marks); } @@ -162,7 +180,8 @@ public: { Poco::File(part_path).createDirectories(); - index_stream = new WriteBufferFromFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY); + index_file_stream = new WriteBufferFromFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY); + index_stream = new HashingWriteBuffer(*index_file_stream); columns_list = storage.getColumnsList(); for (const auto & it : columns_list) @@ -207,23 +226,44 @@ public: size_t written_for_last_mark = (storage.index_granularity - index_offset + rows) % storage.index_granularity; index_offset = (storage.index_granularity - written_for_last_mark) % storage.index_granularity; } - - void writeSuffix() + + void writeSuffix() override { - /// Заканчиваем запись. + throw Exception("Method writeSuffix is not supported by MergedBlockOutputStream", ErrorCodes::NOT_IMPLEMENTED); + } + + MergeTreeData::DataPart::Checksums writeSuffixAndGetChecksums() + { + /// Заканчиваем запись и достаем чексуммы. + MergeTreeData::DataPart::Checksums checksums; + index_stream->next(); - index_stream = NULL; + checksums.files["primary.idx"].size = index_stream->count(); + checksums.files["primary.idx"].hash = index_stream->getHash(); for (ColumnStreams::iterator it = column_streams.begin(); it != column_streams.end(); ++it) + { it->second->finalize(); + it->second->addToChecksums(checksums); + } + index_stream = NULL; column_streams.clear(); if (marks_count == 0) { /// Кусок пустой - все записи удалились. Poco::File(part_path).remove(true); + checksums.files.clear(); } + else + { + /// Записываем файл с чексуммами. + WriteBufferFromFile out(part_path + "checksums.txt", 1024); + checksums.writeText(out); + } + + return checksums; } MergeTreeData::DataPart::Index & getIndex() @@ -243,7 +283,8 @@ private: size_t marks_count; - SharedPtr index_stream; + SharedPtr index_file_stream; + SharedPtr index_stream; MergeTreeData::DataPart::Index index_vec; }; @@ -284,19 +325,30 @@ public: index_offset = (storage.index_granularity - written_for_last_mark) % storage.index_granularity; } - void writeSuffix() + void writeSuffix() override { + throw Exception("Method writeSuffix is not supported by MergedColumnOnlyOutputStream", ErrorCodes::NOT_IMPLEMENTED); + } + + MergeTreeData::DataPart::Checksums writeSuffixAndGetChecksums() + { + MergeTreeData::DataPart::Checksums checksums; + for (auto & column_stream : column_streams) { column_stream.second->finalize(); if (sync) column_stream.second->sync(); std::string column = escapeForFileName(column_stream.first); + column_stream.second->addToChecksums(checksums, column); Poco::File(part_path + prefix + column + ".bin").renameTo(part_path + column + ".bin"); Poco::File(part_path + prefix + column + ".mrk").renameTo(part_path + column + ".mrk"); } + column_streams.clear(); initialized = false; + + return checksums; } private: diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 4199283126f..9c8fc38fc44 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -9,6 +9,7 @@ #include #include #include +#include @@ -366,26 +367,15 @@ void MergeTreeData::removeColumnFiles(String column_name) void MergeTreeData::createConvertExpression(const String & in_column_name, const String & out_type, ExpressionActionsPtr & out_expression, String & out_column) { - ASTFunction * function = new ASTFunction; - ASTPtr function_ptr = function; + Names out_names; + out_expression = new ExpressionActions( + NamesAndTypesList(1, NameAndTypePair(in_column_name, getDataTypeByName(in_column_name))), context.getSettingsRef()); - ASTExpressionList * arguments = new ASTExpressionList; - ASTPtr arguments_ptr = arguments; + FunctionPtr function = context.getFunctionFactory().get("to" + out_type, context); + out_expression->add(ExpressionActions::Action::applyFunction(function, Names(1, in_column_name)), out_names); + out_expression->add(ExpressionActions::Action::removeColumn(in_column_name)); - function->name = "to" + out_type; - function->arguments = arguments_ptr; - function->children.push_back(arguments_ptr); - - ASTIdentifier * in_column = new ASTIdentifier; - ASTPtr in_column_ptr = in_column; - - arguments->children.push_back(in_column_ptr); - - in_column->name = in_column_name; - in_column->kind = ASTIdentifier::Column; - - out_expression = ExpressionAnalyzer(function_ptr, context, *columns).getActions(false); - out_column = function->getColumnName(); + out_column = out_names[0]; } static DataTypePtr getDataTypeByName(const String & name, const NamesAndTypesList & columns) @@ -445,18 +435,29 @@ void MergeTreeData::prepareAlterModify(const ASTAlterQuery::Parameters & params) ExpressionBlockInputStream in(new MergeTreeBlockInputStream(full_path + part->name + '/', DEFAULT_MERGE_BLOCK_SIZE, column_name, *this, part, ranges, false, NULL, ""), expr); MergedColumnOnlyOutputStream out(*this, full_path + part->name + '/', true); + in.readPrefix(); out.writePrefix(); try { while(DB::Block b = in.read()) - { - /// оставляем только столбец с результатом - b.erase(0); out.write(b); + + in.readSuffix(); + DataPart::Checksums add_checksums = out.writeSuffixAndGetChecksums(); + + /// Запишем обновленные контрольные суммы во временный файл. + if (!part->checksums.empty()) + { + DataPart::Checksums new_checksums = part->checksums; + std::string escaped_name = escapeForFileName(name_type.name); + std::string escaped_out_column = escapeForFileName(out_column); + new_checksums.files[escaped_name + ".bin"] = add_checksums.files[escaped_out_column + ".bin"]; + new_checksums.files[escaped_name + ".mrk"] = add_checksums.files[escaped_out_column + ".mrk"]; + + WriteBufferFromFile checksums_file(full_path + part->name + '/' + escaped_out_column + ".checksums.txt", 1024); + new_checksums.writeText(checksums_file); } - LOG_TRACE(log, "Write Suffix"); - out.writeSuffix(); } catch (const Exception & e) { @@ -486,40 +487,63 @@ void MergeTreeData::commitAlterModify(const ASTAlterQuery::Parameters & params) /// переименовываем старые столбцы, добавляя расширение .old for (DataPartPtr & part : parts) { - std::string path = full_path + part->name + '/' + escapeForFileName(name_type.name); + std::string part_path = full_path + part->name + '/'; + std::string path = part_path + escapeForFileName(name_type.name); if (Poco::File(path + ".bin").exists()) { LOG_TRACE(log, "Renaming " << path + ".bin" << " to " << path + ".bin" + ".old"); Poco::File(path + ".bin").renameTo(path + ".bin" + ".old"); LOG_TRACE(log, "Renaming " << path + ".mrk" << " to " << path + ".mrk" + ".old"); Poco::File(path + ".mrk").renameTo(path + ".mrk" + ".old"); + + if (Poco::File(part_path + "checksums.txt").exists()) + { + LOG_TRACE(log, "Renaming " << part_path + "checksums.txt" << " to " << part_path + "checksums.txt" + ".old"); + Poco::File(part_path + "checksums.txt").renameTo(part_path + "checksums.txt" + ".old"); + } } } /// переименовываем временные столбцы for (DataPartPtr & part : parts) { - std::string path = full_path + part->name + '/' + escapeForFileName(out_column); - std::string new_path = full_path + part->name + '/' + escapeForFileName(name_type.name); + std::string part_path = full_path + part->name + '/'; + std::string name = escapeForFileName(out_column); + std::string new_name = escapeForFileName(name_type.name); + std::string path = part_path + name; + std::string new_path = part_path + new_name; if (Poco::File(path + ".bin").exists()) { LOG_TRACE(log, "Renaming " << path + ".bin" << " to " << new_path + ".bin"); Poco::File(path + ".bin").renameTo(new_path + ".bin"); LOG_TRACE(log, "Renaming " << path + ".mrk" << " to " << new_path + ".mrk"); Poco::File(path + ".mrk").renameTo(new_path + ".mrk"); + + if (Poco::File(path + ".checksums.txt").exists()) + { + LOG_TRACE(log, "Renaming " << path + ".checksums.txt" << " to " << part_path + ".checksums.txt"); + Poco::File(path + ".checksums.txt").renameTo(part_path + "checksums.txt"); + } } } // удаляем старые столбцы for (DataPartPtr & part : parts) { - std::string path = full_path + part->name + '/' + escapeForFileName(name_type.name); + std::string part_path = full_path + part->name + '/'; + std::string path = part_path + escapeForFileName(name_type.name); if (Poco::File(path + ".bin" + ".old").exists()) { LOG_TRACE(log, "Removing old column " << path + ".bin" + ".old"); Poco::File(path + ".bin" + ".old").remove(); LOG_TRACE(log, "Removing old column " << path + ".mrk" + ".old"); Poco::File(path + ".mrk" + ".old").remove(); + + if (Poco::File(part_path + "checksums.txt" + ".old").exists()) + { + LOG_TRACE(log, "Removing old checksums " << part_path + "checksums.txt" + ".old"); + Poco::File(part_path + "checksums.txt" + ".old").remove(); + } } } @@ -691,20 +715,20 @@ MergeTreeData::DataParts MergeTreeData::getDataParts() void MergeTreeData::DataPart::Checksums::check(const Checksums & rhs) const { - for (const auto & it : rhs.file_checksums) + for (const auto & it : rhs.files) { const String & name = it.first; - if (!file_checksums.count(name)) + if (!files.count(name)) throw Exception("Unexpected file " + name + " in data part", ErrorCodes::UNEXPECTED_FILE_IN_DATA_PART); } - for (const auto & it : file_checksums) + for (const auto & it : files) { const String & name = it.first; - auto jt = rhs.file_checksums.find(name); - if (jt == rhs.file_checksums.end()) + auto jt = rhs.files.find(name); + if (jt == rhs.files.end()) throw Exception("No file " + name + " in data part", ErrorCodes::NO_FILE_IN_DATA_PART); const Checksum & expected = it.second; @@ -720,7 +744,7 @@ void MergeTreeData::DataPart::Checksums::check(const Checksums & rhs) const void MergeTreeData::DataPart::Checksums::readText(ReadBuffer & in) { - file_checksums.clear(); + files.clear(); size_t count; DB::assertString("checksums format version: 1\n", in); @@ -741,17 +765,17 @@ void MergeTreeData::DataPart::Checksums::readText(ReadBuffer & in) DB::readText(sum.hash.second, in); DB::assertString("\n", in); - file_checksums.insert(std::make_pair(name, sum)); + files.insert(std::make_pair(name, sum)); } } void MergeTreeData::DataPart::Checksums::writeText(WriteBuffer & out) const { DB::writeString("checksums format version: 1\n", out); - DB::writeText(file_checksums.size(), out); + DB::writeText(files.size(), out); DB::writeString(" files:\n", out); - for (const auto & it : file_checksums) + for (const auto & it : files) { DB::writeString(it.first, out); DB::writeString("\n\tsize: ", out); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp index f10faef52ca..9477d64c280 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp @@ -311,7 +311,7 @@ String MergeTreeDataMerger::mergeParts(const MergeTreeData::DataPartsVector & pa } merged_stream->readSuffix(); - to->writeSuffix(); + new_data_part->checksums = to->writeSuffixAndGetChecksums(); new_data_part->index.swap(to->getIndex()); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp index c1e6051e7bc..6d6cbbe6a12 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -106,7 +106,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa out.writePrefix(); out.write(block); - out.writeSuffix(); + MergeTreeData::DataPart::Checksums checksums = out.writeSuffixAndGetChecksums(); MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared(data); new_data_part->left_date = DayNum_t(min_date); @@ -120,6 +120,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa new_data_part->left_month = date_lut.toFirstDayNumOfMonth(new_data_part->left_date); new_data_part->right_month = date_lut.toFirstDayNumOfMonth(new_data_part->right_date); new_data_part->index.swap(out.getIndex()); + new_data_part->checksums = checksums; return new_data_part; }