Implemented YAMLParser class

This commit is contained in:
BoloniniD 2021-05-21 21:56:22 +03:00
parent 2c4920186b
commit 33755f1f3d
124 changed files with 3873 additions and 23 deletions

4
.gitmodules vendored
View File

@ -228,3 +228,7 @@
[submodule "contrib/datasketches-cpp"]
path = contrib/datasketches-cpp
url = https://github.com/ClickHouse-Extras/datasketches-cpp.git
[submodule "contrib/yaml-cpp"]
path = contrib/yaml-cpp
url = https://github.com/ClickHouse-Extras/yaml-cpp.git

View File

@ -527,6 +527,7 @@ include (cmake/find/nanodbc.cmake)
include (cmake/find/rocksdb.cmake)
include (cmake/find/libpqxx.cmake)
include (cmake/find/nuraft.cmake)
include (cmake/find/yaml-cpp.cmake)
if(NOT USE_INTERNAL_PARQUET_LIBRARY)

View File

@ -3,5 +3,11 @@ add_library (bridge
)
target_include_directories (daemon PUBLIC ..)
target_link_libraries (bridge PRIVATE daemon dbms Poco::Data Poco::Data::ODBC)
target_link_libraries (bridge
PRIVATE
daemon
dbms
Poco::Data
Poco::Data::ODBC
)

View File

@ -0,0 +1,9 @@
option(USE_YAML_CPP "Enable yaml-cpp" ${ENABLE_LIBRARIES})
if (NOT USE_YAML_CPP)
return()
endif()
if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/yaml-cpp")
message (ERROR "submodule contrib/yaml-cpp is missing. to fix try run: \n git submodule update --init --recursive")
endif()

View File

@ -50,6 +50,10 @@ add_subdirectory (replxx-cmake)
add_subdirectory (unixodbc-cmake)
add_subdirectory (nanodbc-cmake)
if (USE_YAML_CPP)
add_subdirectory (yaml-cpp-cmake)
endif()
if (USE_INTERNAL_XZ_LIBRARY)
add_subdirectory (xz)
endif()

1
contrib/yaml-cpp vendored Submodule

@ -0,0 +1 @@
Subproject commit 0c86adac6d117ee2b4afcedb8ade19036ca0327d

View File

@ -0,0 +1,39 @@
set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/yaml-cpp)
set (SRCS
${LIBRARY_DIR}/src/binary.cpp
${LIBRARY_DIR}/src/emitterutils.cpp
${LIBRARY_DIR}/src/null.cpp
${LIBRARY_DIR}/src/scantoken.cpp
${LIBRARY_DIR}/src/convert.cpp
${LIBRARY_DIR}/src/exceptions.cpp
${LIBRARY_DIR}/src/ostream_wrapper.cpp
${LIBRARY_DIR}/src/simplekey.cpp
${LIBRARY_DIR}/src/depthguard.cpp
${LIBRARY_DIR}/src/exp.cpp
${LIBRARY_DIR}/src/parse.cpp
${LIBRARY_DIR}/src/singledocparser.cpp
${LIBRARY_DIR}/src/directives.cpp
${LIBRARY_DIR}/src/memory.cpp
${LIBRARY_DIR}/src/parser.cpp
${LIBRARY_DIR}/src/stream.cpp
${LIBRARY_DIR}/src/emit.cpp
${LIBRARY_DIR}/src/nodebuilder.cpp
${LIBRARY_DIR}/src/regex_yaml.cpp
${LIBRARY_DIR}/src/tag.cpp
${LIBRARY_DIR}/src/emitfromevents.cpp
${LIBRARY_DIR}/src/node.cpp
${LIBRARY_DIR}/src/scanner.cpp
${LIBRARY_DIR}/src/emitter.cpp
${LIBRARY_DIR}/src/node_data.cpp
${LIBRARY_DIR}/src/scanscalar.cpp
${LIBRARY_DIR}/src/emitterstate.cpp
${LIBRARY_DIR}/src/nodeevents.cpp
${LIBRARY_DIR}/src/scantag.cpp
)
add_library (yaml-cpp ${SRCS})
target_include_directories(yaml-cpp PRIVATE ${LIBRARY_DIR}/include/yaml-cpp)
target_include_directories(yaml-cpp SYSTEM BEFORE PUBLIC ${LIBRARY_DIR}/include)

View File

@ -0,0 +1,86 @@
# We can use 3 main node types in YAML: Scalar, Map and Sequence.
# A Scalar is a simple key-value pair:
scalar: 123
# Here we have a key "scalar" and value "123"
# If we rewrite this in XML, we will get <scalar>123</scalar>
# We can also represent an empty value with '':
key: ''
# A Map is a node, which contains other nodes:
map:
key1: value1
key2: value2
small_map:
key3: value3
# This map can be converted into:
# <map>
# <key1>value1</key1>
# <key2>value2</key2>
# <small_map>
# <key3>value3</key3>
# </small_map>
# </map>
# A Sequence is a node, which contains also other nodes.
# The main difference from Map is that Sequence can also contain simple values.
sequence:
- val1
- val2
- key: 123
- map:
mkey1: foo
mkey2: bar
# We can represent it in XML this way:
# <sequence>val1</sequence>
# <sequence>val2</sequence>
# <sequence>
# <key>123</key>
# </sequence>
# <sequence>
# <map>
# <mkey1>foo</mkey1>
# <mkey2>bar</mkey2>
# </map>
# </sequence>
# YAML does not have direct support for structures like XML attributes.
# We represent them as nodes with @ prefix in key. Note, that @ is reserved by YAML standard,
# so you will need to write double quotes around the key. Both Map and Sequence can have
# attributes as children nodes
map:
"@attr1": value1
"@attr2": value2
key: 123
# This gives us:
# <map attr1="value1" attr2="value2">
# <key>123</key>
# </map>
sequence:
- "@attr1": value1
- "@attr2": value2
- 123
- abc
# And this gives us:
# <map attr1="value1" attr2="value2">123</map>
# <map attr1="value1" attr2="value2">abc</map>

View File

@ -3,6 +3,7 @@ set (SRCS
ConfigProcessor.cpp
configReadClient.cpp
ConfigReloader.cpp
YAMLParser.cpp
)
add_library(clickhouse_common_config ${SRCS})
@ -15,3 +16,10 @@ target_link_libraries(clickhouse_common_config
PRIVATE
string_utils
)
if (USE_YAML_CPP)
target_link_libraries(clickhouse_common_config
PRIVATE
yaml-cpp
)
endif()

View File

