未验证 提交 1a067484 编写于 作者: T tavplubix 提交者: GitHub

Merge pull request #10137 from kekekekule/direct_queries

Add custom settings support for certain dictionary sources
...@@ -35,6 +35,25 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration ...@@ -35,6 +35,25 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration
The source is configured in the `source` section. 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
<source>
<file>
<path>/opt/dictionaries/os.tsv</path>
<format>TabSeparated</format>
</file>
<settings>
<format_csv_allow_single_quotes>0</format_csv_allow_single_quotes>
</settings>
</source>
```
Types of sources (`source_type`): Types of sources (`source_type`):
- [Local file](#dicts-external_dicts_dict_sources-local_file) - [Local file](#dicts-external_dicts_dict_sources-local_file)
......
...@@ -30,6 +30,25 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration ...@@ -30,6 +30,25 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration
Источник настраивается в разделе `source`. Источник настраивается в разделе `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
<source>
<file>
<path>/opt/dictionaries/os.tsv</path>
<format>TabSeparated</format>
</file>
<settings>
<format_csv_allow_single_quotes>0</format_csv_allow_single_quotes>
</settings>
</source>
```
Типы источников (`source_type`): Типы источников (`source_type`):
- [Локальный файл](#dicts-external_dicts_dict_sources-local_file) - [Локальный файл](#dicts-external_dicts_dict_sources-local_file)
......
...@@ -10,6 +10,11 @@ struct SettingChange ...@@ -10,6 +10,11 @@ struct SettingChange
{ {
String name; String name;
Field value; 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.name == rhs.name) && (lhs.value == rhs.value); }
friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); }
......
...@@ -13,7 +13,7 @@ ...@@ -13,7 +13,7 @@
#include "readInvalidateQuery.h" #include "readInvalidateQuery.h"
#include "writeParenthesisedString.h" #include "writeParenthesisedString.h"
#include "DictionaryFactory.h" #include "DictionaryFactory.h"
#include "DictionarySourceHelpers.h"
namespace DB namespace DB
{ {
...@@ -50,6 +50,7 @@ static ConnectionPoolWithFailoverPtr createPool( ...@@ -50,6 +50,7 @@ static ConnectionPoolWithFailoverPtr createPool(
ClickHouseDictionarySource::ClickHouseDictionarySource( ClickHouseDictionarySource::ClickHouseDictionarySource(
const DictionaryStructure & dict_struct_, const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & path_to_settings,
const std::string & config_prefix, const std::string & config_prefix,
const Block & sample_block_, const Block & sample_block_,
const Context & context_) const Context & context_)
...@@ -74,8 +75,11 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( ...@@ -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). /// 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.setUser(user, password, Poco::Net::SocketAddress("127.0.0.1", 0), {});
context = copyContextAndApplySettings(path_to_settings, context, config);
/// Processors are not supported here yet. /// Processors are not supported here yet.
context.setSetting("experimental_use_processors", false); context.setSetting("experimental_use_processors", false);
/// Query context is needed because some code in executeQuery function may assume it exists. /// Query context is needed because some code in executeQuery function may assume it exists.
/// Current example is Context::getSampleBlockCache from InterpreterSelectWithUnionQuery::getSampleBlock. /// Current example is Context::getSampleBlockCache from InterpreterSelectWithUnionQuery::getSampleBlock.
context.makeQueryContext(); context.makeQueryContext();
...@@ -227,7 +231,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) ...@@ -227,7 +231,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
const Context & context, const Context & context,
bool /* check_config */) -> DictionarySourcePtr bool /* check_config */) -> DictionarySourcePtr
{ {
return std::make_unique<ClickHouseDictionarySource>(dict_struct, config, config_prefix + ".clickhouse", sample_block, context); return std::make_unique<ClickHouseDictionarySource>(dict_struct, config, config_prefix, config_prefix + ".clickhouse", sample_block, context);
}; };
factory.registerSource("clickhouse", create_table_source); factory.registerSource("clickhouse", create_table_source);
} }
......
...@@ -21,6 +21,7 @@ public: ...@@ -21,6 +21,7 @@ public:
ClickHouseDictionarySource( ClickHouseDictionarySource(
const DictionaryStructure & dict_struct_, const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & path_to_settings,
const std::string & config_prefix, const std::string & config_prefix,
const Block & sample_block_, const Block & sample_block_,
const Context & context); const Context & context);
......
...@@ -84,11 +84,12 @@ DictionarySourcePtr DictionarySourceFactory::create( ...@@ -84,11 +84,12 @@ DictionarySourcePtr DictionarySourceFactory::create(
{ {
Poco::Util::AbstractConfiguration::Keys keys; Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, 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}; 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); const auto found = registered_sources.find(source_type);
if (found != registered_sources.end()) if (found != registered_sources.end())
......
...@@ -6,7 +6,10 @@ ...@@ -6,7 +6,10 @@
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include "DictionaryStructure.h" #include "DictionaryStructure.h"
#include <Interpreters/Context.h>
#include <Core/Settings.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/SettingsChanges.h>
namespace DB namespace DB
{ {
...@@ -50,4 +53,30 @@ void formatKeys( ...@@ -50,4 +53,30 @@ void formatKeys(
out->flush(); 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;
}
} }
...@@ -3,7 +3,8 @@ ...@@ -3,7 +3,8 @@
#include <vector> #include <vector>
#include <Columns/IColumn.h> #include <Columns/IColumn.h>
#include <common/types.h> #include <common/types.h>
#include <Poco/File.h>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB namespace DB
{ {
...@@ -11,6 +12,7 @@ class IBlockOutputStream; ...@@ -11,6 +12,7 @@ class IBlockOutputStream;
using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>; using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>;
struct DictionaryStructure; struct DictionaryStructure;
class Context;
/// Write keys to block output stream. /// Write keys to block output stream.
...@@ -24,4 +26,14 @@ void formatKeys( ...@@ -24,4 +26,14 @@ void formatKeys(
const Columns & key_columns, const Columns & key_columns,
const std::vector<size_t> & requested_rows); const std::vector<size_t> & 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);
} }
...@@ -16,7 +16,6 @@ ...@@ -16,7 +16,6 @@
#include "DictionaryStructure.h" #include "DictionaryStructure.h"
#include "registerDictionaries.h" #include "registerDictionaries.h"
namespace DB namespace DB
{ {
static const UInt64 max_block_size = 8192; static const UInt64 max_block_size = 8192;
...@@ -232,9 +231,11 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) ...@@ -232,9 +231,11 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
if (check_config) if (check_config)
throw Exception("Dictionaries with Executable dictionary source is not allowed", ErrorCodes::DICTIONARY_ACCESS_DENIED); 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<ExecutableDictionarySource>( return std::make_unique<ExecutableDictionarySource>(
dict_struct, config, config_prefix + ".executable", dict_struct, config, config_prefix + ".executable",
sample_block, context); sample_block, context_local_copy);
}; };
factory.registerSource("executable", create_table_source); factory.registerSource("executable", create_table_source);
} }
......
...@@ -3,7 +3,7 @@ ...@@ -3,7 +3,7 @@
#include "DictionaryStructure.h" #include "DictionaryStructure.h"
#include "IDictionarySource.h" #include "IDictionarySource.h"
#include <Core/Block.h> #include <Core/Block.h>
#include <Interpreters/Context.h>
namespace Poco { class Logger; } namespace Poco { class Logger; }
...@@ -56,7 +56,7 @@ private: ...@@ -56,7 +56,7 @@ private:
const std::string update_field; const std::string update_field;
const std::string format; const std::string format;
Block sample_block; Block sample_block;
const Context & context; Context context;
}; };
} }
...@@ -7,6 +7,7 @@ ...@@ -7,6 +7,7 @@
#include "DictionarySourceFactory.h" #include "DictionarySourceFactory.h"
#include "DictionaryStructure.h" #include "DictionaryStructure.h"
#include "registerDictionaries.h" #include "registerDictionaries.h"
#include "DictionarySourceHelpers.h"
namespace DB namespace DB
{ {
...@@ -83,7 +84,9 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory) ...@@ -83,7 +84,9 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
const auto filepath = config.getString(config_prefix + ".file.path"); const auto filepath = config.getString(config_prefix + ".file.path");
const auto format = config.getString(config_prefix + ".file.format"); const auto format = config.getString(config_prefix + ".file.format");
return std::make_unique<FileDictionarySource>(filepath, format, sample_block, context, check_config); Context context_local_copy = copyContextAndApplySettings(config_prefix, context, config);
return std::make_unique<FileDictionarySource>(filepath, format, sample_block, context_local_copy, check_config);
}; };
factory.registerSource("file", create_table_source); factory.registerSource("file", create_table_source);
......
...@@ -3,7 +3,7 @@ ...@@ -3,7 +3,7 @@
#include <Poco/Timestamp.h> #include <Poco/Timestamp.h>
#include "IDictionarySource.h" #include "IDictionarySource.h"
#include <Core/Block.h> #include <Core/Block.h>
#include <Interpreters/Context.h>
namespace DB namespace DB
{ {
...@@ -11,7 +11,6 @@ namespace ErrorCodes ...@@ -11,7 +11,6 @@ namespace ErrorCodes
{ {
extern const int NOT_IMPLEMENTED; extern const int NOT_IMPLEMENTED;
} }
class Context;
/// Allows loading dictionaries from a file with given format, does not support "random access" /// Allows loading dictionaries from a file with given format, does not support "random access"
class FileDictionarySource final : public IDictionarySource class FileDictionarySource final : public IDictionarySource
...@@ -62,7 +61,7 @@ private: ...@@ -62,7 +61,7 @@ private:
const std::string filepath; const std::string filepath;
const std::string format; const std::string format;
Block sample_block; Block sample_block;
const Context & context; const Context context;
Poco::Timestamp last_modification; Poco::Timestamp last_modification;
}; };
......
...@@ -202,9 +202,11 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) ...@@ -202,9 +202,11 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
if (dict_struct.has_expressions) if (dict_struct.has_expressions)
throw Exception{"Dictionary source of type `http` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR}; 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<HTTPDictionarySource>( return std::make_unique<HTTPDictionarySource>(
dict_struct, config, config_prefix + ".http", dict_struct, config, config_prefix + ".http",
sample_block, context, check_config); sample_block, context_local_copy, check_config);
}; };
factory.registerSource("http", create_table_source); factory.registerSource("http", create_table_source);
} }
......
...@@ -7,6 +7,7 @@ ...@@ -7,6 +7,7 @@
#include <common/LocalDateTime.h> #include <common/LocalDateTime.h>
#include "DictionaryStructure.h" #include "DictionaryStructure.h"
#include "IDictionarySource.h" #include "IDictionarySource.h"
#include <Interpreters/Context.h>
namespace Poco namespace Poco
{ {
...@@ -64,7 +65,7 @@ private: ...@@ -64,7 +65,7 @@ private:
std::string update_field; std::string update_field;
const std::string format; const std::string format;
Block sample_block; Block sample_block;
const Context & context; Context context;
ConnectionTimeouts timeouts; ConnectionTimeouts timeouts;
}; };
......
<?xml version="1.0"?>
<yandex>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
<size>1000M</size>
<count>10</count>
</logger>
<tcp_port>9000</tcp_port>
<listen_host>127.0.0.1</listen_host>
<openSSL>
<client>
<cacheSessions>true</cacheSessions>
<verificationMode>none</verificationMode>
<invalidCertificateHandler>
<name>AcceptCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
<max_concurrent_queries>500</max_concurrent_queries>
<mark_cache_size>5368709120</mark_cache_size>
<path>./clickhouse/</path>
<users_config>users.xml</users_config>
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
</yandex>
<yandex>
<dictionary>
<name>test_clickhouse</name>
<source>
<clickhouse>
<host>localhost</host>
<port>9000</port>
<user>default</user>
<password></password>
<db>default</db>
<table>source</table>
</clickhouse>
<settings>
<max_result_bytes>1</max_result_bytes>
</settings>
</source>
<lifetime>600</lifetime>
<layout>
<flat/>
</layout>
<structure>
<id>
<name>id</name>
</id>
<attribute>
<name>first</name>
<type>String</type>
<null_value></null_value>
</attribute>
<attribute>
<name>second</name>
<type>String</type>
<null_value></null_value>
</attribute>
<attribute>
<name>third</name>
<type>String</type>
<null_value></null_value>
</attribute>
</structure>
</dictionary>
</yandex>
<yandex>
<dictionary>
<name>test_executable</name>
<source>
<executable>
<command>cat /etc/clickhouse-server/config.d/source.csv</command>
<format>CSVWithNames</format>
</executable>
<settings>
<format_csv_allow_single_quotes>0</format_csv_allow_single_quotes>
<format_csv_allow_double_quotes>0</format_csv_allow_double_quotes>
</settings>
</source>
<lifetime>600</lifetime>
<layout>
<flat/>
</layout>
<structure>
<id>
<name>id</name>
</id>
<attribute>
<name>first</name>
<type>String</type>
<null_value></null_value>
</attribute>
<attribute>
<name>second</name>
<type>String</type>
<null_value></null_value>
</attribute>
<attribute>
<name>third</name>
<type>String</type>
<null_value></null_value>
</attribute>
</structure>
</dictionary>
</yandex>
<yandex>
<dictionary>
<name>test_file</name>
<source>
<file>
<path>/etc/clickhouse-server/config.d/source.csv</path>
<format>CSVWithNames</format>
</file>
<settings>
<format_csv_allow_single_quotes>0</format_csv_allow_single_quotes>
<format_csv_allow_double_quotes>0</format_csv_allow_double_quotes>
</settings>
</source>
<lifetime>600</lifetime>
<layout>
<flat/>
</layout>
<structure>
<id>
<name>id</name>
</id>
<attribute>
<name>first</name>
<type>String</type>
<null_value></null_value>
</attribute>
<attribute>
<name>second</name>
<type>String</type>
<null_value></null_value>
</attribute>
<attribute>
<name>third</name>
<type>String</type>
<null_value></null_value>
</attribute>
</structure>
</dictionary>
</yandex>
<yandex>
<dictionary>
<name>test_http</name>
<source>
<http>
<url>http://localhost:5555/source.csv</url>
<format>CSVWithNames</format>
<credentials>
<user>foo</user>
<password>bar</password>
</credentials>
<headers>
<header>
<name>api-key</name>
<value>secret</value>
</header>
</headers>
</http>
<settings>
<format_csv_allow_single_quotes>0</format_csv_allow_single_quotes>
<format_csv_allow_double_quotes>0</format_csv_allow_double_quotes>
</settings>
</source>
<lifetime>600</lifetime>
<layout>
<flat/>
</layout>
<structure>
<id>
<name>id</name>
</id>
<attribute>
<name>first</name>
<type>String</type>
<null_value></null_value>
</attribute>
<attribute>
<name>second</name>
<type>String</type>
<null_value></null_value>
</attribute>
<attribute>
<name>third</name>
<type>String</type>
<null_value></null_value>
</attribute>
</structure>
</dictionary>
</yandex>
<?xml version="1.0"?>
<yandex>
<profiles>
<default>
</default>
</profiles>
<users>
<default>
<password></password>
<networks incl="networks" replace="replace">
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
</default>
</users>
<quotas>
<default>
</default>
</quotas>
</yandex>
# -*- 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)
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
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册