diff --git a/docs/en/sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md b/docs/en/sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md index 1d3b78635d041d7a1e3cbf3eb9bb62ef54ed1f82..ca8aef24ea1bd94021cc269fa8c53884783ab864 100644 --- a/docs/en/sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md +++ b/docs/en/sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md @@ -35,6 +35,25 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration The source is configured in the `source` section. +For source types +[Local file](#dicts-external_dicts_dict_sources-local_file), +[Executable file](#dicts-external_dicts_dict_sources-executable), +[HTTP(s)](#dicts-external_dicts_dict_sources-http), +[ClickHouse](#dicts-external_dicts_dict_sources-clickhouse) +optional settings are available: + +``` xml + + + /opt/dictionaries/os.tsv + TabSeparated + + + 0 + + +``` + Types of sources (`source_type`): - [Local file](#dicts-external_dicts_dict_sources-local_file) diff --git a/docs/ru/sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md b/docs/ru/sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md index 32115e703f4653178dfee6c9d92ec89d4e4a74d4..102a7cc0e7f275bd50dd78f32a0bc635504a2ed0 100644 --- a/docs/ru/sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md +++ b/docs/ru/sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md @@ -30,6 +30,25 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration Источник настраивается в разделе `source`. +Для типов источников +[Локальный файл](#dicts-external_dicts_dict_sources-local_file), +[Исполняемый файл](#dicts-external_dicts_dict_sources-executable), +[HTTP(s)](#dicts-external_dicts_dict_sources-http), +[ClickHouse](#dicts-external_dicts_dict_sources-clickhouse) +доступны дополнительные настройки: + +``` xml + + + /opt/dictionaries/os.tsv + TabSeparated + + + 0 + + +``` + Типы источников (`source_type`): - [Локальный файл](#dicts-external_dicts_dict_sources-local_file) diff --git a/src/Common/SettingsChanges.h b/src/Common/SettingsChanges.h index 2e037a50a17300f16ffb15fba13253d8e73ff7b4..004a08c3b4b62b805d4a45a3611947dec1f9cfd3 100644 --- a/src/Common/SettingsChanges.h +++ b/src/Common/SettingsChanges.h @@ -10,6 +10,11 @@ struct SettingChange { String name; Field value; + SettingChange() {} + + SettingChange(const String & name_, const Field value_) + : name(name_) + , value(value_) {} friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value); } friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index ad08754e4e77ad9ca5f654ff75768c0ed675f22f..98ed7985acb431c7881ede99b4ef637c9718f588 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -13,7 +13,7 @@ #include "readInvalidateQuery.h" #include "writeParenthesisedString.h" #include "DictionaryFactory.h" - +#include "DictionarySourceHelpers.h" namespace DB { @@ -50,6 +50,7 @@ static ConnectionPoolWithFailoverPtr createPool( ClickHouseDictionarySource::ClickHouseDictionarySource( const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config, + const std::string & path_to_settings, const std::string & config_prefix, const Block & sample_block_, const Context & context_) @@ -74,8 +75,11 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( { /// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication). context.setUser(user, password, Poco::Net::SocketAddress("127.0.0.1", 0), {}); + context = copyContextAndApplySettings(path_to_settings, context, config); + /// Processors are not supported here yet. context.setSetting("experimental_use_processors", false); + /// Query context is needed because some code in executeQuery function may assume it exists. /// Current example is Context::getSampleBlockCache from InterpreterSelectWithUnionQuery::getSampleBlock. context.makeQueryContext(); @@ -227,7 +231,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) const Context & context, bool /* check_config */) -> DictionarySourcePtr { - return std::make_unique(dict_struct, config, config_prefix + ".clickhouse", sample_block, context); + return std::make_unique(dict_struct, config, config_prefix, config_prefix + ".clickhouse", sample_block, context); }; factory.registerSource("clickhouse", create_table_source); } diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index 84bbd78b93ad553e89b538f9f4cc90cbc1f59e23..13dc0323039d8770bc7c6302ce0f5b61296203a1 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -21,6 +21,7 @@ public: ClickHouseDictionarySource( const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config, + const std::string & path_to_settings, const std::string & config_prefix, const Block & sample_block_, const Context & context); diff --git a/src/Dictionaries/DictionarySourceFactory.cpp b/src/Dictionaries/DictionarySourceFactory.cpp index fa3b3017ad2acc842e24e5627bae00cbe9607eab..a9e90024745e3bbe41f403b565d50e7dd8cecb60 100644 --- a/src/Dictionaries/DictionarySourceFactory.cpp +++ b/src/Dictionaries/DictionarySourceFactory.cpp @@ -84,11 +84,12 @@ DictionarySourcePtr DictionarySourceFactory::create( { Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix, keys); - if (keys.size() != 1) - throw Exception{name + ": element dictionary.source should have exactly one child element", + + if (keys.empty() || keys.size() > 2) + throw Exception{name + ": element dictionary.source should have one or two child elements", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG}; - const auto & source_type = keys.front(); + const std::string & source_type = keys.front() == "settings" ? keys.back() : keys.front(); const auto found = registered_sources.find(source_type); if (found != registered_sources.end()) diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index d01c75608328fe8992440ece6c24cc76afb8186c..309bc64e179ef0ad8ad3e7042328997f3a3a9e24 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -6,7 +6,10 @@ #include #include #include "DictionaryStructure.h" - +#include +#include +#include +#include namespace DB { @@ -50,4 +53,30 @@ void formatKeys( out->flush(); } +Context copyContextAndApplySettings( + const std::string & config_prefix, + const Context & context, + const Poco::Util::AbstractConfiguration & config) +{ + Context local_context(context); + if (config.has(config_prefix + ".settings")) + { + const auto prefix = config_prefix + ".settings"; + + Poco::Util::AbstractConfiguration::Keys config_keys; + config.keys(prefix, config_keys); + + SettingsChanges changes; + + for (const std::string & key : config_keys) + { + const auto value = config.getString(prefix + "." + key); + changes.emplace_back(key, value); + } + + local_context.applySettingsChanges(changes); + } + return local_context; +} + } diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index 99d5df5bda7f58848b016dde835282aadb68b6d5..3f42700d3362395e38e46e9f3cbe2ae0cceb0a45 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -3,7 +3,8 @@ #include #include #include - +#include +#include namespace DB { @@ -11,6 +12,7 @@ class IBlockOutputStream; using BlockOutputStreamPtr = std::shared_ptr; struct DictionaryStructure; +class Context; /// Write keys to block output stream. @@ -24,4 +26,14 @@ void formatKeys( const Columns & key_columns, const std::vector & requested_rows); +/// Used for applying settings to copied context in some register[...]Source functions +Context copyContextAndApplySettings( + const std::string & config_prefix, + const Context & context, + const Poco::Util::AbstractConfiguration & config); + +void applySettingsToContext( + const std::string & config_prefix, + Context & context, + const Poco::Util::AbstractConfiguration & config); } diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index d9903f759c07df78bed4e1e082efbbea30533041..34943d62b44c519428a4755ba2bdf69d31195bc4 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -16,7 +16,6 @@ #include "DictionaryStructure.h" #include "registerDictionaries.h" - namespace DB { static const UInt64 max_block_size = 8192; @@ -232,9 +231,11 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) if (check_config) throw Exception("Dictionaries with Executable dictionary source is not allowed", ErrorCodes::DICTIONARY_ACCESS_DENIED); + Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config); + return std::make_unique( dict_struct, config, config_prefix + ".executable", - sample_block, context); + sample_block, context_local_copy); }; factory.registerSource("executable", create_table_source); } diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index 879248663dcccf083fbca3df23a625e9019d75b5..f28d71ca5e31eaec6b91f3d76379af26b9a75d9a 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -3,7 +3,7 @@ #include "DictionaryStructure.h" #include "IDictionarySource.h" #include - +#include namespace Poco { class Logger; } @@ -56,7 +56,7 @@ private: const std::string update_field; const std::string format; Block sample_block; - const Context & context; + Context context; }; } diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index 5e7a784183cde3a6c50f6699132b3159d4c16a7c..0f20ab8edc4ddf176d61b189929c2136c040831f 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -7,6 +7,7 @@ #include "DictionarySourceFactory.h" #include "DictionaryStructure.h" #include "registerDictionaries.h" +#include "DictionarySourceHelpers.h" namespace DB { @@ -83,7 +84,9 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory) const auto filepath = config.getString(config_prefix + ".file.path"); const auto format = config.getString(config_prefix + ".file.format"); - return std::make_unique(filepath, format, sample_block, context, check_config); + Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config); + + return std::make_unique(filepath, format, sample_block, context_local_copy, check_config); }; factory.registerSource("file", create_table_source); diff --git a/src/Dictionaries/FileDictionarySource.h b/src/Dictionaries/FileDictionarySource.h index 3d00c026e0710859e271f57271e48bdf4266095f..fa47b280911897836b8db334a0351f1ac5720a24 100644 --- a/src/Dictionaries/FileDictionarySource.h +++ b/src/Dictionaries/FileDictionarySource.h @@ -3,7 +3,7 @@ #include #include "IDictionarySource.h" #include - +#include namespace DB { @@ -11,7 +11,6 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; } -class Context; /// Allows loading dictionaries from a file with given format, does not support "random access" class FileDictionarySource final : public IDictionarySource @@ -62,7 +61,7 @@ private: const std::string filepath; const std::string format; Block sample_block; - const Context & context; + const Context context; Poco::Timestamp last_modification; }; diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 87d5381f30fe9a1e7dc93e636fcee07624593982..61f16797ce0da92ed9ae5350baf51e2ac32150c0 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -202,9 +202,11 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) if (dict_struct.has_expressions) throw Exception{"Dictionary source of type `http` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR}; + Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config); + return std::make_unique( dict_struct, config, config_prefix + ".http", - sample_block, context, check_config); + sample_block, context_local_copy, check_config); }; factory.registerSource("http", create_table_source); } diff --git a/src/Dictionaries/HTTPDictionarySource.h b/src/Dictionaries/HTTPDictionarySource.h index d04a6f1a789137d86562c1eb9dc3e23ebe9a9938..e7920132e8376de18d2c5a77de618a85c12a516a 100644 --- a/src/Dictionaries/HTTPDictionarySource.h +++ b/src/Dictionaries/HTTPDictionarySource.h @@ -7,6 +7,7 @@ #include #include "DictionaryStructure.h" #include "IDictionarySource.h" +#include namespace Poco { @@ -64,7 +65,7 @@ private: std::string update_field; const std::string format; Block sample_block; - const Context & context; + Context context; ConnectionTimeouts timeouts; }; diff --git a/tests/integration/test_dictionary_custom_settings/__init__.py b/tests/integration/test_dictionary_custom_settings/__init__.py new file mode 100644 index 0000000000000000000000000000000000000000..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 diff --git a/tests/integration/test_dictionary_custom_settings/configs/config.xml b/tests/integration/test_dictionary_custom_settings/configs/config.xml new file mode 100644 index 0000000000000000000000000000000000000000..1e4c14585a964dfe7b6f268f9c5d9bcc48f40471 --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/configs/config.xml @@ -0,0 +1,30 @@ + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + + 9000 + 127.0.0.1 + + + + true + none + + AcceptCertificateHandler + + + + + 500 + 5368709120 + ./clickhouse/ + users.xml + + /etc/clickhouse-server/config.d/*.xml + diff --git a/tests/integration/test_dictionary_custom_settings/configs/dictionaries/ClickHouseSourceConfig.xml b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/ClickHouseSourceConfig.xml new file mode 100644 index 0000000000000000000000000000000000000000..2191c8ded8a56d8b374dffa18a3ae55ac5835069 --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/ClickHouseSourceConfig.xml @@ -0,0 +1,48 @@ + + + test_clickhouse + + + + localhost + 9000 + default + + default + source
+
+ + + 1 + + + + + 600 + + + + + + + + id + + + first + String + + + + second + String + + + + third + String + + + +
+
diff --git a/tests/integration/test_dictionary_custom_settings/configs/dictionaries/ExecutableSourceConfig.xml b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/ExecutableSourceConfig.xml new file mode 100644 index 0000000000000000000000000000000000000000..3191118c4e92cd42c44cc5557139f0fce32af7a3 --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/ExecutableSourceConfig.xml @@ -0,0 +1,45 @@ + + + test_executable + + + + cat /etc/clickhouse-server/config.d/source.csv + CSVWithNames + + + + 0 + 0 + + + + + 600 + + + + + + + + id + + + first + String + + + + second + String + + + + third + String + + + + + diff --git a/tests/integration/test_dictionary_custom_settings/configs/dictionaries/FileSourceConfig.xml b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/FileSourceConfig.xml new file mode 100644 index 0000000000000000000000000000000000000000..ff7baf29be02adfcf94cb26c69069ea04be2e186 --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/FileSourceConfig.xml @@ -0,0 +1,45 @@ + + + test_file + + + + /etc/clickhouse-server/config.d/source.csv + CSVWithNames + + + + 0 + 0 + + + + + 600 + + + + + + + + id + + + first + String + + + + second + String + + + + third + String + + + + + diff --git a/tests/integration/test_dictionary_custom_settings/configs/dictionaries/HTTPSourceConfig.xml b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/HTTPSourceConfig.xml new file mode 100644 index 0000000000000000000000000000000000000000..dc03974c4b62f935e1b00bd8f1132987f29e2394 --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/HTTPSourceConfig.xml @@ -0,0 +1,54 @@ + + + test_http + + + http://localhost:5555/source.csv + CSVWithNames + + foo + bar + + +
+ api-key + secret +
+
+
+ + + 0 + 0 + + + + + 600 + + + + + + + + id + + + first + String + + + + second + String + + + + third + String + + + +
+
diff --git a/tests/integration/test_dictionary_custom_settings/configs/dictionaries/source.csv b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/source.csv new file mode 100644 index 0000000000000000000000000000000000000000..23d113e522598d4e59faa953392ec7634f7f6d95 --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/configs/dictionaries/source.csv @@ -0,0 +1,3 @@ +id,first,second,third +1,'a,"b,c +2,'d,"e,f diff --git a/tests/integration/test_dictionary_custom_settings/configs/users.xml b/tests/integration/test_dictionary_custom_settings/configs/users.xml new file mode 100644 index 0000000000000000000000000000000000000000..6061af8e33d7d30e2b2229e4d1a9fe8ad50e9640 --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/configs/users.xml @@ -0,0 +1,23 @@ + + + + + + + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/tests/integration/test_dictionary_custom_settings/http_server.py b/tests/integration/test_dictionary_custom_settings/http_server.py new file mode 100644 index 0000000000000000000000000000000000000000..c7920a9024d75f70ed21f08e713aa4c1c3338eaa --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/http_server.py @@ -0,0 +1,86 @@ +# -*- coding: utf-8 -*- +import argparse +from BaseHTTPServer import BaseHTTPRequestHandler, HTTPServer +import socket +import ssl +import csv + + +# Decorator used to see if authentication works for external dictionary who use a HTTP source. +def check_auth(fn): + def wrapper(req): + auth_header = req.headers.get('authorization', None) + api_key = req.headers.get('api-key', None) + if not auth_header or auth_header != 'Basic Zm9vOmJhcg==' or not api_key or api_key != 'secret': + req.send_response(401) + else: + fn(req) + return wrapper + + +def start_server(server_address, data_path, schema, cert_path, address_family): + class TSVHTTPHandler(BaseHTTPRequestHandler): + @check_auth + def do_GET(self): + self.__send_headers() + self.__send_data() + + @check_auth + def do_POST(self): + ids = self.__read_and_decode_post_ids() + print "ids=", ids + self.__send_headers() + self.__send_data(ids) + + def __send_headers(self): + self.send_response(200) + self.send_header('Content-type', 'text/csv') + self.end_headers() + + def __send_data(self, only_ids = None): + with open(data_path, 'r') as fl: + reader = csv.reader(fl, delimiter='\t') + for row in reader: + if not only_ids or (row[0] in only_ids): + self.wfile.write('\t'.join(row) + '\n') + + def __read_and_decode_post_ids(self): + data = self.__read_and_decode_post_data() + return filter(None, data.split()) + + def __read_and_decode_post_data(self): + transfer_encoding = self.headers.get("Transfer-encoding") + decoded = "" + if transfer_encoding == "chunked": + while True: + s = self.rfile.readline() + chunk_length = int(s, 16) + if not chunk_length: + break + decoded += self.rfile.read(chunk_length) + self.rfile.readline() + else: + content_length = int(self.headers.get("Content-Length", 0)) + decoded = self.rfile.read(content_length) + return decoded + + if address_family == "ipv6": + HTTPServer.address_family = socket.AF_INET6 + httpd = HTTPServer(server_address, TSVHTTPHandler) + if schema == "https": + httpd.socket = ssl.wrap_socket(httpd.socket, certfile=cert_path, server_side=True) + httpd.serve_forever() + + +if __name__ == "__main__": + parser = argparse.ArgumentParser(description="Simple HTTP server returns data from file") + parser.add_argument("--host", default="localhost") + parser.add_argument("--port", default=5555, type=int) + parser.add_argument("--data-path", required=True) + parser.add_argument("--schema", choices=("http", "https"), required=True) + parser.add_argument("--cert-path", default="./fake_cert.pem") + parser.add_argument('--address-family', choices=("ipv4", "ipv6"), default="ipv4") + + args = parser.parse_args() + + start_server((args.host, args.port), args.data_path, args.schema, args.cert_path, args.address_family) diff --git a/tests/integration/test_dictionary_custom_settings/test.py b/tests/integration/test_dictionary_custom_settings/test.py new file mode 100644 index 0000000000000000000000000000000000000000..978748795250592b27564ad49da4659e7a08b58d --- /dev/null +++ b/tests/integration/test_dictionary_custom_settings/test.py @@ -0,0 +1,62 @@ +import os +import pytest + +from helpers.cluster import ClickHouseCluster + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +config_dir = os.path.join(SCRIPT_DIR, './configs') +DICTIONARY_FILES = [ + 'configs/dictionaries/FileSourceConfig.xml', + 'configs/dictionaries/ExecutableSourceConfig.xml', + 'configs/dictionaries/source.csv', + 'configs/dictionaries/HTTPSourceConfig.xml', + 'configs/dictionaries/ClickHouseSourceConfig.xml' +] + +cluster = ClickHouseCluster(__file__, base_configs_dir=config_dir) +instance = cluster.add_instance('node', main_configs=DICTIONARY_FILES, config_dir=config_dir) + +def prepare(): + node = instance + path = "/source.csv" + + script_dir = os.path.dirname(os.path.realpath(__file__)) + node.copy_file_to_container(os.path.join(script_dir, './http_server.py'), '/http_server.py') + node.copy_file_to_container(os.path.join(script_dir, 'configs/dictionaries/source.csv'), './source.csv') + node.exec_in_container([ + "bash", + "-c", + "python2 /http_server.py --data-path={tbl} --schema=http --host=localhost --port=5555".format( + tbl=path) + ], detach=True) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + prepare() + yield cluster + finally: + cluster.shutdown() + +def test_work(start_cluster): + query = instance.query + + assert query("SELECT dictGetString('test_file', 'first', toUInt64(1))") == "\\\'a\n" + assert query("SELECT dictGetString('test_file', 'second', toUInt64(1))") == "\"b\n" + assert query("SELECT dictGetString('test_executable', 'first', toUInt64(1))") == "\\\'a\n" + assert query("SELECT dictGetString('test_executable', 'second', toUInt64(1))") == "\"b\n" + + caught_exception = '' + try: + instance.query("CREATE TABLE source (id UInt64, first String, second String, third String) ENGINE=TinyLog;") + instance.query("INSERT INTO default.source VALUES (1, 'aaa', 'bbb', 'cccc'), (2, 'ddd', 'eee', 'fff')") + instance.query("SELECT dictGetString('test_clickhouse', 'second', toUInt64(1))") + except Exception as e: + caught_exception = str(e) + + assert caught_exception.find("Limit for result exceeded") != -1 + + assert query("SELECT dictGetString('test_http', 'first', toUInt64(1))") == "\\\'a\n" + assert query("SELECT dictGetString('test_http', 'second', toUInt64(1))") == "\"b\n" \ No newline at end of file