@ -1,4 +1,8 @@
#if !defined(ARCADIA_BUILD)
#include <Common/config.h>
#endif
#include "ConfigProcessor.h"
#include "YAMLParser.h"
#include <sys/utsname.h>
#include <cerrno>
@ -20,10 +24,8 @@
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#define PREPROCESSED_SUFFIX "-preprocessed"
namespace fs = std::filesystem;
using namespace Poco::XML;
@ -438,8 +440,10 @@ ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string &
std::string base_name = path.getBaseName();
// Skip non-config and temporary files
if (file.isFile() && (extension == "xml" || extension == "conf") && !startsWith(base_name, "."))
files.push_back(file.path());
if (file.isFile() && (extension == "xml" || extension == "conf" || extension == "yaml" || extension == "yml") && !startsWith(base_name, "."))
{
files.push_back(file.path());
}
}
}
@ -453,12 +457,21 @@ XMLDocumentPtr ConfigProcessor::processConfig(
zkutil::ZooKeeperNodeCache * zk_node_cache,
const zkutil::EventPtr & zk_changed_event)
{
XMLDocumentPtr config;
LOG_DEBUG(log, "Processing configuration file '{}'.", path);
XMLDocumentPtr config;
if (fs::exists(path))
{
config = dom_parser.parse(path);
fs::path p(path);
if (p.extension() == ".xml")
{
config = dom_parser.parse(path);
}
else if (p.extension() == ".yaml" || p.extension() == ".yml")
{
config = YAMLParser::parse(path);
}
}
else
{
@ -493,8 +506,20 @@ XMLDocumentPtr ConfigProcessor::processConfig(
{
LOG_DEBUG(log, "Merging configuration file '{}'.", merge_file);
XMLDocumentPtr with = dom_parser.parse(merge_file);
XMLDocumentPtr with;
fs::path p(merge_file);
if (p.extension() == ".yaml" || p.extension() == ".yml")
{
with = YAMLParser::parse(merge_file);
}
else
{
with = dom_parser.parse(merge_file);
}
merge(config, with);
contributing_files.push_back(merge_file);
}
catch (Exception & e)

View File

@ -1,5 +1,9 @@
#pragma once
#if !defined(ARCADIA_BUILD)
#include <Common/config.h>
#endif
#include <string>
#include <unordered_set>
#include <vector>
@ -141,3 +145,4 @@ private:
};
}

View File

@ -0,0 +1,166 @@
#if !defined(ARCADIA_BUILD)
#include <Common/config.h>
#endif
#if USE_YAML_CPP
#include "YAMLParser.h"
#include <string>
#include <cstring>
#include <vector>
#include <Poco/DOM/Document.h>
#include <Poco/DOM/DOMParser.h>
#include <Poco/DOM/DOMWriter.h>
#include <Poco/DOM/NodeList.h>
#include <Poco/DOM/Element.h>
#include <Poco/DOM/AutoPtr.h>
#include <Poco/DOM/NamedNodeMap.h>
#include <Poco/DOM/Text.h>
#include <Common/Exception.h>
#include <yaml-cpp/yaml.h> // Y_IGNORE
#include <common/logger_useful.h>
using namespace Poco::XML;
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_OPEN_FILE;
extern const int CANNOT_PARSE_YAML;
}
/// A prefix symbol in yaml key
/// We add attributes to nodes by using a prefix symbol in the key part.
/// Currently we use @ as a prefix symbol. Note, that @ is reserved
/// by YAML standard, so we need to write a key-value pair like this: "@attribute": attr_value
const char YAML_ATTRIBUTE_PREFIX = '@';
namespace
{
Poco::AutoPtr<Poco::XML::Element> createCloneNode(Poco::XML::Element & original_node)
{
Poco::AutoPtr<Poco::XML::Element> clone_node = original_node.ownerDocument()->createElement(original_node.nodeName());
original_node.parentNode()->appendChild(clone_node);
return clone_node;
}
void processNode(const YAML::Node & node, Poco::XML::Element & parent_xml_element)
{
auto * xml_document = parent_xml_element.ownerDocument();
switch (node.Type())
{
case YAML::NodeType::Scalar:
{
auto value = node.as<std::string>();
Poco::AutoPtr<Poco::XML::Text> xml_value = xml_document->createTextNode(value);
parent_xml_element.appendChild(xml_value);
break;
}
/// We process YAML Sequences as a
/// list of <key>value</key> tags with same key and different values.
/// For example, we translate this sequence
/// seq:
/// - val1
/// - val2
///
/// into this:
/// <seq>val1</seq>
/// <seq>val2</seq>
case YAML::NodeType::Sequence:
{
for (const auto & child_node : node)
if (parent_xml_element.hasChildNodes())
{
/// We want to process sequences like that:
/// seq:
/// - val1
/// - k2: val2
/// - val3
/// - k4: val4
/// - val5
/// into xml like this:
/// <seq>val1</seq>
/// <seq>
/// <k2>val2</k2>
/// </seq>
/// <seq>val3</seq>
/// <seq>
/// <k4>val4</k4>
/// </seq>
/// <seq>val5</seq>
/// So, we create a new parent node with same tag for each child node
processNode(child_node, *createCloneNode(parent_xml_element));
}
else
{
processNode(child_node, parent_xml_element);
}
break;
}
case YAML::NodeType::Map:
{
for (const auto & key_value_pair : node)
{
const auto & key_node = key_value_pair.first;
const auto & value_node = key_value_pair.second;
auto key = key_node.as<std::string>();
bool is_attribute = (key.starts_with(YAML_ATTRIBUTE_PREFIX) && value_node.IsScalar());
if (is_attribute)
{
/// we use substr(1) here to remove YAML_ATTRIBUTE_PREFIX from key
auto attribute_name = key.substr(1);
auto value = value_node.as<std::string>();
parent_xml_element.setAttribute(attribute_name, value);
}
else
{
Poco::AutoPtr<Poco::XML::Element> xml_key = xml_document->createElement(key);
parent_xml_element.appendChild(xml_key);
processNode(value_node, *xml_key);
}
}
break;
}
case YAML::NodeType::Null: break;
case YAML::NodeType::Undefined:
{
throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "YAMLParser has encountered node with undefined type and cannot continue parsing of the file");
}
}
}
}
Poco::AutoPtr<Poco::XML::Document> YAMLParser::parse(const String& path)
{
YAML::Node node_yml;
try
{
node_yml = YAML::LoadFile(path);
}
catch (const YAML::ParserException& e)
{
/// yaml-cpp cannot parse the file because its contents are incorrect
throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "Unable to parse YAML configuration file {}", path, e.what());
}
catch (const YAML::BadFile&)
{
/// yaml-cpp cannot open the file even though it exists
throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Unable to open YAML configuration file {}", path);
}
Poco::AutoPtr<Poco::XML::Document> xml = new Document;
Poco::AutoPtr<Poco::XML::Element> root_node = xml->createElement("yandex");
xml->appendChild(root_node);
processNode(node_yml, *root_node);
return xml;
}
}
#endif

View File

@ -0,0 +1,55 @@
#pragma once
#if !defined(ARCADIA_BUILD)
#include <Common/config.h>
#endif
#include <string>
#include <Poco/DOM/Document.h>
#include "Poco/DOM/AutoPtr.h"
#include <common/logger_useful.h>
#if USE_YAML_CPP
namespace DB
{
/// Real YAML parser: loads yaml file into a YAML::Node
class YAMLParserImpl
{
public:
static Poco::AutoPtr<Poco::XML::Document> parse(const String& path);
};
using YAMLParser = YAMLParserImpl;
}
#else
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_PARSE_YAML;
}
/// Fake YAML parser: throws an exception if we try to parse YAML configs in a build without yaml-cpp
class DummyYAMLParser
{
public:
static Poco::AutoPtr<Poco::XML::Document> parse(const String& path)
{
Poco::AutoPtr<Poco::XML::Document> xml = new Poco::XML::Document;
throw Exception(ErrorCodes::CANNOT_PARSE_YAML, "Unable to parse YAML configuration file {} without usage of yaml-cpp library", path);
return xml;
}
};
using YAMLParser = DummyYAMLParser;
}
#endif

View File

@ -552,6 +552,7 @@
M(582, NO_SUCH_PROJECTION_IN_TABLE) \
M(583, ILLEGAL_PROJECTION) \
M(584, PROJECTION_NOT_USED) \
M(585, CANNOT_PARSE_YAML) \
\
M(998, POSTGRESQL_CONNECTION_FAILURE) \
M(999, KEEPER_EXCEPTION) \

View File

@ -16,3 +16,4 @@
#cmakedefine01 USE_STATS
#cmakedefine01 CLICKHOUSE_SPLIT_BINARY
#cmakedefine01 USE_DATASKETCHES
#cmakedefine01 USE_YAML_CPP

View File

@ -126,7 +126,8 @@ class ClickHouseCluster:
"""
def __init__(self, base_path, name=None, base_config_dir=None, server_bin_path=None, client_bin_path=None,
odbc_bridge_bin_path=None, library_bridge_bin_path=None, zookeeper_config_path=None, custom_dockerd_host=None):
odbc_bridge_bin_path=None, library_bridge_bin_path=None, zookeeper_config_path=None,
custom_dockerd_host=None):
for param in list(os.environ.keys()):
print("ENV %40s %s" % (param, os.environ[param]))
self.base_dir = p.dirname(base_path)
@ -219,7 +220,9 @@ class ClickHouseCluster:
with_redis=False, with_minio=False, with_cassandra=False,
hostname=None, env_variables=None, image="yandex/clickhouse-integration-test", tag=None,
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None,
zookeeper_docker_compose_path=None, zookeeper_use_tmpfs=True, minio_certs_dir=None, use_keeper=True):
zookeeper_docker_compose_path=None, zookeeper_use_tmpfs=True, minio_certs_dir=None, use_keeper=True,
main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True):
"""Add an instance to the cluster.
name - the name of the instance directory and the value of the 'instance' macro in ClickHouse.
@ -280,6 +283,9 @@ class ClickHouseCluster:
ipv4_address=ipv4_address,
ipv6_address=ipv6_address,
with_installed_binary=with_installed_binary,
main_config_name=main_config_name,
users_config_name=users_config_name,
copy_common_configs=copy_common_configs,
tmpfs=tmpfs or [])
docker_compose_yml_dir = get_docker_compose_path()
@ -944,7 +950,7 @@ class ClickHouseCluster:
subprocess_check_call(self.base_zookeeper_cmd + ["start", n])
CLICKHOUSE_START_COMMAND = "clickhouse server --config-file=/etc/clickhouse-server/config.xml --log-file=/var/log/clickhouse-server/clickhouse-server.log --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log"
CLICKHOUSE_START_COMMAND = "clickhouse server --config-file=/etc/clickhouse-server/{main_config_file} --log-file=/var/log/clickhouse-server/clickhouse-server.log --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log"
CLICKHOUSE_STAY_ALIVE_COMMAND = 'bash -c "{} --daemon; tail -f /dev/null"'.format(CLICKHOUSE_START_COMMAND)
@ -1000,6 +1006,8 @@ class ClickHouseInstance:
macros, with_zookeeper, zookeeper_config_path, with_mysql, with_mysql_cluster, with_kafka, with_kerberized_kafka, with_rabbitmq, with_kerberized_hdfs,
with_mongo, with_redis, with_minio,
with_cassandra, server_bin_path, odbc_bridge_bin_path, library_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers,
clickhouse_start_command=CLICKHOUSE_START_COMMAND,
main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True,
hostname=None, env_variables=None,
image="yandex/clickhouse-integration-test", tag="latest",
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None):
@ -1036,6 +1044,12 @@ class ClickHouseInstance:
self.with_minio = with_minio
self.with_cassandra = with_cassandra
self.main_config_name = main_config_name
self.users_config_name = users_config_name
self.copy_common_configs = copy_common_configs
self.clickhouse_start_command = clickhouse_start_command.replace("{main_config_file}", self.main_config_name)
self.path = p.join(self.cluster.instances_dir, name)
self.docker_compose_path = p.join(self.path, 'docker-compose.yml')
self.env_variables = env_variables or {}
@ -1177,7 +1191,7 @@ class ClickHouseInstance:
if not self.stay_alive:
raise Exception("clickhouse can be started again only with stay_alive=True instance")
self.exec_in_container(["bash", "-c", "{} --daemon".format(CLICKHOUSE_START_COMMAND)], user=str(os.getuid()))
self.exec_in_container(["bash", "-c", "{} --daemon".format(self.clickhouse_start_command)], user=str(os.getuid()))
# wait start
from helpers.test_tools import assert_eq_with_retry
assert_eq_with_retry(self, "select 1", "1", retry_count=int(start_wait_sec / 0.5), sleep_time=0.5)
@ -1263,7 +1277,7 @@ class ClickHouseInstance:
self.exec_in_container(["bash", "-c",
"cp /usr/share/clickhouse-odbc-bridge_fresh /usr/bin/clickhouse-odbc-bridge && chmod 777 /usr/bin/clickhouse"],
user='root')
self.exec_in_container(["bash", "-c", "{} --daemon".format(CLICKHOUSE_START_COMMAND)], user=str(os.getuid()))
self.exec_in_container(["bash", "-c", "{} --daemon".format(self.clickhouse_start_command)], user=str(os.getuid()))
from helpers.test_tools import assert_eq_with_retry
# wait start
assert_eq_with_retry(self, "select 1", "1", retry_count=retries)
@ -1404,8 +1418,10 @@ class ClickHouseInstance:
os.makedirs(instance_config_dir)
print("Copy common default production configuration from {}".format(self.base_config_dir))
shutil.copyfile(p.join(self.base_config_dir, 'config.xml'), p.join(instance_config_dir, 'config.xml'))
shutil.copyfile(p.join(self.base_config_dir, 'users.xml'), p.join(instance_config_dir, 'users.xml'))
shutil.copyfile(p.join(self.base_config_dir, self.main_config_name), p.join(instance_config_dir, self.main_config_name))
shutil.copyfile(p.join(self.base_config_dir, self.users_config_name), p.join(instance_config_dir, self.users_config_name))
print("Create directory for configuration generated in this helper")
# used by all utils with any config
@ -1423,7 +1439,9 @@ class ClickHouseInstance:
print("Copy common configuration from helpers")
# The file is named with 0_ prefix to be processed before other configuration overloads.
shutil.copy(p.join(HELPERS_DIR, '0_common_instance_config.xml'), self.config_d_dir)
if self.copy_common_configs:
shutil.copy(p.join(HELPERS_DIR, '0_common_instance_config.xml'), self.config_d_dir)
shutil.copy(p.join(HELPERS_DIR, '0_common_instance_users.xml'), users_d_dir)
if len(self.custom_dictionaries_paths):
shutil.copy(p.join(HELPERS_DIR, '0_common_enable_dictionaries.xml'), self.config_d_dir)
@ -1502,11 +1520,11 @@ class ClickHouseInstance:
self._create_odbc_config_file()
odbc_ini_path = '- ' + self.odbc_ini_path
entrypoint_cmd = CLICKHOUSE_START_COMMAND
entrypoint_cmd = self.clickhouse_start_command
if self.stay_alive:
entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND
entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND.replace("{main_config_file}", self.main_config_name)
print("Entrypoint cmd: {}".format(entrypoint_cmd))
networks = app_net = ipv4_address = ipv6_address = net_aliases = net_alias1 = ""

View File

@ -70,11 +70,11 @@ def check_args_and_update_paths(args):
if not os.path.exists(path):
raise Exception("Path {} doesn't exist".format(path))
if not os.path.exists(os.path.join(args.base_configs_dir, "config.xml")):
raise Exception("No configs.xml in {}".format(args.base_configs_dir))
if (not os.path.exists(os.path.join(args.base_configs_dir, "config.xml"))) and (not os.path.exists(os.path.join(args.base_configs_dir, "config.yaml"))):
raise Exception("No configs.xml or configs.yaml in {}".format(args.base_configs_dir))
if not os.path.exists(os.path.join(args.base_configs_dir, "users.xml")):
raise Exception("No users.xml in {}".format(args.base_configs_dir))
if (not os.path.exists(os.path.join(args.base_configs_dir, "users.xml"))) and (not os.path.exists(os.path.join(args.base_configs_dir, "users.yaml"))):
raise Exception("No users.xml or users.yaml in {}".format(args.base_configs_dir))
def docker_kill_handler_handler(signum, frame):
subprocess.check_call('docker kill $(docker ps -a -q --filter name={name} --format="{{{{.ID}}}}")'.format(name=CONTAINER_NAME), shell=True)

View File

@ -0,0 +1,13 @@
<yandex>
<!-- Sources to read users, roles, access rights, profiles of settings, quotas. -->
<user_directories replace="replace">
<users_xml>
<!-- Path to configuration file with predefined users. -->
<path>users.xml</path>
</users_xml>
<local_directory>
<!-- Path to folder where users created by SQL commands are stored. -->
<path>access/</path>
</local_directory>
</user_directories>
</yandex>

View File

@ -0,0 +1,23 @@
<yandex>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<coordination_settings>
<operation_timeout_ms>10000</operation_timeout_ms>
<session_timeout_ms>30000</session_timeout_ms>
<force_sync>false</force_sync>
<startup_timeout>60000</startup_timeout>
<!-- we want all logs for complex problems investigation -->
<reserved_log_items>1000000000000000</reserved_log_items>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>localhost</hostname>
<port>44444</port>
</server>
</raft_configuration>
</keeper_server>
</yandex>

View File

@ -0,0 +1,7 @@
<yandex>
<logger>
<console>true</console>
<log remove="remove"/>
<errorlog remove="remove"/>
</logger>
</yandex>

View File

@ -0,0 +1,8 @@
<yandex>
<logger>
<!-- Disable rotation
https://pocoproject.org/docs/Poco.FileChannel.html
-->
<size>never</size>
</logger>
</yandex>

View File

@ -0,0 +1,9 @@
<yandex>
<macros>
<test>Hello, world!</test>
<shard>s1</shard>
<replica>r1</replica>
<default_path_test>/clickhouse/tables/{database}/{shard}/</default_path_test>
<default_name_test>table_{table}</default_name_test>
</macros>
</yandex>

View File

@ -0,0 +1,8 @@
<yandex>
<metric_log>
<database>system</database>
<table>metric_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
</metric_log>
</yandex>

View File

@ -0,0 +1,49 @@
<yandex>
<remote_servers>
<![CDATA[
You can run additional servers simply as
./clickhouse-server -- --path=9001 --tcp_port=9001
]]>
<single_remote_shard_at_port_9001>
<shard>
<replica>
<host>localhost</host>
<port>9001</port>
</replica>
</shard>
</single_remote_shard_at_port_9001>
<two_remote_shards_at_port_9001_9002>
<shard>
<replica>
<host>localhost</host>
<port>9001</port>
</replica>
</shard>
<shard>
<replica>
<host>localhost</host>
<port>9002</port>
</replica>
</shard>
</two_remote_shards_at_port_9001_9002>
<two_shards_one_local_one_remote_at_port_9001>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>localhost</host>
<port>9001</port>
</replica>
</shard>
</two_shards_one_local_one_remote_at_port_9001>
</remote_servers>
</yandex>

View File

@ -0,0 +1,8 @@
<yandex>
<part_log>
<database>system</database>
<table>part_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</part_log>
</yandex>

View File

@ -0,0 +1,8 @@
<yandex>
<path replace="replace">./</path>
<tmp_path replace="replace">./tmp/</tmp_path>
<user_files_path replace="replace">./user_files/</user_files_path>
<format_schema_path replace="replace">./format_schemas/</format_schema_path>
<access_control_path replace="replace">./access/</access_control_path>
<top_level_domains_path replace="replace">./top_level_domains/</top_level_domains_path>
</yandex>

View File

@ -0,0 +1,10 @@
<?xml version="1.0"?>
<!-- Config for test server -->
<yandex>
<query_masking_rules>
<rule>
<regexp>TOPSECRET.TOPSECRET</regexp>
<replace>[hidden]</replace>
</rule>
</query_masking_rules>
</yandex>

View File

@ -0,0 +1,3 @@
<yandex>
<tcp_with_proxy_port>9010</tcp_with_proxy_port>
</yandex>

View File

@ -0,0 +1,7 @@
<yandex>
<text_log>
<database>system</database>
<table>text_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</text_log>
</yandex>

View File

@ -0,0 +1,8 @@
<yandex>
<zookeeper>
<node index="1">
<host>localhost</host>
<port>9181</port>
</node>
</zookeeper>
</yandex>

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,40 @@
<?xml version="1.0"?>
<!-- Config that is used when server is run without config file. -->
<yandex>
<logger>
<level>trace</level>
<console>true</console>
</logger>
<http_port>8123</http_port>
<tcp_port>9000</tcp_port>
<mysql_port>9004</mysql_port>
<path>./</path>
<uncompressed_cache_size>8589934592</uncompressed_cache_size>
<mark_cache_size>5368709120</mark_cache_size>
<mlock_executable>true</mlock_executable>
<users>
<default>
<password></password>
<networks>
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
<access_management>1</access_management>
</default>
</users>
<profiles>
<default/>
</profiles>
<quotas>
<default />
</quotas>
</yandex>

View File

@ -0,0 +1,8 @@
<?xml version="1.0"?>
<yandex>
<profiles>
<default>
<allow_introspection_functions>1</allow_introspection_functions>
</default>
</profiles>
</yandex>

View File

@ -0,0 +1,7 @@
<yandex>
<profiles>
<default>
<log_queries>1</log_queries>
</default>
</profiles>
</yandex>

View File

@ -0,0 +1,120 @@
<?xml version="1.0"?>
<yandex>
<!-- Profiles of settings. -->
<profiles>
<!-- Default settings. -->
<default>
<!-- Maximum memory usage for processing single query, in bytes. -->
<max_memory_usage>10000000000</max_memory_usage>
<max_block_size>64999</max_block_size>
<!-- How to choose between replicas during distributed query processing.
random - choose random replica from set of replicas with minimum number of errors
nearest_hostname - from set of replicas with minimum number of errors, choose replica
with minimum number of different symbols between replica's hostname and local hostname
(Hamming distance).
in_order - first live replica is chosen in specified order.
first_or_random - if first replica one has higher number of errors, pick a random one from replicas with minimum number of errors.
-->
<load_balancing>random</load_balancing>
</default>
<!-- Profile that allows only read queries. -->
<readonly>
<readonly>1</readonly>
</readonly>
</profiles>
<!-- Users and ACL. -->
<users>
<!-- If user name was not specified, 'default' user is used. -->
<default>
<!-- Password could be specified in plaintext or in SHA256 (in hex format).
If you want to specify password in plaintext (not recommended), place it in 'password' element.
Example: <password>qwerty</password>.
Password could be empty.
If you want to specify SHA256, place it in 'password_sha256_hex' element.
Example: <password_sha256_hex>65e84be33532fb784c48129675f9eff3a682b27168c0ea744b2cf58ee02337c5</password_sha256_hex>
Restrictions of SHA256: impossibility to connect to ClickHouse using MySQL JS client (as of July 2019).
If you want to specify double SHA1, place it in 'password_double_sha1_hex' element.
Example: <password_double_sha1_hex>e395796d6546b1b65db9d665cd43f0e858dd4303</password_double_sha1_hex>
If you want to specify a previously defined LDAP server (see 'ldap_servers' in the main config) for authentication,
place its name in 'server' element inside 'ldap' element.
Example: <ldap><server>my_ldap_server</server></ldap>
If you want to authenticate the user via Kerberos (assuming Kerberos is enabled, see 'kerberos' in the main config),
place 'kerberos' element instead of 'password' (and similar) elements.
The name part of the canonical principal name of the initiator must match the user name for authentication to succeed.
You can also place 'realm' element inside 'kerberos' element to further restrict authentication to only those requests
whose initiator's realm matches it.
Example: <kerberos />
Example: <kerberos><realm>EXAMPLE.COM</realm></kerberos>
How to generate decent password:
Execute: PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | sha256sum | tr -d '-'
In first line will be password and in second - corresponding SHA256.
How to generate double SHA1:
Execute: PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | sha1sum | tr -d '-' | xxd -r -p | sha1sum | tr -d '-'
In first line will be password and in second - corresponding double SHA1.
-->
<password></password>
<!-- List of networks with open access.
To open access from everywhere, specify:
<ip>::/0</ip>
To open access only from localhost, specify:
<ip>::1</ip>
<ip>127.0.0.1</ip>
Each element of list has one of the following forms:
<ip> IP-address or network mask. Examples: 213.180.204.3 or 10.0.0.1/8 or 10.0.0.1/255.255.255.0
2a02:6b8::3 or 2a02:6b8::3/64 or 2a02:6b8::3/ffff:ffff:ffff:ffff::.
<host> Hostname. Example: server01.yandex.ru.
To check access, DNS query is performed, and all received addresses compared to peer address.
<host_regexp> Regular expression for host names. Example, ^server\d\d-\d\d-\d\.yandex\.ru$
To check access, DNS PTR query is performed for peer address and then regexp is applied.
Then, for result of PTR query, another DNS query is performed and all received addresses compared to peer address.
Strongly recommended that regexp is ends with $
All results of DNS requests are cached till server restart.
-->
<networks>
<ip>::/0</ip>
</networks>
<!-- Settings profile for user. -->
<profile>default</profile>
<!-- Quota for user. -->
<quota>default</quota>
<!-- User can create other users and grant rights to them. -->
<!-- <access_management>1</access_management> -->
</default>
</users>
<!-- Quotas. -->
<quotas>
<!-- Name of quota. -->
<default>
<!-- Limits for time interval. You could specify many intervals with different limits. -->
<interval>
<!-- Length of interval. -->
<duration>3600</duration>
<!-- No limits. Just calculate resource usage for time interval. -->
<queries>0</queries>
<errors>0</errors>
<result_rows>0</result_rows>
<read_rows>0</read_rows>
<execution_time>0</execution_time>
</interval>
</default>
</quotas>
</yandex>

View File

@ -0,0 +1,40 @@
import time
import threading
from os import path as p, unlink
from tempfile import NamedTemporaryFile
import helpers
import pytest
from helpers.cluster import ClickHouseCluster
def test_xml_full_conf():
# all configs are in XML
cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/config.d/zookeeper.xml')
all_confd = ['configs/config.d/access_control.xml',
'configs/config.d/keeper_port.xml',
'configs/config.d/logging_no_rotate.xml',
'configs/config.d/log_to_console.xml',
'configs/config.d/macros.xml',
'configs/config.d/metric_log.xml',
'configs/config.d/more_clusters.xml',
'configs/config.d/part_log.xml',
'configs/config.d/path.xml',
'configs/config.d/query_masking_rules.xml',
'configs/config.d/tcp_with_proxy.xml',
'configs/config.d/text_log.xml',
'configs/config.d/zookeeper.xml']
all_userd = ['configs/users.d/allow_introspection_functions.xml',
'configs/users.d/log_queries.xml']
node = cluster.add_instance('node', base_config_dir='configs', main_configs=all_confd, user_configs=all_userd, with_zookeeper=False)
try:
cluster.start()
assert(node.query("select value from system.settings where name = 'max_memory_usage'") == "10000000000\n")
assert(node.query("select value from system.settings where name = 'max_block_size'") == "64999\n")
finally:
cluster.shutdown()

View File

@ -0,0 +1,7 @@
user_directories:
users_xml:
path: users.xml
local_directory:
path: access/
"@replace": replace

View File

@ -0,0 +1,15 @@
keeper_server:
tcp_port: 9181
server_id: 1
coordination_settings:
operation_timeout_ms: 10000
session_timeout_ms: 30000
force_sync: false
startup_timeout: 60000
reserved_log_items: 1000000000000000
raft_configuration:
server:
id: 1
hostname: localhost
port: 44444

View File

@ -0,0 +1,7 @@
logger:
console: true
log:
"@remove": remove
errorlog:
"@remove": remove

View File

@ -0,0 +1,2 @@
logger:
size: never

View File

@ -0,0 +1,7 @@
macros:
test: 'Hello, world!'
shard: s1
replica: r1
default_path_test: '/clickhouse/tables/{database}/{shard}/'
default_name_test: 'table_{table}'

View File

@ -0,0 +1,6 @@
metric_log:
database: system
table: metric_log
flush_interval_milliseconds: 7500
collect_interval_milliseconds: 1000

View File

@ -0,0 +1,23 @@
remote_servers:
single_remote_shard_at_port_9001:
shard:
replica:
host: localhost
port: 9001
two_remote_shards_at_port_9001_9002:
shard:
- replica:
host: localhost
port: 9001
- replica:
host: localhost
port: 9002
two_shards_one_local_one_remote_at_port_9001:
shard:
- replica:
host: localhost
port: 9000
- replica:
host: localhost
port: 9001

View File

@ -0,0 +1,5 @@
part_log:
database: system
table: part_log
flush_interval_milliseconds: 7500

View File

@ -0,0 +1,18 @@
path:
- ./
- "@replace": replace
tmp_path:
- ./tmp/
- "@replace": replace
user_files_path:
- ./user_files/
- "@replace": replace
format_schema_path:
- ./format_schemas/
- "@replace": replace
access_control_path:
- ./access/
- "@replace": replace
top_level_domains_path:
- ./top_level_domains/
- "@replace": replace

View File

@ -0,0 +1,4 @@
query_masking_rules:
rule:
regexp: TOPSECRET.TOPSECRET
replace: '[hidden]'

View File

@ -0,0 +1 @@
tcp_with_proxy_port: 9010

View File

@ -0,0 +1,11 @@
remote_servers:
test_cluster_with_incorrect_pw:
shard:
internal_replication: true
replica:
- host: 127.0.0.1
port: 9000
password: foo
- host: 127.0.0.2
port: 9000
password: foo

View File

@ -0,0 +1,4 @@
text_log:
database: system
table: text_log
flush_interval_milliseconds: 7500

View File

@ -0,0 +1,5 @@
zookeeper:
node:
host: localhost
port: 9181
"@index": 1

View File

@ -0,0 +1,277 @@
<?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>
<http_port>8123</http_port>
<tcp_port>9000</tcp_port>
<mysql_port>9004</mysql_port>
<postgresql_port>9005</postgresql_port>
<interserver_http_port>9009</interserver_http_port>
<max_connections>4096</max_connections>
<keep_alive_timeout>3</keep_alive_timeout>
<grpc>
<enable_ssl>false</enable_ssl>
<ssl_cert_file>/path/to/ssl_cert_file</ssl_cert_file>
<ssl_key_file>/path/to/ssl_key_file</ssl_key_file>
<ssl_require_client_auth>false</ssl_require_client_auth>
<ssl_ca_cert_file>/path/to/ssl_ca_cert_file</ssl_ca_cert_file>
<compression>deflate</compression>
<compression_level>medium</compression_level>
<max_send_message_size>-1</max_send_message_size>
<max_receive_message_size>-1</max_receive_message_size>
<verbose_logs>false</verbose_logs>
</grpc>
<openSSL>
<server>
<certificateFile>/etc/clickhouse-server/server.crt</certificateFile>
<privateKeyFile>/etc/clickhouse-server/server.key</privateKeyFile>
<dhParamsFile>/etc/clickhouse-server/dhparam.pem</dhParamsFile>
<verificationMode>none</verificationMode>
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
</server>
<client>
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
<invalidCertificateHandler>
<name>RejectCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
<max_concurrent_queries>100</max_concurrent_queries>
<max_server_memory_usage>0</max_server_memory_usage>
<max_thread_pool_size>10000</max_thread_pool_size>
<max_server_memory_usage_to_ram_ratio>0.9</max_server_memory_usage_to_ram_ratio>
<total_memory_profiler_step>4194304</total_memory_profiler_step>
<total_memory_tracker_sample_probability>0</total_memory_tracker_sample_probability>
<uncompressed_cache_size>8589934592</uncompressed_cache_size>
<mark_cache_size>5368709120</mark_cache_size>
<mmap_cache_size>1000</mmap_cache_size>
<path>/var/lib/clickhouse/</path>
<tmp_path>/var/lib/clickhouse/tmp/</tmp_path>
<user_files_path>/var/lib/clickhouse/user_files/</user_files_path>
<ldap_servers>
</ldap_servers>
<user_directories>
<users_xml>
<path>users.xml</path>
</users_xml>
<local_directory>
<path>/var/lib/clickhouse/access/</path>
</local_directory>
</user_directories>
<default_profile>default</default_profile>
<custom_settings_prefixes></custom_settings_prefixes>
<default_database>default</default_database>
<mlock_executable>true</mlock_executable>
<remap_executable>false</remap_executable>
<remote_servers>
<test_shard_localhost>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
</test_shard_localhost>
<test_cluster_two_shards_localhost>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
</test_cluster_two_shards_localhost>
<test_cluster_two_shards>
<shard>
<replica>
<host>127.0.0.1</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>127.0.0.2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster_two_shards>
<test_cluster_two_shards_internal_replication>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>127.0.0.1</host>
<port>9000</port>
</replica>
</shard>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>127.0.0.2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster_two_shards_internal_replication>
<test_shard_localhost_secure>
<shard>
<replica>
<host>localhost</host>
<port>9440</port>
<secure>1</secure>
</replica>
</shard>
</test_shard_localhost_secure>
<test_unavailable_shard>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>localhost</host>
<port>1</port>
</replica>
</shard>
</test_unavailable_shard>
</remote_servers>
<builtin_dictionaries_reload_interval>3600</builtin_dictionaries_reload_interval>
<max_session_timeout>3600</max_session_timeout>
<default_session_timeout>60</default_session_timeout>
<query_log>
<database>system</database>
<table>query_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</query_log>
<trace_log>
<database>system</database>
<table>trace_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</trace_log>
<query_thread_log>
<database>system</database>
<table>query_thread_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</query_thread_log>
<metric_log>
<database>system</database>
<table>metric_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
</metric_log>
<asynchronous_metric_log>
<database>system</database>
<table>asynchronous_metric_log</table>
<flush_interval_milliseconds>60000</flush_interval_milliseconds>
</asynchronous_metric_log>
<opentelemetry_span_log>
<engine>
engine MergeTree
partition by toYYYYMM(finish_date)
order by (finish_date, finish_time_us, trace_id)
</engine>
<database>system</database>
<table>opentelemetry_span_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</opentelemetry_span_log>
<crash_log>
<database>system</database>
<table>crash_log</table>
<partition_by />
<flush_interval_milliseconds>1000</flush_interval_milliseconds>
</crash_log>
<top_level_domains_lists>
</top_level_domains_lists>
<dictionaries_config>*_dictionary.xml</dictionaries_config>
<distributed_ddl>
<path>/clickhouse/task_queue/ddl</path>
</distributed_ddl>
<graphite_rollup_example>
<pattern>
<regexp>click_cost</regexp>
<function>any</function>
<retention>
<age>0</age>
<precision>3600</precision>
</retention>
<retention>
<age>86400</age>
<precision>60</precision>
</retention>
</pattern>
<default>
<function>max</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>3600</age>
<precision>300</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</default>
</graphite_rollup_example>
<format_schema_path>/var/lib/clickhouse/format_schemas/</format_schema_path>
<query_masking_rules>
<rule>
<name>hide encrypt/decrypt arguments</name>
<regexp>((?:aes_)?(?:encrypt|decrypt)(?:_mysql)?)\s*\(\s*(?:'(?:\\'|.)+'|.*?)\s*\)</regexp>
<replace>\1(???)</replace>
</rule>
</query_masking_rules>
<send_crash_reports>
<enabled>false</enabled>
<anonymize>false</anonymize>
<endpoint>https://6f33034cfe684dd7a3ab9875e57b1c8d@o388870.ingest.sentry.io/5226277</endpoint>
</send_crash_reports>
</yandex>

View File

@ -0,0 +1,40 @@
<?xml version="1.0"?>
<!-- Config that is used when server is run without config file. -->
<yandex>
<logger>
<level>trace</level>
<console>true</console>
</logger>
<http_port>8123</http_port>
<tcp_port>9000</tcp_port>
<mysql_port>9004</mysql_port>
<path>./</path>
<uncompressed_cache_size>8589934592</uncompressed_cache_size>
<mark_cache_size>5368709120</mark_cache_size>
<mlock_executable>true</mlock_executable>
<users>
<default>
<password></password>
<networks>
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
<access_management>1</access_management>
</default>
</users>
<profiles>
<default/>
</profiles>
<quotas>
<default />
</quotas>
</yandex>

View File

@ -0,0 +1,3 @@
profiles:
default:
allow_introspection_functions: 1

View File

@ -0,0 +1,3 @@
profiles:
default:
log_queries: 1

View File

@ -0,0 +1,19 @@
<?xml version="1.0"?>
<yandex>
<profiles>
<default>
<max_memory_usage>10000000000</max_memory_usage>
<max_block_size>64999</max_block_size>
</default>
</profiles>
<users>
<default>
<password></password>
<networks replace="replace">
<ip>::/0</ip>
</networks>
<profile>default</profile>
</default>
</users>
</yandex>

View File

@ -0,0 +1,43 @@
import time
import threading
from os import path as p, unlink
from tempfile import NamedTemporaryFile
import helpers
import pytest
from helpers.cluster import ClickHouseCluster
def test_xml_main_conf():
# main configs are in XML; config.d and users.d are in YAML
cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/config.d/zookeeper.yaml')
all_confd = ['configs/config.d/access_control.yaml',
'configs/config.d/keeper_port.yaml',
'configs/config.d/logging_no_rotate.yaml',
'configs/config.d/log_to_console.yaml',
'configs/config.d/macros.yaml',
'configs/config.d/metric_log.yaml',
'configs/config.d/more_clusters.yaml',
'configs/config.d/part_log.yaml',
'configs/config.d/path.yaml',
'configs/config.d/query_masking_rules.yaml',
'configs/config.d/tcp_with_proxy.yaml',
'configs/config.d/test_cluster_with_incorrect_pw.yaml',
'configs/config.d/text_log.yaml',
'configs/config.d/zookeeper.yaml']
all_userd = ['configs/users.d/allow_introspection_functions.yaml',
'configs/users.d/log_queries.yaml']
node = cluster.add_instance('node', base_config_dir='configs', main_configs=all_confd, user_configs=all_userd, with_zookeeper=False)
try:
cluster.start()
assert(node.query("select value from system.settings where name = 'max_memory_usage'") == "10000000000\n")
assert(node.query("select value from system.settings where name = 'max_block_size'") == "64999\n")
finally:
cluster.shutdown()

View File

@ -0,0 +1,6 @@
timezone: Europe/Moscow
listen_host: 0.0.0.0
custom_settings_prefixes: custom_
path: /var/lib/clickhouse/
tmp_path: /var/lib/clickhouse/tmp/
users_config: users.yaml

View File

@ -0,0 +1,7 @@
user_directories:
users_xml:
path: users.yaml
local_directory:
path: access/
"@replace": replace

View File

@ -0,0 +1,23 @@
<yandex>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<coordination_settings>
<operation_timeout_ms>10000</operation_timeout_ms>
<session_timeout_ms>30000</session_timeout_ms>
<force_sync>false</force_sync>
<startup_timeout>60000</startup_timeout>
<!-- we want all logs for complex problems investigation -->
<reserved_log_items>1000000000000000</reserved_log_items>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>localhost</hostname>
<port>44444</port>
</server>
</raft_configuration>
</keeper_server>
</yandex>

View File

@ -0,0 +1,7 @@
logger:
console: true
log:
"@remove": remove
errorlog:
"@remove": remove

View File

@ -0,0 +1,8 @@
<yandex>
<logger>
<!-- Disable rotation
https://pocoproject.org/docs/Poco.FileChannel.html
-->
<size>never</size>
</logger>
</yandex>

View File

@ -0,0 +1,7 @@
macros:
test: 'Hello, world!'
shard: s1
replica: r1
default_path_test: '/clickhouse/tables/{database}/{shard}/'
default_name_test: 'table_{table}'

View File

@ -0,0 +1,8 @@
<yandex>
<metric_log>
<database>system</database>
<table>metric_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
</metric_log>
</yandex>

View File

@ -0,0 +1,23 @@
remote_servers:
single_remote_shard_at_port_9001:
shard:
replica:
host: localhost
port: 9001
two_remote_shards_at_port_9001_9002:
shard:
- replica:
host: localhost
port: 9001
- replica:
host: localhost
port: 9002
two_shards_one_local_one_remote_at_port_9001:
shard:
- replica:
host: localhost
port: 9000
- replica:
host: localhost
port: 9001

View File

@ -0,0 +1,8 @@
<yandex>
<part_log>
<database>system</database>
<table>part_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</part_log>
</yandex>

View File

@ -0,0 +1,18 @@
path:
- ./
- "@replace": replace
tmp_path:
- ./tmp/
- "@replace": replace
user_files_path:
- ./user_files/
- "@replace": replace
format_schema_path:
- ./format_schemas/
- "@replace": replace
access_control_path:
- ./access/
- "@replace": replace
top_level_domains_path:
- ./top_level_domains/
- "@replace": replace

View File

@ -0,0 +1,10 @@
<?xml version="1.0"?>
<!-- Config for test server -->
<yandex>
<query_masking_rules>
<rule>
<regexp>TOPSECRET.TOPSECRET</regexp>
<replace>[hidden]</replace>
</rule>
</query_masking_rules>
</yandex>

View File

@ -0,0 +1 @@
tcp_with_proxy_port: 9010

View File

@ -0,0 +1,21 @@
<yandex>
<remote_servers>
<test_cluster_with_incorrect_pw>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>127.0.0.1</host>
<port>9000</port>
<!-- password is incorrect -->
<password>foo</password>
</replica>
<replica>
<host>127.0.0.2</host>
<port>9000</port>
<!-- password is incorrect -->
<password>foo</password>
</replica>
</shard>
</test_cluster_with_incorrect_pw>
</remote_servers>
</yandex>

View File

@ -0,0 +1,4 @@
text_log:
database: system
table: text_log
flush_interval_milliseconds: 7500

View File

@ -0,0 +1,8 @@
<yandex>
<zookeeper>
<node index="1">
<host>localhost</host>
<port>9181</port>
</node>
</zookeeper>
</yandex>

View File

@ -0,0 +1,277 @@
<?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>
<http_port>8123</http_port>
<tcp_port>9000</tcp_port>
<mysql_port>9004</mysql_port>
<postgresql_port>9005</postgresql_port>
<interserver_http_port>9009</interserver_http_port>
<max_connections>4096</max_connections>
<keep_alive_timeout>3</keep_alive_timeout>
<grpc>
<enable_ssl>false</enable_ssl>
<ssl_cert_file>/path/to/ssl_cert_file</ssl_cert_file>
<ssl_key_file>/path/to/ssl_key_file</ssl_key_file>
<ssl_require_client_auth>false</ssl_require_client_auth>
<ssl_ca_cert_file>/path/to/ssl_ca_cert_file</ssl_ca_cert_file>
<compression>deflate</compression>
<compression_level>medium</compression_level>
<max_send_message_size>-1</max_send_message_size>
<max_receive_message_size>-1</max_receive_message_size>
<verbose_logs>false</verbose_logs>
</grpc>
<openSSL>
<server>
<certificateFile>/etc/clickhouse-server/server.crt</certificateFile>
<privateKeyFile>/etc/clickhouse-server/server.key</privateKeyFile>
<dhParamsFile>/etc/clickhouse-server/dhparam.pem</dhParamsFile>
<verificationMode>none</verificationMode>
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
</server>
<client>
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
<invalidCertificateHandler>
<name>RejectCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
<max_concurrent_queries>100</max_concurrent_queries>
<max_server_memory_usage>0</max_server_memory_usage>
<max_thread_pool_size>10000</max_thread_pool_size>
<max_server_memory_usage_to_ram_ratio>0.9</max_server_memory_usage_to_ram_ratio>
<total_memory_profiler_step>4194304</total_memory_profiler_step>
<total_memory_tracker_sample_probability>0</total_memory_tracker_sample_probability>
<uncompressed_cache_size>8589934592</uncompressed_cache_size>
<mark_cache_size>5368709120</mark_cache_size>
<mmap_cache_size>1000</mmap_cache_size>
<path>/var/lib/clickhouse/</path>
<tmp_path>/var/lib/clickhouse/tmp/</tmp_path>
<user_files_path>/var/lib/clickhouse/user_files/</user_files_path>
<ldap_servers>
</ldap_servers>
<user_directories>
<users_xml>
<path>users.yaml</path>
</users_xml>
<local_directory>
<path>/var/lib/clickhouse/access/</path>
</local_directory>
</user_directories>
<default_profile>default</default_profile>
<custom_settings_prefixes></custom_settings_prefixes>
<default_database>default</default_database>
<mlock_executable>true</mlock_executable>
<remap_executable>false</remap_executable>
<remote_servers>
<test_shard_localhost>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
</test_shard_localhost>
<test_cluster_two_shards_localhost>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
</test_cluster_two_shards_localhost>
<test_cluster_two_shards>
<shard>
<replica>
<host>127.0.0.1</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>127.0.0.2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster_two_shards>
<test_cluster_two_shards_internal_replication>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>127.0.0.1</host>
<port>9000</port>
</replica>
</shard>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>127.0.0.2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster_two_shards_internal_replication>
<test_shard_localhost_secure>
<shard>
<replica>
<host>localhost</host>
<port>9440</port>
<secure>1</secure>
</replica>
</shard>
</test_shard_localhost_secure>
<test_unavailable_shard>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>localhost</host>
<port>1</port>
</replica>
</shard>
</test_unavailable_shard>
</remote_servers>
<builtin_dictionaries_reload_interval>3600</builtin_dictionaries_reload_interval>
<max_session_timeout>3600</max_session_timeout>
<default_session_timeout>60</default_session_timeout>
<query_log>
<database>system</database>
<table>query_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</query_log>
<trace_log>
<database>system</database>
<table>trace_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</trace_log>
<query_thread_log>
<database>system</database>
<table>query_thread_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</query_thread_log>
<metric_log>
<database>system</database>
<table>metric_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
</metric_log>
<asynchronous_metric_log>
<database>system</database>
<table>asynchronous_metric_log</table>
<flush_interval_milliseconds>60000</flush_interval_milliseconds>
</asynchronous_metric_log>
<opentelemetry_span_log>
<engine>
engine MergeTree
partition by toYYYYMM(finish_date)
order by (finish_date, finish_time_us, trace_id)
</engine>
<database>system</database>
<table>opentelemetry_span_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</opentelemetry_span_log>
<crash_log>
<database>system</database>
<table>crash_log</table>
<partition_by />
<flush_interval_milliseconds>1000</flush_interval_milliseconds>
</crash_log>
<top_level_domains_lists>
</top_level_domains_lists>
<dictionaries_config>*_dictionary.xml</dictionaries_config>
<distributed_ddl>
<path>/clickhouse/task_queue/ddl</path>
</distributed_ddl>
<graphite_rollup_example>
<pattern>
<regexp>click_cost</regexp>
<function>any</function>
<retention>
<age>0</age>
<precision>3600</precision>
</retention>
<retention>
<age>86400</age>
<precision>60</precision>
</retention>
</pattern>
<default>
<function>max</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>3600</age>
<precision>300</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</default>
</graphite_rollup_example>
<format_schema_path>/var/lib/clickhouse/format_schemas/</format_schema_path>
<query_masking_rules>
<rule>
<name>hide encrypt/decrypt arguments</name>
<regexp>((?:aes_)?(?:encrypt|decrypt)(?:_mysql)?)\s*\(\s*(?:'(?:\\'|.)+'|.*?)\s*\)</regexp>
<replace>\1(???)</replace>
</rule>
</query_masking_rules>
<send_crash_reports>
<enabled>false</enabled>
<anonymize>false</anonymize>
<endpoint>https://6f33034cfe684dd7a3ab9875e57b1c8d@o388870.ingest.sentry.io/5226277</endpoint>
</send_crash_reports>
</yandex>

View File

@ -0,0 +1,40 @@
<?xml version="1.0"?>
<!-- Config that is used when server is run without config file. -->
<yandex>
<logger>
<level>trace</level>
<console>true</console>
</logger>
<http_port>8123</http_port>
<tcp_port>9000</tcp_port>
<mysql_port>9004</mysql_port>
<path>./</path>
<uncompressed_cache_size>8589934592</uncompressed_cache_size>
<mark_cache_size>5368709120</mark_cache_size>
<mlock_executable>true</mlock_executable>
<users>
<default>
<password></password>
<networks>
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
<access_management>1</access_management>
</default>
</users>
<profiles>
<default/>
</profiles>
<quotas>
<default />
</quotas>
</yandex>

View File

@ -0,0 +1,8 @@
<?xml version="1.0"?>
<yandex>
<profiles>
<default>
<allow_introspection_functions>1</allow_introspection_functions>
</default>
</profiles>
</yandex>

View File

@ -0,0 +1,3 @@
profiles:
default:
log_queries: 1

View File

@ -0,0 +1,12 @@
profiles:
default:
max_memory_usage: 10000000000
max_block_size: 64999
users:
default:
password: ''
networks:
"@replace": replace
ip: '::/0'
profile: default

View File

@ -0,0 +1,43 @@
import time
import threading
from os import path as p, unlink
from tempfile import NamedTemporaryFile
import helpers
import pytest
from helpers.cluster import ClickHouseCluster
def test_extra_yaml_mix():
# some configs are written in XML, others are written in YAML
cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/config.d/zookeeper.xml')
all_confd = ['configs/config.d/0_common_instance_config.yaml',
'configs/config.d/access_control.yaml',
'configs/config.d/keeper_port.xml',
'configs/config.d/logging_no_rotate.xml',
'configs/config.d/log_to_console.yaml',
'configs/config.d/macros.yaml',
'configs/config.d/metric_log.xml',
'configs/config.d/more_clusters.yaml',
'configs/config.d/part_log.xml',
'configs/config.d/path.yaml',
'configs/config.d/query_masking_rules.xml',
'configs/config.d/tcp_with_proxy.yaml',
'configs/config.d/test_cluster_with_incorrect_pw.xml',
'configs/config.d/text_log.yaml',
'configs/config.d/zookeeper.xml']
all_userd = ['configs/users.d/allow_introspection_functions.xml',
'configs/users.d/log_queries.yaml']
node = cluster.add_instance('node', base_config_dir='configs', main_configs=all_confd, user_configs=all_userd, with_zookeeper=False,
users_config_name="users.yaml", copy_common_configs=False)
try:
cluster.start()
assert(node.query("select value from system.settings where name = 'max_memory_usage'") == "10000000000\n")
assert(node.query("select value from system.settings where name = 'max_block_size'") == "64999\n")
finally:
cluster.shutdown()

View File

@ -0,0 +1,6 @@
timezone: Europe/Moscow
listen_host: 0.0.0.0
custom_settings_prefixes: custom_
path: /var/lib/clickhouse/
tmp_path: /var/lib/clickhouse/tmp/
users_config: users.yaml

View File

@ -0,0 +1,7 @@
user_directories:
users_xml:
path: users.yaml
local_directory:
path: access/
"@replace": replace

View File

@ -0,0 +1,15 @@
keeper_server:
tcp_port: 9181
server_id: 1
coordination_settings:
operation_timeout_ms: 10000
session_timeout_ms: 30000
force_sync: false
startup_timeout: 60000
reserved_log_items: 1000000000000000
raft_configuration:
server:
id: 1
hostname: localhost
port: 44444

View File

@ -0,0 +1,7 @@
logger:
console: true
log:
"@remove": remove
errorlog:
"@remove": remove

View File

@ -0,0 +1,2 @@
logger:
size: never

View File

@ -0,0 +1,7 @@
macros:
test: 'Hello, world!'
shard: s1
replica: r1
default_path_test: '/clickhouse/tables/{database}/{shard}/'
default_name_test: 'table_{table}'

View File

@ -0,0 +1,6 @@
metric_log:
database: system
table: metric_log
flush_interval_milliseconds: 7500
collect_interval_milliseconds: 1000

View File

@ -0,0 +1,23 @@
remote_servers:
single_remote_shard_at_port_9001:
shard:
replica:
host: localhost
port: 9001
two_remote_shards_at_port_9001_9002:
shard:
- replica:
host: localhost
port: 9001
- replica:
host: localhost
port: 9002
two_shards_one_local_one_remote_at_port_9001:
shard:
- replica:
host: localhost
port: 9000
- replica:
host: localhost
port: 9001

View File

@ -0,0 +1,5 @@
part_log:
database: system
table: part_log
flush_interval_milliseconds: 7500

View File

@ -0,0 +1,18 @@
path:
- ./
- "@replace": replace
tmp_path:
- ./tmp/
- "@replace": replace
user_files_path:
- ./user_files/
- "@replace": replace
format_schema_path:
- ./format_schemas/
- "@replace": replace
access_control_path:
- ./access/
- "@replace": replace
top_level_domains_path:
- ./top_level_domains/
- "@replace": replace

View File

@ -0,0 +1,4 @@
query_masking_rules:
rule:
regexp: TOPSECRET.TOPSECRET
replace: '[hidden]'

View File

@ -0,0 +1 @@
tcp_with_proxy_port: 9010

View File

@ -0,0 +1,11 @@
remote_servers:
test_cluster_with_incorrect_pw:
shard:
internal_replication: true
replica:
- host: 127.0.0.1
port: 9000
password: foo
- host: 127.0.0.2
port: 9000
password: foo

View File

@ -0,0 +1,4 @@
text_log:
database: system
table: text_log
flush_interval_milliseconds: 7500

View File

@ -0,0 +1,5 @@
zookeeper:
node:
host: localhost
port: 9181
"@index": 1

View File

@ -0,0 +1,183 @@
logger:
level: trace
log: /var/log/clickhouse-server/clickhouse-server.log
errorlog: /var/log/clickhouse-server/clickhouse-server.err.log
size: 1000M
count: 10
http_port: 8123
tcp_port: 9000
mysql_port: 9004
postgresql_port: 9005
interserver_http_port: 9009
max_connections: 4096
keep_alive_timeout: 3
grpc:
enable_ssl: false
ssl_cert_file: /path/to/ssl_cert_file
ssl_key_file: /path/to/ssl_key_file
ssl_require_client_auth: false
ssl_ca_cert_file: /path/to/ssl_ca_cert_file
compression: deflate
compression_level: medium
max_send_message_size: -1
max_receive_message_size: -1
verbose_logs: false
openSSL:
server:
certificateFile: /etc/clickhouse-server/server.crt
privateKeyFile: /etc/clickhouse-server/server.key
dhParamsFile: /etc/clickhouse-server/dhparam.pem
verificationMode: none
loadDefaultCAFile: true
cacheSessions: true
disableProtocols: 'sslv2,sslv3'
preferServerCiphers: true
client:
loadDefaultCAFile: true
cacheSessions: true
disableProtocols: 'sslv2,sslv3'
preferServerCiphers: true
invalidCertificateHandler:
name: RejectCertificateHandler
max_concurrent_queries: 100
max_server_memory_usage: 0
max_thread_pool_size: 10000
max_server_memory_usage_to_ram_ratio: 0.9
total_memory_profiler_step: 4194304
total_memory_tracker_sample_probability: 0
uncompressed_cache_size: 8589934592
mark_cache_size: 5368709120
mmap_cache_size: 1000
path: /var/lib/clickhouse/
tmp_path: /var/lib/clickhouse/tmp/
user_files_path: /var/lib/clickhouse/user_files/
ldap_servers: ''
user_directories:
users_xml:
path: users.yaml
local_directory:
path: /var/lib/clickhouse/access/
default_profile: default
custom_settings_prefixes: ''
default_database: default
mlock_executable: true
remap_executable: false
remote_servers:
test_shard_localhost:
shard:
replica:
host: localhost
port: 9000
test_cluster_two_shards_localhost:
shard:
- replica:
host: localhost
port: 9000
- replica:
host: localhost
port: 9000
test_cluster_two_shards:
shard:
- replica:
host: 127.0.0.1
port: 9000
- replica:
host: 127.0.0.2
port: 9000
test_cluster_two_shards_internal_replication:
shard:
- internal_replication: true
replica:
host: 127.0.0.1
port: 9000
- internal_replication: true
replica:
host: 127.0.0.2
port: 9000
test_shard_localhost_secure:
shard:
replica:
host: localhost
port: 9440
secure: 1
test_unavailable_shard:
shard:
- replica:
host: localhost
port: 9000
- replica:
host: localhost
port: 1
builtin_dictionaries_reload_interval: 3600
max_session_timeout: 3600
default_session_timeout: 60
query_log:
database: system
table: query_log
partition_by: toYYYYMM(event_date)
flush_interval_milliseconds: 7500
trace_log:
database: system
table: trace_log
partition_by: toYYYYMM(event_date)
flush_interval_milliseconds: 7500
query_thread_log:
database: system
table: query_thread_log
partition_by: toYYYYMM(event_date)
flush_interval_milliseconds: 7500
metric_log:
database: system
table: metric_log
flush_interval_milliseconds: 7500
collect_interval_milliseconds: 1000
asynchronous_metric_log:
database: system
table: asynchronous_metric_log
flush_interval_milliseconds: 60000
opentelemetry_span_log:
engine: |-
engine MergeTree
partition by toYYYYMM(finish_date)
order by (finish_date, finish_time_us, trace_id)
database: system
table: opentelemetry_span_log
flush_interval_milliseconds: 7500
crash_log:
database: system
table: crash_log
partition_by: ''
flush_interval_milliseconds: 1000
top_level_domains_lists: ''
dictionaries_config: '*_dictionary.xml'
distributed_ddl:
path: /clickhouse/task_queue/ddl
graphite_rollup_example:
pattern:
regexp: click_cost
function: any
retention:
- age: 0
precision: 3600
- age: 86400
precision: 60
default:
function: max
retention:
- age: 0
precision: 60
- age: 3600
precision: 300
- age: 86400
precision: 3600
format_schema_path: /var/lib/clickhouse/format_schemas/
query_masking_rules:
rule:
name: hide encrypt/decrypt arguments
regexp: '((?:aes_)?(?:encrypt|decrypt)(?:_mysql)?)\s*\(\s*(?:''(?:\\''|.)+''|.*?)\s*\)'
replace: \1(???)
send_crash_reports:
enabled: false
anonymize: false
endpoint: 'https://6f33034cfe684dd7a3ab9875e57b1c8d@o388870.ingest.sentry.io/5226277'

View File

@ -0,0 +1,40 @@
<?xml version="1.0"?>
<!-- Config that is used when server is run without config file. -->
<yandex>
<logger>
<level>trace</level>
<console>true</console>
</logger>
<http_port>8123</http_port>
<tcp_port>9000</tcp_port>
<mysql_port>9004</mysql_port>
<path>./</path>
<uncompressed_cache_size>8589934592</uncompressed_cache_size>
<mark_cache_size>5368709120</mark_cache_size>
<mlock_executable>true</mlock_executable>
<users>
<default>
<password></password>
<networks>
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
<access_management>1</access_management>
</default>
</users>
<profiles>
<default/>
</profiles>
<quotas>
<default />
</quotas>
</yandex>

View File

@ -0,0 +1,3 @@
profiles:
default:
allow_introspection_functions: 1

View File

@ -0,0 +1,3 @@
profiles:
default:
log_queries: 1

Some files were not shown because too many files have changed in this diff Show More