mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-03 13:02:00 +00:00
Merge branch 'master' into fix_attach_mv
This commit is contained in:
commit
2479c80fb7
@ -11,6 +11,8 @@
|
||||
* Now replicas that are processing the `ALTER TABLE ATTACH PART[ITION]` command search in their `detached/` folders before fetching the data from other replicas. As an implementation detail, a new command `ATTACH_PART` is introduced in the replicated log. Parts are searched and compared by their checksums. [#18978](https://github.com/ClickHouse/ClickHouse/pull/18978) ([Mike Kot](https://github.com/myrrc)). **Note**:
|
||||
* `ATTACH PART[ITION]` queries may not work during cluster upgrade.
|
||||
* It's not possible to rollback to older ClickHouse version after executing `ALTER ... ATTACH` query in new version as the old servers would fail to pass the `ATTACH_PART` entry in the replicated log.
|
||||
* In this version, empty `<remote_url_allow_hosts></remote_url_allow_hosts>` will block all access to remote hosts while in previous versions it did nothing. If you want to keep old behaviour and you have empty `remote_url_allow_hosts` element in configuration file, remove it. [#20058](https://github.com/ClickHouse/ClickHouse/pull/20058) ([Vladimir Chebotarev](https://github.com/excitoon)).
|
||||
|
||||
|
||||
#### New Feature
|
||||
|
||||
@ -132,7 +134,6 @@
|
||||
* Fix receive and send timeouts and non-blocking read in secure socket. [#21429](https://github.com/ClickHouse/ClickHouse/pull/21429) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* `force_drop_table` flag didn't work for `MATERIALIZED VIEW`, it's fixed. Fixes [#18943](https://github.com/ClickHouse/ClickHouse/issues/18943). [#20626](https://github.com/ClickHouse/ClickHouse/pull/20626) ([tavplubix](https://github.com/tavplubix)).
|
||||
* Fix name clashes in `PredicateRewriteVisitor`. It caused incorrect `WHERE` filtration after full join. Close [#20497](https://github.com/ClickHouse/ClickHouse/issues/20497). [#20622](https://github.com/ClickHouse/ClickHouse/pull/20622) ([Vladimir](https://github.com/vdimir)).
|
||||
* Fixed open behavior of remote host filter in case when there is `remote_url_allow_hosts` section in configuration but no entries there. [#20058](https://github.com/ClickHouse/ClickHouse/pull/20058) ([Vladimir Chebotarev](https://github.com/excitoon)).
|
||||
|
||||
#### Build/Testing/Packaging Improvement
|
||||
|
||||
|
@ -7,8 +7,7 @@
|
||||
#include <condition_variable>
|
||||
|
||||
#include <common/defines.h>
|
||||
|
||||
#include <Common/MoveOrCopyIfThrow.h>
|
||||
#include <common/MoveOrCopyIfThrow.h>
|
||||
|
||||
/** Pool for limited size objects that cannot be used from different threads simultaneously.
|
||||
* The main use case is to have fixed size of objects that can be reused in difference threads during their lifetime
|
2
contrib/NuRaft
vendored
2
contrib/NuRaft
vendored
@ -1 +1 @@
|
||||
Subproject commit c35819f2c8a378d4ba88cc930c17bc20aeb875eb
|
||||
Subproject commit d2feb5978b979729a07c3ca76eaa4ab94cef4ceb
|
@ -366,6 +366,9 @@ function run_tests
|
||||
|
||||
# JSON functions
|
||||
01666_blns
|
||||
|
||||
# Depends on AWS
|
||||
01801_s3_cluster
|
||||
)
|
||||
|
||||
(time clickhouse-test --hung-check -j 8 --order=random --use-skip-list --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" -- "$FASTTEST_FOCUS" 2>&1 ||:) | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/test_log.txt"
|
||||
|
@ -1,7 +1,7 @@
|
||||
#!/usr/bin/env python3
|
||||
# -*- coding: utf-8 -*-
|
||||
from multiprocessing import cpu_count
|
||||
from subprocess import Popen, call, STDOUT
|
||||
from subprocess import Popen, call, check_output, STDOUT
|
||||
import os
|
||||
import sys
|
||||
import shutil
|
||||
@ -85,10 +85,27 @@ def prepare_for_hung_check():
|
||||
# Issue #21004, live views are experimental, so let's just suppress it
|
||||
call("""clickhouse client -q "KILL QUERY WHERE upper(query) LIKE 'WATCH %'" """, shell=True, stderr=STDOUT)
|
||||
|
||||
# Wait for last queries to finish if any, not longer than 120 seconds
|
||||
# Kill other queries which known to be slow
|
||||
# It's query from 01232_preparing_sets_race_condition_long, it may take up to 1000 seconds in slow builds
|
||||
call("""clickhouse client -q "KILL QUERY WHERE query LIKE 'insert into tableB select %'" """, shell=True, stderr=STDOUT)
|
||||
# Long query from 00084_external_agregation
|
||||
call("""clickhouse client -q "KILL QUERY WHERE query LIKE 'SELECT URL, uniq(SearchPhrase) AS u FROM test.hits GROUP BY URL ORDER BY u %'" """, shell=True, stderr=STDOUT)
|
||||
|
||||
# Wait for last queries to finish if any, not longer than 300 seconds
|
||||
call("""clickhouse client -q "select sleepEachRow((
|
||||
select maxOrDefault(120 - elapsed) + 1 from system.processes where query not like '%from system.processes%' and elapsed < 120
|
||||
) / 120) from numbers(120) format Null" """, shell=True, stderr=STDOUT)
|
||||
select maxOrDefault(300 - elapsed) + 1 from system.processes where query not like '%from system.processes%' and elapsed < 300
|
||||
) / 300) from numbers(300) format Null" """, shell=True, stderr=STDOUT)
|
||||
|
||||
# Even if all clickhouse-test processes are finished, there are probably some sh scripts,
|
||||
# which still run some new queries. Let's ignore them.
|
||||
try:
|
||||
query = """clickhouse client -q "SELECT count() FROM system.processes where where elapsed > 300" """
|
||||
output = check_output(query, shell=True, stderr=STDOUT).decode('utf-8').strip()
|
||||
if int(output) == 0:
|
||||
return False
|
||||
except:
|
||||
pass
|
||||
return True
|
||||
|
||||
if __name__ == "__main__":
|
||||
logging.basicConfig(level=logging.INFO, format='%(asctime)s %(message)s')
|
||||
@ -119,12 +136,12 @@ if __name__ == "__main__":
|
||||
|
||||
logging.info("All processes finished")
|
||||
if args.hung_check:
|
||||
prepare_for_hung_check()
|
||||
have_long_running_queries = prepare_for_hung_check()
|
||||
logging.info("Checking if some queries hung")
|
||||
cmd = "{} {} {}".format(args.test_cmd, "--hung-check", "00001_select_1")
|
||||
res = call(cmd, shell=True, stderr=STDOUT)
|
||||
hung_check_status = "No queries hung\tOK\n"
|
||||
if res != 0:
|
||||
if res != 0 and have_long_running_queries:
|
||||
logging.info("Hung check failed with exit code {}".format(res))
|
||||
hung_check_status = "Hung check failed\tFAIL\n"
|
||||
open(os.path.join(args.output_folder, "test_results.tsv"), 'w+').write(hung_check_status)
|
||||
|
@ -94,10 +94,10 @@ postgres=# INSERT INTO test (int_id, str, "float") VALUES (1,'test',2);
|
||||
INSERT 0 1
|
||||
|
||||
postgresql> SELECT * FROM test;
|
||||
int_id | int_nullable | float | str | float_nullable
|
||||
--------+--------------+-------+------+----------------
|
||||
1 | | 2 | test |
|
||||
(1 row)
|
||||
int_id | int_nullable | float | str | float_nullable
|
||||
--------+--------------+-------+------+----------------
|
||||
1 | | 2 | test |
|
||||
(1 row)
|
||||
```
|
||||
|
||||
Table in ClickHouse, retrieving data from the PostgreSQL table created above:
|
||||
|
@ -33,7 +33,7 @@ SELECT bitmapBuild([1, 2, 3, 4, 5]) AS res, toTypeName(res);
|
||||
|
||||
``` text
|
||||
┌─res─┬─toTypeName(bitmapBuild([1, 2, 3, 4, 5]))─────┐
|
||||
│ │ AggregateFunction(groupBitmap, UInt8) │
|
||||
│ │ AggregateFunction(groupBitmap, UInt8) │
|
||||
└─────┴──────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
|
@ -437,13 +437,13 @@ A [FixedString(16)](../../sql-reference/data-types/fixedstring.md) data type has
|
||||
**Example**
|
||||
|
||||
``` sql
|
||||
SELECT murmurHash3_128('example_string') AS MurmurHash3, toTypeName(MurmurHash3) AS type;
|
||||
SELECT hex(murmurHash3_128('example_string')) AS MurmurHash3, toTypeName(MurmurHash3) AS type;
|
||||
```
|
||||
|
||||
``` text
|
||||
┌─MurmurHash3──────┬─type────────────┐
|
||||
│ 6<EFBFBD>1<1C>4"S5KT<4B>~~q │ FixedString(16) │
|
||||
└──────────────────┴─────────────────┘
|
||||
┌─MurmurHash3──────────────────────┬─type───┐
|
||||
│ 368A1A311CB7342253354B548E7E7E71 │ String │
|
||||
└──────────────────────────────────┴────────┘
|
||||
```
|
||||
|
||||
## xxHash32, xxHash64 {#hash-functions-xxhash32}
|
||||
|
@ -65,9 +65,9 @@ postgres=# INSERT INTO test (int_id, str, "float") VALUES (1,'test',2);
|
||||
INSERT 0 1
|
||||
|
||||
postgresql> SELECT * FROM test;
|
||||
int_id | int_nullable | float | str | float_nullable
|
||||
--------+--------------+-------+------+----------------
|
||||
1 | | 2 | test |
|
||||
int_id | int_nullable | float | str | float_nullable
|
||||
--------+--------------+-------+------+----------------
|
||||
1 | | 2 | test |
|
||||
(1 row)
|
||||
```
|
||||
|
||||
|
@ -35,7 +35,7 @@ SELECT bitmapBuild([1, 2, 3, 4, 5]) AS res, toTypeName(res)
|
||||
|
||||
``` text
|
||||
┌─res─┬─toTypeName(bitmapBuild([1, 2, 3, 4, 5]))─────┐
|
||||
│ │ AggregateFunction(groupBitmap, UInt8) │
|
||||
│ │ AggregateFunction(groupBitmap, UInt8) │
|
||||
└─────┴──────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
|
@ -434,13 +434,13 @@ A [FixedString(16)](../../sql-reference/data-types/fixedstring.md) データ型
|
||||
**例**
|
||||
|
||||
``` sql
|
||||
SELECT murmurHash3_128('example_string') AS MurmurHash3, toTypeName(MurmurHash3) AS type
|
||||
SELECT hex(murmurHash3_128('example_string')) AS MurmurHash3, toTypeName(MurmurHash3) AS type;
|
||||
```
|
||||
|
||||
``` text
|
||||
┌─MurmurHash3──────┬─type────────────┐
|
||||
│ 6<EFBFBD>1<1C>4"S5KT<4B>~~q │ FixedString(16) │
|
||||
└──────────────────┴─────────────────┘
|
||||
┌─MurmurHash3──────────────────────┬─type───┐
|
||||
│ 368A1A311CB7342253354B548E7E7E71 │ String │
|
||||
└──────────────────────────────────┴────────┘
|
||||
```
|
||||
|
||||
## xxHash32,xxHash64 {#hash-functions-xxhash32}
|
||||
|
@ -94,10 +94,10 @@ postgres=# INSERT INTO test (int_id, str, "float") VALUES (1,'test',2);
|
||||
INSERT 0 1
|
||||
|
||||
postgresql> SELECT * FROM test;
|
||||
int_id | int_nullable | float | str | float_nullable
|
||||
--------+--------------+-------+------+----------------
|
||||
1 | | 2 | test |
|
||||
(1 row)
|
||||
int_id | int_nullable | float | str | float_nullable
|
||||
--------+--------------+-------+------+----------------
|
||||
1 | | 2 | test |
|
||||
(1 row)
|
||||
```
|
||||
|
||||
Таблица в ClickHouse, получение данных из PostgreSQL таблицы, созданной выше:
|
||||
|
@ -25,7 +25,7 @@ SELECT bitmapBuild([1, 2, 3, 4, 5]) AS res, toTypeName(res);
|
||||
|
||||
``` text
|
||||
┌─res─┬─toTypeName(bitmapBuild([1, 2, 3, 4, 5]))─────┐
|
||||
│ │ AggregateFunction(groupBitmap, UInt8) │
|
||||
│ │ AggregateFunction(groupBitmap, UInt8) │
|
||||
└─────┴──────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
|
@ -430,7 +430,7 @@ murmurHash3_128( expr )
|
||||
|
||||
**Аргументы**
|
||||
|
||||
- `expr` — [выражение](../syntax.md#syntax-expressions), возвращающее значение типа[String](../../sql-reference/functions/hash-functions.md).
|
||||
- `expr` — [выражение](../syntax.md#syntax-expressions), возвращающее значение типа [String](../../sql-reference/functions/hash-functions.md).
|
||||
|
||||
**Возвращаемое значение**
|
||||
|
||||
@ -439,13 +439,13 @@ murmurHash3_128( expr )
|
||||
**Пример**
|
||||
|
||||
``` sql
|
||||
SELECT murmurHash3_128('example_string') AS MurmurHash3, toTypeName(MurmurHash3) AS type;
|
||||
SELECT hex(murmurHash3_128('example_string')) AS MurmurHash3, toTypeName(MurmurHash3) AS type;
|
||||
```
|
||||
|
||||
``` text
|
||||
┌─MurmurHash3──────┬─type────────────┐
|
||||
│ 6<EFBFBD>1<1C>4"S5KT<4B>~~q │ FixedString(16) │
|
||||
└──────────────────┴─────────────────┘
|
||||
┌─MurmurHash3──────────────────────┬─type───┐
|
||||
│ 368A1A311CB7342253354B548E7E7E71 │ String │
|
||||
└──────────────────────────────────┴────────┘
|
||||
```
|
||||
|
||||
## xxHash32, xxHash64 {#hash-functions-xxhash32-xxhash64}
|
||||
|
@ -65,10 +65,10 @@ postgres=# INSERT INTO test (int_id, str, "float") VALUES (1,'test',2);
|
||||
INSERT 0 1
|
||||
|
||||
postgresql> SELECT * FROM test;
|
||||
int_id | int_nullable | float | str | float_nullable
|
||||
--------+--------------+-------+------+----------------
|
||||
1 | | 2 | test |
|
||||
(1 row)
|
||||
int_id | int_nullable | float | str | float_nullable
|
||||
--------+--------------+-------+------+----------------
|
||||
1 | | 2 | test |
|
||||
(1 row)
|
||||
```
|
||||
|
||||
Получение данных в ClickHouse:
|
||||
|
@ -109,7 +109,8 @@ def build_single_page_version(lang, args, nav, cfg):
|
||||
extra['single_page'] = True
|
||||
extra['is_amp'] = False
|
||||
|
||||
with open(os.path.join(args.docs_dir, lang, 'single.md'), 'w') as single_md:
|
||||
single_md_path = os.path.join(args.docs_dir, lang, 'single.md')
|
||||
with open(single_md_path, 'w') as single_md:
|
||||
concatenate(lang, args.docs_dir, single_md, nav)
|
||||
|
||||
with util.temp_dir() as site_temp:
|
||||
@ -221,3 +222,7 @@ def build_single_page_version(lang, args, nav, cfg):
|
||||
subprocess.check_call(' '.join(create_pdf_command), shell=True)
|
||||
|
||||
logging.info(f'Finished building single page version for {lang}')
|
||||
|
||||
if os.path.exists(single_md_path):
|
||||
os.unlink(single_md_path)
|
||||
|
@ -108,14 +108,6 @@ void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeo
|
||||
" UNION ALL "
|
||||
"SELECT cluster FROM system.clusters"
|
||||
" UNION ALL "
|
||||
"SELECT name FROM system.errors"
|
||||
" UNION ALL "
|
||||
"SELECT event FROM system.events"
|
||||
" UNION ALL "
|
||||
"SELECT metric FROM system.asynchronous_metrics"
|
||||
" UNION ALL "
|
||||
"SELECT metric FROM system.metrics"
|
||||
" UNION ALL "
|
||||
"SELECT macro FROM system.macros"
|
||||
" UNION ALL "
|
||||
"SELECT policy_name FROM system.storage_policies"
|
||||
@ -139,17 +131,12 @@ void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeo
|
||||
|
||||
query << ") WHERE notEmpty(res)";
|
||||
|
||||
Settings settings;
|
||||
/// To show all rows from:
|
||||
/// - system.errors
|
||||
/// - system.events
|
||||
settings.system_events_show_zero_values = true;
|
||||
fetch(connection, timeouts, query.str(), settings);
|
||||
fetch(connection, timeouts, query.str());
|
||||
}
|
||||
|
||||
void Suggest::fetch(Connection & connection, const ConnectionTimeouts & timeouts, const std::string & query, Settings & settings)
|
||||
void Suggest::fetch(Connection & connection, const ConnectionTimeouts & timeouts, const std::string & query)
|
||||
{
|
||||
connection.sendQuery(timeouts, query, "" /* query_id */, QueryProcessingStage::Complete, &settings);
|
||||
connection.sendQuery(timeouts, query, "" /* query_id */, QueryProcessingStage::Complete);
|
||||
|
||||
while (true)
|
||||
{
|
||||
|
@ -33,7 +33,7 @@ public:
|
||||
private:
|
||||
|
||||
void loadImpl(Connection & connection, const ConnectionTimeouts & timeouts, size_t suggestion_limit);
|
||||
void fetch(Connection & connection, const ConnectionTimeouts & timeouts, const std::string & query, Settings & settings);
|
||||
void fetch(Connection & connection, const ConnectionTimeouts & timeouts, const std::string & query);
|
||||
void fillWordsFromBlock(const Block & block);
|
||||
|
||||
/// Words are fetched asynchronously.
|
||||
|
@ -71,6 +71,9 @@ namespace ErrorCodes
|
||||
|
||||
}
|
||||
|
||||
/// ANSI escape sequence for intense color in terminal.
|
||||
#define HILITE "\033[1m"
|
||||
#define END_HILITE "\033[0m"
|
||||
|
||||
using namespace DB;
|
||||
namespace po = boost::program_options;
|
||||
@ -563,12 +566,12 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
|
||||
|
||||
if (has_password_for_default_user)
|
||||
{
|
||||
fmt::print("Password for default user is already specified. To remind or reset, see {} and {}.\n",
|
||||
fmt::print(HILITE "Password for default user is already specified. To remind or reset, see {} and {}." END_HILITE,
|
||||
users_config_file.string(), users_d.string());
|
||||
}
|
||||
else if (!is_interactive)
|
||||
{
|
||||
fmt::print("Password for default user is empty string. See {} and {} to change it.\n",
|
||||
fmt::print(HILITE "Password for default user is empty string. See {} and {} to change it." END_HILITE,
|
||||
users_config_file.string(), users_d.string());
|
||||
}
|
||||
else
|
||||
|
@ -1,6 +1,6 @@
|
||||
set (CLICKHOUSE_LIBRARY_BRIDGE_SOURCES
|
||||
library-bridge.cpp
|
||||
library-log.cpp
|
||||
LibraryInterface.cpp
|
||||
LibraryBridge.cpp
|
||||
Handlers.cpp
|
||||
HandlerFactory.cpp
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include "LibraryDictionarySourceExternal.h"
|
||||
#include "LibraryInterface.h"
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
namespace
|
@ -1,11 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Dictionaries/LibraryDictionarySourceExternal.h>
|
||||
#include <Core/Block.h>
|
||||
#include <ext/bit_cast.h>
|
||||
#include <ext/range.h>
|
||||
|
||||
#include "LibraryInterface.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,66 +0,0 @@
|
||||
#include <Dictionaries/LibraryDictionarySourceExternal.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
namespace
|
||||
{
|
||||
const char DICT_LOGGER_NAME[] = "LibraryDictionarySourceExternal";
|
||||
}
|
||||
|
||||
namespace ClickHouseLibrary
|
||||
{
|
||||
|
||||
std::string_view LIBRARY_CREATE_NEW_FUNC_NAME = "ClickHouseDictionary_v3_libNew";
|
||||
std::string_view LIBRARY_CLONE_FUNC_NAME = "ClickHouseDictionary_v3_libClone";
|
||||
std::string_view LIBRARY_DELETE_FUNC_NAME = "ClickHouseDictionary_v3_libDelete";
|
||||
|
||||
std::string_view LIBRARY_DATA_NEW_FUNC_NAME = "ClickHouseDictionary_v3_dataNew";
|
||||
std::string_view LIBRARY_DATA_DELETE_FUNC_NAME = "ClickHouseDictionary_v3_dataDelete";
|
||||
|
||||
std::string_view LIBRARY_LOAD_ALL_FUNC_NAME = "ClickHouseDictionary_v3_loadAll";
|
||||
std::string_view LIBRARY_LOAD_IDS_FUNC_NAME = "ClickHouseDictionary_v3_loadIds";
|
||||
std::string_view LIBRARY_LOAD_KEYS_FUNC_NAME = "ClickHouseDictionary_v3_loadKeys";
|
||||
|
||||
std::string_view LIBRARY_IS_MODIFIED_FUNC_NAME = "ClickHouseDictionary_v3_isModified";
|
||||
std::string_view LIBRARY_SUPPORTS_SELECTIVE_LOAD_FUNC_NAME = "ClickHouseDictionary_v3_supportsSelectiveLoad";
|
||||
|
||||
void log(LogLevel level, CString msg)
|
||||
{
|
||||
auto & logger = Poco::Logger::get(DICT_LOGGER_NAME);
|
||||
switch (level)
|
||||
{
|
||||
case LogLevel::TRACE:
|
||||
if (logger.trace())
|
||||
logger.trace(msg);
|
||||
break;
|
||||
case LogLevel::DEBUG:
|
||||
if (logger.debug())
|
||||
logger.debug(msg);
|
||||
break;
|
||||
case LogLevel::INFORMATION:
|
||||
if (logger.information())
|
||||
logger.information(msg);
|
||||
break;
|
||||
case LogLevel::NOTICE:
|
||||
if (logger.notice())
|
||||
logger.notice(msg);
|
||||
break;
|
||||
case LogLevel::WARNING:
|
||||
if (logger.warning())
|
||||
logger.warning(msg);
|
||||
break;
|
||||
case LogLevel::ERROR:
|
||||
if (logger.error())
|
||||
logger.error(msg);
|
||||
break;
|
||||
case LogLevel::CRITICAL:
|
||||
if (logger.critical())
|
||||
logger.critical(msg);
|
||||
break;
|
||||
case LogLevel::FATAL:
|
||||
if (logger.fatal())
|
||||
logger.fatal(msg);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -3,7 +3,7 @@
|
||||
#include <common/logger_useful.h>
|
||||
#include <nanodbc/nanodbc.h>
|
||||
#include <mutex>
|
||||
#include <Common/BorrowedObjectPool.h>
|
||||
#include <common/BorrowedObjectPool.h>
|
||||
#include <unordered_map>
|
||||
|
||||
|
||||
|
@ -173,18 +173,24 @@ int waitServersToFinish(std::vector<DB::ProtocolServerAdapter> & servers, size_t
|
||||
const int sleep_one_ms = 100;
|
||||
int sleep_current_ms = 0;
|
||||
int current_connections = 0;
|
||||
while (sleep_current_ms < sleep_max_ms)
|
||||
for (;;)
|
||||
{
|
||||
current_connections = 0;
|
||||
|
||||
for (auto & server : servers)
|
||||
{
|
||||
server.stop();
|
||||
current_connections += server.currentConnections();
|
||||
}
|
||||
|
||||
if (!current_connections)
|
||||
break;
|
||||
|
||||
sleep_current_ms += sleep_one_ms;
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(sleep_one_ms));
|
||||
if (sleep_current_ms < sleep_max_ms)
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(sleep_one_ms));
|
||||
else
|
||||
break;
|
||||
}
|
||||
return current_connections;
|
||||
}
|
||||
@ -879,10 +885,30 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
servers_to_start_before_tables->emplace_back(
|
||||
port_name,
|
||||
std::make_unique<Poco::Net::TCPServer>(
|
||||
new KeeperTCPHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams));
|
||||
new KeeperTCPHandlerFactory(*this, false), server_pool, socket, new Poco::Net::TCPServerParams));
|
||||
|
||||
LOG_INFO(log, "Listening for connections to Keeper (tcp): {}", address.toString());
|
||||
});
|
||||
|
||||
const char * secure_port_name = "keeper_server.tcp_port_secure";
|
||||
createServer(listen_host, secure_port_name, listen_try, [&](UInt16 port)
|
||||
{
|
||||
#if USE_SSL
|
||||
Poco::Net::SecureServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
|
||||
socket.setReceiveTimeout(settings.receive_timeout);
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
servers_to_start_before_tables->emplace_back(
|
||||
secure_port_name,
|
||||
std::make_unique<Poco::Net::TCPServer>(
|
||||
new KeeperTCPHandlerFactory(*this, true), server_pool, socket, new Poco::Net::TCPServerParams));
|
||||
LOG_INFO(log, "Listening for connections to Keeper with secure protocol (tcp_secure): {}", address.toString());
|
||||
#else
|
||||
UNUSED(port);
|
||||
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
|
||||
ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
#endif
|
||||
});
|
||||
}
|
||||
#else
|
||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "ClickHouse server built without NuRaft library. Cannot use internal coordination.");
|
||||
@ -931,6 +957,9 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
global_context->shutdownKeeperStorageDispatcher();
|
||||
}
|
||||
|
||||
/// Wait server pool to avoid use-after-free of destroyed context in the handlers
|
||||
server_pool.joinAll();
|
||||
|
||||
/** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available.
|
||||
* At this moment, no one could own shared part of Context.
|
||||
*/
|
||||
|
@ -551,6 +551,15 @@ void Connection::sendIgnoredPartUUIDs(const std::vector<UUID> & uuids)
|
||||
out->next();
|
||||
}
|
||||
|
||||
|
||||
void Connection::sendReadTaskResponse(const String & response)
|
||||
{
|
||||
writeVarUInt(Protocol::Client::ReadTaskResponse, *out);
|
||||
writeVarUInt(DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION, *out);
|
||||
writeStringBinary(response, *out);
|
||||
out->next();
|
||||
}
|
||||
|
||||
void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String & name)
|
||||
{
|
||||
/// NOTE 'Throttler' is not used in this method (could use, but it's not important right now).
|
||||
@ -807,6 +816,9 @@ Packet Connection::receivePacket()
|
||||
readVectorBinary(res.part_uuids, *in);
|
||||
return res;
|
||||
|
||||
case Protocol::Server::ReadTaskRequest:
|
||||
return res;
|
||||
|
||||
default:
|
||||
/// In unknown state, disconnect - to not leave unsynchronised connection.
|
||||
disconnect();
|
||||
@ -907,13 +919,13 @@ void Connection::setDescription()
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<Exception> Connection::receiveException()
|
||||
std::unique_ptr<Exception> Connection::receiveException() const
|
||||
{
|
||||
return std::make_unique<Exception>(readException(*in, "Received from " + getDescription(), true /* remote */));
|
||||
}
|
||||
|
||||
|
||||
std::vector<String> Connection::receiveMultistringMessage(UInt64 msg_type)
|
||||
std::vector<String> Connection::receiveMultistringMessage(UInt64 msg_type) const
|
||||
{
|
||||
size_t num = Protocol::Server::stringsInMessage(msg_type);
|
||||
std::vector<String> strings(num);
|
||||
@ -923,7 +935,7 @@ std::vector<String> Connection::receiveMultistringMessage(UInt64 msg_type)
|
||||
}
|
||||
|
||||
|
||||
Progress Connection::receiveProgress()
|
||||
Progress Connection::receiveProgress() const
|
||||
{
|
||||
Progress progress;
|
||||
progress.read(*in, server_revision);
|
||||
@ -931,7 +943,7 @@ Progress Connection::receiveProgress()
|
||||
}
|
||||
|
||||
|
||||
BlockStreamProfileInfo Connection::receiveProfileInfo()
|
||||
BlockStreamProfileInfo Connection::receiveProfileInfo() const
|
||||
{
|
||||
BlockStreamProfileInfo profile_info;
|
||||
profile_info.read(*in);
|
||||
|
@ -159,6 +159,8 @@ public:
|
||||
/// Send parts' uuids to excluded them from query processing
|
||||
void sendIgnoredPartUUIDs(const std::vector<UUID> & uuids);
|
||||
|
||||
void sendReadTaskResponse(const String &);
|
||||
|
||||
/// Send prepared block of data (serialized and, if need, compressed), that will be read from 'input'.
|
||||
/// You could pass size of serialized/compressed block.
|
||||
void sendPreparedData(ReadBuffer & input, size_t size, const String & name = "");
|
||||
@ -269,7 +271,7 @@ private:
|
||||
class LoggerWrapper
|
||||
{
|
||||
public:
|
||||
LoggerWrapper(Connection & parent_)
|
||||
explicit LoggerWrapper(Connection & parent_)
|
||||
: log(nullptr), parent(parent_)
|
||||
{
|
||||
}
|
||||
@ -304,10 +306,10 @@ private:
|
||||
Block receiveLogData();
|
||||
Block receiveDataImpl(BlockInputStreamPtr & stream);
|
||||
|
||||
std::vector<String> receiveMultistringMessage(UInt64 msg_type);
|
||||
std::unique_ptr<Exception> receiveException();
|
||||
Progress receiveProgress();
|
||||
BlockStreamProfileInfo receiveProfileInfo();
|
||||
std::vector<String> receiveMultistringMessage(UInt64 msg_type) const;
|
||||
std::unique_ptr<Exception> receiveException() const;
|
||||
Progress receiveProgress() const;
|
||||
BlockStreamProfileInfo receiveProfileInfo() const;
|
||||
|
||||
void initInputBuffers();
|
||||
void initBlockInput();
|
||||
|
@ -26,7 +26,7 @@ public:
|
||||
using Entry = PoolBase<Connection>::Entry;
|
||||
|
||||
public:
|
||||
virtual ~IConnectionPool() {}
|
||||
virtual ~IConnectionPool() = default;
|
||||
|
||||
/// Selects the connection to work.
|
||||
/// If force_connected is false, the client must manually ensure that returned connection is good.
|
||||
|
@ -14,6 +14,12 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
/** To receive data from multiple replicas (connections) from one shard asynchronously.
|
||||
* The principe of Hedged Connections is used to reduce tail latency:
|
||||
* if we don't receive data from replica and there is no progress in query execution
|
||||
@ -84,6 +90,11 @@ public:
|
||||
const ClientInfo & client_info,
|
||||
bool with_pending_data) override;
|
||||
|
||||
void sendReadTaskResponse(const String &) override
|
||||
{
|
||||
throw Exception("sendReadTaskResponse in not supported with HedgedConnections", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
Packet receivePacket() override;
|
||||
|
||||
Packet receivePacketUnlocked(AsyncCallback async_callback) override;
|
||||
|
@ -24,6 +24,8 @@ public:
|
||||
const ClientInfo & client_info,
|
||||
bool with_pending_data) = 0;
|
||||
|
||||
virtual void sendReadTaskResponse(const String &) = 0;
|
||||
|
||||
/// Get packet from any replica.
|
||||
virtual Packet receivePacket() = 0;
|
||||
|
||||
|
@ -155,6 +155,15 @@ void MultiplexedConnections::sendIgnoredPartUUIDs(const std::vector<UUID> & uuid
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void MultiplexedConnections::sendReadTaskResponse(const String & response)
|
||||
{
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
if (cancelled)
|
||||
return;
|
||||
current_connection->sendReadTaskResponse(response);
|
||||
}
|
||||
|
||||
Packet MultiplexedConnections::receivePacket()
|
||||
{
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
@ -210,6 +219,7 @@ Packet MultiplexedConnections::drain()
|
||||
|
||||
switch (packet.type)
|
||||
{
|
||||
case Protocol::Server::ReadTaskRequest:
|
||||
case Protocol::Server::PartUUIDs:
|
||||
case Protocol::Server::Data:
|
||||
case Protocol::Server::Progress:
|
||||
@ -273,6 +283,7 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac
|
||||
|
||||
switch (packet.type)
|
||||
{
|
||||
case Protocol::Server::ReadTaskRequest:
|
||||
case Protocol::Server::PartUUIDs:
|
||||
case Protocol::Server::Data:
|
||||
case Protocol::Server::Progress:
|
||||
|
@ -39,6 +39,8 @@ public:
|
||||
const ClientInfo & client_info,
|
||||
bool with_pending_data) override;
|
||||
|
||||
void sendReadTaskResponse(const String &) override;
|
||||
|
||||
Packet receivePacket() override;
|
||||
|
||||
void disconnect() override;
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <Poco/Mutex.h>
|
||||
#include <Poco/Semaphore.h>
|
||||
|
||||
#include <Common/MoveOrCopyIfThrow.h>
|
||||
#include <common/MoveOrCopyIfThrow.h>
|
||||
|
||||
/** A very simple thread-safe queue of limited size.
|
||||
* If you try to pop an item from an empty queue, the thread is blocked until the queue becomes nonempty.
|
||||
|
@ -51,7 +51,7 @@ private:
|
||||
*/
|
||||
struct PoolEntryHelper
|
||||
{
|
||||
PoolEntryHelper(PooledObject & data_) : data(data_) { data.in_use = true; }
|
||||
explicit PoolEntryHelper(PooledObject & data_) : data(data_) { data.in_use = true; }
|
||||
~PoolEntryHelper()
|
||||
{
|
||||
std::unique_lock lock(data.pool.mutex);
|
||||
@ -69,7 +69,7 @@ public:
|
||||
public:
|
||||
friend class PoolBase<Object>;
|
||||
|
||||
Entry() {} /// For deferred initialization.
|
||||
Entry() = default; /// For deferred initialization.
|
||||
|
||||
/** The `Entry` object protects the resource from being used by another thread.
|
||||
* The following methods are forbidden for `rvalue`, so you can not write a similar to
|
||||
@ -99,10 +99,10 @@ public:
|
||||
private:
|
||||
std::shared_ptr<PoolEntryHelper> data;
|
||||
|
||||
Entry(PooledObject & object) : data(std::make_shared<PoolEntryHelper>(object)) {}
|
||||
explicit Entry(PooledObject & object) : data(std::make_shared<PoolEntryHelper>(object)) {}
|
||||
};
|
||||
|
||||
virtual ~PoolBase() {}
|
||||
virtual ~PoolBase() = default;
|
||||
|
||||
/** Allocates the object. Wait for free object in pool for 'timeout'. With 'timeout' < 0, the timeout is infinite. */
|
||||
Entry get(Poco::Timespan::TimeDiff timeout)
|
||||
|
@ -421,26 +421,38 @@ std::pair<ResponsePtr, Undo> TestKeeperMultiRequest::process(TestKeeper::Contain
|
||||
|
||||
try
|
||||
{
|
||||
for (const auto & request : requests)
|
||||
auto request_it = requests.begin();
|
||||
response.error = Error::ZOK;
|
||||
while (request_it != requests.end())
|
||||
{
|
||||
const TestKeeperRequest & concrete_request = dynamic_cast<const TestKeeperRequest &>(*request);
|
||||
const TestKeeperRequest & concrete_request = dynamic_cast<const TestKeeperRequest &>(**request_it);
|
||||
++request_it;
|
||||
auto [ cur_response, undo_action ] = concrete_request.process(container, zxid);
|
||||
response.responses.emplace_back(cur_response);
|
||||
if (cur_response->error != Error::ZOK)
|
||||
{
|
||||
response.error = cur_response->error;
|
||||
|
||||
for (auto it = undo_actions.rbegin(); it != undo_actions.rend(); ++it)
|
||||
if (*it)
|
||||
(*it)();
|
||||
|
||||
return { std::make_shared<MultiResponse>(response), {} };
|
||||
break;
|
||||
}
|
||||
|
||||
undo_actions.emplace_back(std::move(undo_action));
|
||||
}
|
||||
|
||||
if (response.error != Error::ZOK)
|
||||
{
|
||||
for (auto it = undo_actions.rbegin(); it != undo_actions.rend(); ++it)
|
||||
if (*it)
|
||||
(*it)();
|
||||
|
||||
while (request_it != requests.end())
|
||||
{
|
||||
const TestKeeperRequest & concrete_request = dynamic_cast<const TestKeeperRequest &>(**request_it);
|
||||
++request_it;
|
||||
response.responses.emplace_back(concrete_request.createResponse());
|
||||
response.responses.back()->error = Error::ZRUNTIMEINCONSISTENCY;
|
||||
}
|
||||
else
|
||||
undo_actions.emplace_back(std::move(undo_action));
|
||||
}
|
||||
|
||||
response.error = Error::ZOK;
|
||||
return { std::make_shared<MultiResponse>(response), {} };
|
||||
}
|
||||
catch (...)
|
||||
|
@ -1,4 +1,9 @@
|
||||
#include <Coordination/KeeperServer.h>
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
|
||||
#include <Coordination/LoggerWrapper.h>
|
||||
#include <Coordination/KeeperStateMachine.h>
|
||||
#include <Coordination/KeeperStateManager.h>
|
||||
@ -9,6 +14,7 @@
|
||||
#include <chrono>
|
||||
#include <Common/ZooKeeper/ZooKeeperIO.h>
|
||||
#include <string>
|
||||
#include <Poco/Util/Application.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -16,6 +22,42 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int RAFT_ERROR;
|
||||
extern const int NO_ELEMENTS_IN_CONFIG;
|
||||
extern const int SUPPORT_IS_DISABLED;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
#if USE_SSL
|
||||
void setSSLParams(nuraft::asio_service::options & asio_opts)
|
||||
{
|
||||
const Poco::Util::LayeredConfiguration & config = Poco::Util::Application::instance().config();
|
||||
String certificate_file_property = "openSSL.server.certificateFile";
|
||||
String private_key_file_property = "openSSL.server.privateKeyFile";
|
||||
String root_ca_file_property = "openSSL.server.caConfig";
|
||||
|
||||
if (!config.has(certificate_file_property))
|
||||
throw Exception("Server certificate file is not set.", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
|
||||
|
||||
if (!config.has(private_key_file_property))
|
||||
throw Exception("Server private key file is not set.", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
|
||||
|
||||
asio_opts.enable_ssl_ = true;
|
||||
asio_opts.server_cert_file_ = config.getString(certificate_file_property);
|
||||
asio_opts.server_key_file_ = config.getString(private_key_file_property);
|
||||
|
||||
if (config.has(root_ca_file_property))
|
||||
asio_opts.root_cert_file_ = config.getString(root_ca_file_property);
|
||||
|
||||
if (config.getBool("openSSL.server.loadDefaultCAFile", false))
|
||||
asio_opts.load_default_ca_file_ = true;
|
||||
|
||||
if (config.getString("openSSL.server.verificationMode", "none") == "none")
|
||||
asio_opts.skip_verification_ = true;
|
||||
}
|
||||
#endif
|
||||
|
||||
}
|
||||
|
||||
KeeperServer::KeeperServer(
|
||||
@ -72,6 +114,15 @@ void KeeperServer::startup()
|
||||
params.return_method_ = nuraft::raft_params::blocking;
|
||||
|
||||
nuraft::asio_service::options asio_opts{};
|
||||
if (state_manager->isSecure())
|
||||
{
|
||||
#if USE_SSL
|
||||
setSSLParams(asio_opts);
|
||||
#else
|
||||
throw Exception{"SSL support for NuRaft is disabled because ClickHouse was built without SSL support.",
|
||||
ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
#endif
|
||||
}
|
||||
|
||||
launchRaftServer(params, asio_opts);
|
||||
|
||||
|
@ -12,6 +12,7 @@ namespace ErrorCodes
|
||||
KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path)
|
||||
: my_server_id(server_id_)
|
||||
, my_port(port)
|
||||
, secure(false)
|
||||
, log_store(nuraft::cs_new<KeeperLogStore>(logs_path, 5000, false))
|
||||
, cluster_config(nuraft::cs_new<nuraft::cluster_config>())
|
||||
{
|
||||
@ -25,6 +26,7 @@ KeeperStateManager::KeeperStateManager(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const CoordinationSettingsPtr & coordination_settings)
|
||||
: my_server_id(my_server_id_)
|
||||
, secure(config.getBool(config_prefix + ".raft_configuration.secure", false))
|
||||
, log_store(nuraft::cs_new<KeeperLogStore>(
|
||||
config.getString(config_prefix + ".log_storage_path", config.getString("path", DBMS_DEFAULT_PATH) + "coordination/logs"),
|
||||
coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync))
|
||||
@ -37,6 +39,9 @@ KeeperStateManager::KeeperStateManager(
|
||||
|
||||
for (const auto & server_key : keys)
|
||||
{
|
||||
if (!startsWith(server_key, "server"))
|
||||
continue;
|
||||
|
||||
std::string full_prefix = config_prefix + ".raft_configuration." + server_key;
|
||||
int server_id = config.getInt(full_prefix + ".id");
|
||||
std::string hostname = config.getString(full_prefix + ".hostname");
|
||||
@ -44,6 +49,7 @@ KeeperStateManager::KeeperStateManager(
|
||||
bool can_become_leader = config.getBool(full_prefix + ".can_become_leader", true);
|
||||
int32_t priority = config.getInt(full_prefix + ".priority", 1);
|
||||
bool start_as_follower = config.getBool(full_prefix + ".start_as_follower", false);
|
||||
|
||||
if (start_as_follower)
|
||||
start_as_follower_servers.insert(server_id);
|
||||
|
||||
@ -57,6 +63,7 @@ KeeperStateManager::KeeperStateManager(
|
||||
|
||||
cluster_config->get_servers().push_back(peer_config);
|
||||
}
|
||||
|
||||
if (!my_server_config)
|
||||
throw Exception(ErrorCodes::RAFT_ERROR, "Our server id {} not found in raft_configuration section", my_server_id);
|
||||
|
||||
|
@ -52,6 +52,11 @@ public:
|
||||
return start_as_follower_servers.count(my_server_id);
|
||||
}
|
||||
|
||||
bool isSecure() const
|
||||
{
|
||||
return secure;
|
||||
}
|
||||
|
||||
nuraft::ptr<KeeperLogStore> getLogStore() const { return log_store; }
|
||||
|
||||
uint64_t getTotalServers() const { return total_servers; }
|
||||
@ -59,6 +64,7 @@ public:
|
||||
private:
|
||||
int my_server_id;
|
||||
int my_port;
|
||||
bool secure;
|
||||
uint64_t total_servers{0};
|
||||
std::unordered_set<int> start_as_follower_servers;
|
||||
nuraft::ptr<KeeperLogStore> log_store;
|
||||
|
@ -74,6 +74,9 @@
|
||||
/// Minimum revision supporting OpenTelemetry
|
||||
#define DBMS_MIN_REVISION_WITH_OPENTELEMETRY 54442
|
||||
|
||||
|
||||
#define DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION 1
|
||||
|
||||
/// Minimum revision supporting interserver secret.
|
||||
#define DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET 54441
|
||||
|
||||
|
@ -76,8 +76,10 @@ namespace Protocol
|
||||
Log = 10, /// System logs of the query execution
|
||||
TableColumns = 11, /// Columns' description for default values calculation
|
||||
PartUUIDs = 12, /// List of unique parts ids.
|
||||
|
||||
MAX = PartUUIDs,
|
||||
ReadTaskRequest = 13, /// String (UUID) describes a request for which next task is needed
|
||||
/// This is such an inverted logic, where server sends requests
|
||||
/// And client returns back response
|
||||
MAX = ReadTaskRequest,
|
||||
};
|
||||
|
||||
/// NOTE: If the type of packet argument would be Enum, the comparison packet >= 0 && packet < 10
|
||||
@ -100,6 +102,7 @@ namespace Protocol
|
||||
"Log",
|
||||
"TableColumns",
|
||||
"PartUUIDs",
|
||||
"ReadTaskRequest"
|
||||
};
|
||||
return packet <= MAX
|
||||
? data[packet]
|
||||
@ -135,8 +138,9 @@ namespace Protocol
|
||||
KeepAlive = 6, /// Keep the connection alive
|
||||
Scalar = 7, /// A block of data (compressed or not).
|
||||
IgnoredPartUUIDs = 8, /// List of unique parts ids to exclude from query processing
|
||||
ReadTaskResponse = 9, /// TODO:
|
||||
|
||||
MAX = IgnoredPartUUIDs,
|
||||
MAX = ReadTaskResponse,
|
||||
};
|
||||
|
||||
inline const char * toString(UInt64 packet)
|
||||
@ -151,6 +155,7 @@ namespace Protocol
|
||||
"KeepAlive",
|
||||
"Scalar",
|
||||
"IgnoredPartUUIDs",
|
||||
"ReadTaskResponse",
|
||||
};
|
||||
return packet <= MAX
|
||||
? data[packet]
|
||||
|
@ -8,7 +8,7 @@ namespace DB
|
||||
|
||||
/// Prints internal server logs
|
||||
/// Input blocks have to have the same structure as SystemLogsQueue::getSampleBlock()
|
||||
/// NOTE: IRowOutputStream does not suite well for this case
|
||||
/// NOTE: IRowOutputFormat does not suite well for this case
|
||||
class InternalTextLogsRowOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
|
@ -22,20 +22,18 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int UNKNOWN_PACKET_FROM_SERVER;
|
||||
extern const int DUPLICATED_PART_UUIDS;
|
||||
}
|
||||
|
||||
RemoteQueryExecutor::RemoteQueryExecutor(
|
||||
Connection & connection,
|
||||
const String & query_,
|
||||
const Block & header_,
|
||||
ContextPtr context_,
|
||||
ThrottlerPtr throttler,
|
||||
const Scalars & scalars_,
|
||||
const Tables & external_tables_,
|
||||
QueryProcessingStage::Enum stage_)
|
||||
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
|
||||
const String & query_, const Block & header_, ContextPtr context_,
|
||||
ThrottlerPtr throttler, const Scalars & scalars_, const Tables & external_tables_,
|
||||
QueryProcessingStage::Enum stage_, std::shared_ptr<TaskIterator> task_iterator_)
|
||||
: header(header_), query(query_), context(context_)
|
||||
, scalars(scalars_), external_tables(external_tables_), stage(stage_), task_iterator(task_iterator_)
|
||||
{
|
||||
create_connections = [this, &connection, throttler]()
|
||||
{
|
||||
@ -45,14 +43,11 @@ RemoteQueryExecutor::RemoteQueryExecutor(
|
||||
|
||||
RemoteQueryExecutor::RemoteQueryExecutor(
|
||||
std::vector<IConnectionPool::Entry> && connections_,
|
||||
const String & query_,
|
||||
const Block & header_,
|
||||
ContextPtr context_,
|
||||
const ThrottlerPtr & throttler,
|
||||
const Scalars & scalars_,
|
||||
const Tables & external_tables_,
|
||||
QueryProcessingStage::Enum stage_)
|
||||
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
|
||||
const String & query_, const Block & header_, ContextPtr context_,
|
||||
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_,
|
||||
QueryProcessingStage::Enum stage_, std::shared_ptr<TaskIterator> task_iterator_)
|
||||
: header(header_), query(query_), context(context_)
|
||||
, scalars(scalars_), external_tables(external_tables_), stage(stage_), task_iterator(task_iterator_)
|
||||
{
|
||||
create_connections = [this, connections_, throttler]() mutable {
|
||||
return std::make_unique<MultiplexedConnections>(std::move(connections_), context->getSettingsRef(), throttler);
|
||||
@ -61,14 +56,11 @@ RemoteQueryExecutor::RemoteQueryExecutor(
|
||||
|
||||
RemoteQueryExecutor::RemoteQueryExecutor(
|
||||
const ConnectionPoolWithFailoverPtr & pool,
|
||||
const String & query_,
|
||||
const Block & header_,
|
||||
ContextPtr context_,
|
||||
const ThrottlerPtr & throttler,
|
||||
const Scalars & scalars_,
|
||||
const Tables & external_tables_,
|
||||
QueryProcessingStage::Enum stage_)
|
||||
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
|
||||
const String & query_, const Block & header_, ContextPtr context_,
|
||||
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_,
|
||||
QueryProcessingStage::Enum stage_, std::shared_ptr<TaskIterator> task_iterator_)
|
||||
: header(header_), query(query_), context(context_)
|
||||
, scalars(scalars_), external_tables(external_tables_), stage(stage_), task_iterator(task_iterator_)
|
||||
{
|
||||
create_connections = [this, pool, throttler]()->std::unique_ptr<IConnections>
|
||||
{
|
||||
@ -307,6 +299,9 @@ std::optional<Block> RemoteQueryExecutor::processPacket(Packet packet)
|
||||
{
|
||||
switch (packet.type)
|
||||
{
|
||||
case Protocol::Server::ReadTaskRequest:
|
||||
processReadTaskRequest();
|
||||
break;
|
||||
case Protocol::Server::PartUUIDs:
|
||||
if (!setPartUUIDs(packet.part_uuids))
|
||||
got_duplicated_part_uuids = true;
|
||||
@ -385,6 +380,14 @@ bool RemoteQueryExecutor::setPartUUIDs(const std::vector<UUID> & uuids)
|
||||
return true;
|
||||
}
|
||||
|
||||
void RemoteQueryExecutor::processReadTaskRequest()
|
||||
{
|
||||
if (!task_iterator)
|
||||
throw Exception("Distributed task iterator is not initialized", ErrorCodes::LOGICAL_ERROR);
|
||||
auto response = (*task_iterator)();
|
||||
connections->sendReadTaskResponse(response);
|
||||
}
|
||||
|
||||
void RemoteQueryExecutor::finish(std::unique_ptr<ReadContext> * read_context)
|
||||
{
|
||||
/** If one of:
|
||||
|
@ -26,6 +26,9 @@ using ProfileInfoCallback = std::function<void(const BlockStreamProfileInfo & in
|
||||
|
||||
class RemoteQueryExecutorReadContext;
|
||||
|
||||
/// This is the same type as StorageS3Source::IteratorWrapper
|
||||
using TaskIterator = std::function<String()>;
|
||||
|
||||
/// This class allows one to launch queries on remote replicas of one shard and get results
|
||||
class RemoteQueryExecutor
|
||||
{
|
||||
@ -37,21 +40,21 @@ public:
|
||||
Connection & connection,
|
||||
const String & query_, const Block & header_, ContextPtr context_,
|
||||
ThrottlerPtr throttler_ = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, std::shared_ptr<TaskIterator> task_iterator_ = {});
|
||||
|
||||
/// Accepts several connections already taken from pool.
|
||||
RemoteQueryExecutor(
|
||||
std::vector<IConnectionPool::Entry> && connections_,
|
||||
const String & query_, const Block & header_, ContextPtr context_,
|
||||
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, std::shared_ptr<TaskIterator> task_iterator_ = {});
|
||||
|
||||
/// Takes a pool and gets one or several connections from it.
|
||||
RemoteQueryExecutor(
|
||||
const ConnectionPoolWithFailoverPtr & pool,
|
||||
const String & query_, const Block & header_, ContextPtr context_,
|
||||
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, std::shared_ptr<TaskIterator> task_iterator_ = {});
|
||||
|
||||
~RemoteQueryExecutor();
|
||||
|
||||
@ -119,6 +122,8 @@ private:
|
||||
/// Temporary tables needed to be sent to remote servers
|
||||
Tables external_tables;
|
||||
QueryProcessingStage::Enum stage;
|
||||
/// Initiator identifier for distributed task processing
|
||||
std::shared_ptr<TaskIterator> task_iterator;
|
||||
|
||||
/// Streams for reading from temporary tables and following sending of data
|
||||
/// to remote servers for GLOBAL-subqueries
|
||||
@ -179,6 +184,8 @@ private:
|
||||
/// Return true if duplicates found.
|
||||
bool setPartUUIDs(const std::vector<UUID> & uuids);
|
||||
|
||||
void processReadTaskRequest();
|
||||
|
||||
/// Cancell query and restart it with info about duplicated UUIDs
|
||||
/// only for `allow_experimental_query_deduplication`.
|
||||
std::variant<Block, int> restartQueryWithoutDuplicatedUUIDs(std::unique_ptr<ReadContext> * read_context = nullptr);
|
||||
|
@ -52,7 +52,7 @@ static std::unordered_map<String, String> fetchTablesCreateQuery(
|
||||
static std::vector<String> fetchTablesInDB(const mysqlxx::PoolWithFailover::Entry & connection, const std::string & database)
|
||||
{
|
||||
Block header{{std::make_shared<DataTypeString>(), "table_name"}};
|
||||
String query = "SELECT TABLE_NAME AS table_name FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA = " + quoteString(database);
|
||||
String query = "SELECT TABLE_NAME AS table_name FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_TYPE != 'VIEW' AND TABLE_SCHEMA = " + quoteString(database);
|
||||
|
||||
std::vector<String> tables_in_db;
|
||||
MySQLBlockInputStream input(connection, query, header, DEFAULT_BLOCK_SIZE);
|
||||
|
@ -1,7 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/BorrowedObjectPool.h>
|
||||
|
||||
#include <Core/Block.h>
|
||||
#include <Common/BorrowedObjectPool.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include "IDictionarySource.h"
|
||||
|
@ -10,7 +10,6 @@
|
||||
#include "DictionarySourceFactory.h"
|
||||
#include "DictionarySourceHelpers.h"
|
||||
#include "DictionaryStructure.h"
|
||||
#include "LibraryDictionarySourceExternal.h"
|
||||
#include "registerDictionaries.h"
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
@ -1,18 +0,0 @@
|
||||
#include <Formats/IRowInputStream.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
void IRowInputStream::syncAfterError()
|
||||
{
|
||||
throw Exception("Method syncAfterError is not implemented for input format", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
}
|
@ -1,51 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
|
||||
#include <Columns/IColumn.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Contains extra information about read data.
|
||||
struct RowReadExtension
|
||||
{
|
||||
/// IRowInputStream.read() output. It contains non zero for columns that actually read from the source and zero otherwise.
|
||||
/// It's used to attach defaults for partially filled rows.
|
||||
/// Can be empty, this means that all columns are read.
|
||||
std::vector<UInt8> read_columns;
|
||||
};
|
||||
|
||||
/** Interface of stream, that allows to read data by rows.
|
||||
*/
|
||||
class IRowInputStream : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
/** Read next row and append it to the columns.
|
||||
* If no more rows - return false.
|
||||
*/
|
||||
virtual bool read(MutableColumns & columns, RowReadExtension & extra) = 0;
|
||||
|
||||
virtual void readPrefix() {} /// delimiter before begin of result
|
||||
virtual void readSuffix() {} /// delimiter after end of result
|
||||
|
||||
/// Skip data until next row.
|
||||
/// This is intended for text streams, that allow skipping of errors.
|
||||
/// By default - throws not implemented exception.
|
||||
virtual bool allowSyncAfterError() const { return false; }
|
||||
virtual void syncAfterError();
|
||||
|
||||
/// In case of parse error, try to roll back and parse last one or two rows very carefully
|
||||
/// and collect as much as possible diagnostic information about error.
|
||||
/// If not implemented, returns empty string.
|
||||
virtual std::string getDiagnosticInfo() { return {}; }
|
||||
|
||||
virtual ~IRowInputStream() {}
|
||||
};
|
||||
|
||||
using RowInputStreamPtr = std::shared_ptr<IRowInputStream>;
|
||||
|
||||
}
|
@ -1,37 +0,0 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Formats/IRowOutputStream.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
|
||||
void IRowOutputStream::write(const Block & block, size_t row_num)
|
||||
{
|
||||
size_t columns = block.columns();
|
||||
|
||||
writeRowStartDelimiter();
|
||||
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
writeFieldDelimiter();
|
||||
|
||||
const auto & col = block.getByPosition(i);
|
||||
writeField(*col.column, *col.type, row_num);
|
||||
}
|
||||
|
||||
writeRowEndDelimiter();
|
||||
}
|
||||
|
||||
void IRowOutputStream::writeField(const IColumn &, const IDataType &, size_t)
|
||||
{
|
||||
throw Exception("Method writeField is not implemented for output format", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
}
|
@ -1,63 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <memory>
|
||||
#include <cstdint>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <common/types.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Block;
|
||||
class IColumn;
|
||||
class IDataType;
|
||||
struct Progress;
|
||||
|
||||
|
||||
/** Interface of stream for writing data by rows (for example: for output to terminal).
|
||||
*/
|
||||
class IRowOutputStream : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
|
||||
/** Write a row.
|
||||
* Default implementation calls methods to write single values and delimiters
|
||||
* (except delimiter between rows (writeRowBetweenDelimiter())).
|
||||
*/
|
||||
virtual void write(const Block & block, size_t row_num);
|
||||
|
||||
/** Write single value. */
|
||||
virtual void writeField(const IColumn & column, const IDataType & type, size_t row_num);
|
||||
|
||||
/** Write delimiter. */
|
||||
virtual void writeFieldDelimiter() {} /// delimiter between values
|
||||
virtual void writeRowStartDelimiter() {} /// delimiter before each row
|
||||
virtual void writeRowEndDelimiter() {} /// delimiter after each row
|
||||
virtual void writeRowBetweenDelimiter() {} /// delimiter between rows
|
||||
virtual void writePrefix() {} /// delimiter before resultset
|
||||
virtual void writeSuffix() {} /// delimiter after resultset
|
||||
|
||||
/** Flush output buffers if any. */
|
||||
virtual void flush() {}
|
||||
|
||||
/** Methods to set additional information for output in formats, that support it.
|
||||
*/
|
||||
virtual void setRowsBeforeLimit(size_t /*rows_before_limit*/) {}
|
||||
virtual void setTotals(const Block & /*totals*/) {}
|
||||
virtual void setExtremes(const Block & /*extremes*/) {}
|
||||
|
||||
/** Notify about progress. Method could be called from different threads.
|
||||
* Passed value are delta, that must be summarized.
|
||||
*/
|
||||
virtual void onProgress(const Progress & /*progress*/) {}
|
||||
|
||||
/** Content-Type to set when sending HTTP response. */
|
||||
virtual String getContentType() const { return "text/plain; charset=UTF-8"; }
|
||||
|
||||
virtual ~IRowOutputStream() {}
|
||||
};
|
||||
|
||||
using RowOutputStreamPtr = std::shared_ptr<IRowOutputStream>;
|
||||
|
||||
}
|
@ -13,8 +13,6 @@ PEERDIR(
|
||||
SRCS(
|
||||
FormatFactory.cpp
|
||||
FormatSchemaInfo.cpp
|
||||
IRowInputStream.cpp
|
||||
IRowOutputStream.cpp
|
||||
JSONEachRowUtils.cpp
|
||||
MySQLBlockInputStream.cpp
|
||||
NativeFormat.cpp
|
||||
|
@ -82,7 +82,7 @@ inline ALWAYS_INLINE void writeSlice(const GenericArraySlice & slice, GenericArr
|
||||
sink.current_offset += slice.size;
|
||||
}
|
||||
else
|
||||
throw Exception("Function writeSlice expect same column types for GenericArraySlice and GenericArraySink.",
|
||||
throw Exception("Function writeSlice expects same column types for GenericArraySlice and GenericArraySink.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
@ -162,7 +162,7 @@ inline ALWAYS_INLINE void writeSlice(const GenericValueSlice & slice, GenericArr
|
||||
++sink.current_offset;
|
||||
}
|
||||
else
|
||||
throw Exception("Function writeSlice expect same column types for GenericValueSlice and GenericArraySink.",
|
||||
throw Exception("Function writeSlice expects same column types for GenericValueSlice and GenericArraySink.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
@ -609,7 +609,7 @@ bool sliceHas(const GenericArraySlice & first, const GenericArraySlice & second)
|
||||
{
|
||||
/// Generic arrays should have the same type in order to use column.compareAt(...)
|
||||
if (!first.elements->structureEquals(*second.elements))
|
||||
return false;
|
||||
throw Exception("Function sliceHas expects same column types for slices.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto impl = sliceHasImpl<search_type, GenericArraySlice, GenericArraySlice, sliceEqualElements, insliceEqualElements>;
|
||||
return impl(first, second, nullptr, nullptr);
|
||||
@ -670,7 +670,7 @@ void NO_INLINE arrayAllAny(FirstSource && first, SecondSource && second, ColumnU
|
||||
auto & data = result.getData();
|
||||
for (auto row : ext::range(0, size))
|
||||
{
|
||||
data[row] = static_cast<UInt8>(sliceHas<search_type>(first.getWhole(), second.getWhole()) ? 1 : 0);
|
||||
data[row] = static_cast<UInt8>(sliceHas<search_type>(first.getWhole(), second.getWhole()));
|
||||
first.next();
|
||||
second.next();
|
||||
}
|
||||
|
@ -477,7 +477,7 @@ DataTypePtr FunctionOverloadResolverAdaptor::getReturnTypeDefaultImplementationF
|
||||
}
|
||||
if (null_presence.has_nullable)
|
||||
{
|
||||
Block nested_columns = createBlockWithNestedColumns(arguments);
|
||||
auto nested_columns = Block(createBlockWithNestedColumns(arguments));
|
||||
auto return_type = getter(ColumnsWithTypeAndName(nested_columns.begin(), nested_columns.end()));
|
||||
return makeNullable(return_type);
|
||||
}
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
@ -13,9 +14,9 @@
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <Common/memcmpSmall.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include "Columns/ColumnLowCardinality.h"
|
||||
#include "DataTypes/DataTypeLowCardinality.h"
|
||||
#include "Interpreters/castColumn.h"
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -373,11 +374,10 @@ public:
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!arguments[1]->onlyNull() && !allowArguments(array_type->getNestedType(), arguments[1]))
|
||||
throw Exception("Types of array and 2nd argument of function \""
|
||||
+ getName() + "\" must be identical up to nullability, cardinality, "
|
||||
"numeric types, or Enum and numeric type. Passed: "
|
||||
+ arguments[0]->getName() + " and " + arguments[1]->getName() + ".",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Types of array and 2nd argument of function `{}` must be identical up to nullability, cardinality, "
|
||||
"numeric types, or Enum and numeric type. Passed: {} and {}.",
|
||||
getName(), arguments[0]->getName(), arguments[1]->getName());
|
||||
|
||||
return std::make_shared<DataTypeNumber<ResultType>>();
|
||||
}
|
||||
@ -494,86 +494,13 @@ private:
|
||||
inline void moveResult() { result_column = std::move(result); }
|
||||
};
|
||||
|
||||
static inline bool allowNested(const DataTypePtr & left, const DataTypePtr & right)
|
||||
{
|
||||
return ((isNativeNumber(left) || isEnum(left)) && isNativeNumber(right)) || left->equals(*right);
|
||||
}
|
||||
|
||||
static inline bool allowArguments(const DataTypePtr & array_inner_type, const DataTypePtr & arg)
|
||||
{
|
||||
if (allowNested(array_inner_type, arg))
|
||||
return true;
|
||||
auto inner_type_decayed = removeNullable(removeLowCardinality(array_inner_type));
|
||||
auto arg_decayed = removeNullable(removeLowCardinality(arg));
|
||||
|
||||
/// Nullable
|
||||
|
||||
const bool array_is_nullable = array_inner_type->isNullable();
|
||||
const bool arg_is_nullable = arg->isNullable();
|
||||
|
||||
const DataTypePtr arg_or_arg_nullable_nested = arg_is_nullable
|
||||
? checkAndGetDataType<DataTypeNullable>(arg.get())->getNestedType()
|
||||
: arg;
|
||||
|
||||
if (array_is_nullable) // comparing Array(Nullable(T)) elem and U
|
||||
{
|
||||
const DataTypePtr array_nullable_nested =
|
||||
checkAndGetDataType<DataTypeNullable>(array_inner_type.get())->getNestedType();
|
||||
|
||||
// We also allow Nullable(T) and LC(U) if the Nullable(T) and U are allowed,
|
||||
// the LC(U) will be converted to U.
|
||||
return allowNested(
|
||||
array_nullable_nested,
|
||||
recursiveRemoveLowCardinality(arg_or_arg_nullable_nested));
|
||||
}
|
||||
else if (arg_is_nullable) // cannot compare Array(T) elem (namely, T) and Nullable(T)
|
||||
return false;
|
||||
|
||||
/// LowCardinality
|
||||
|
||||
const auto * const array_lc_ptr = checkAndGetDataType<DataTypeLowCardinality>(array_inner_type.get());
|
||||
const auto * const arg_lc_ptr = checkAndGetDataType<DataTypeLowCardinality>(arg.get());
|
||||
|
||||
const DataTypePtr array_lc_inner_type = recursiveRemoveLowCardinality(array_inner_type);
|
||||
const DataTypePtr arg_lc_inner_type = recursiveRemoveLowCardinality(arg);
|
||||
|
||||
const bool array_is_lc = nullptr != array_lc_ptr;
|
||||
const bool arg_is_lc = nullptr != arg_lc_ptr;
|
||||
|
||||
const bool array_lc_inner_type_is_nullable = array_is_lc && array_lc_inner_type->isNullable();
|
||||
const bool arg_lc_inner_type_is_nullable = arg_is_lc && arg_lc_inner_type->isNullable();
|
||||
|
||||
if (array_is_lc) // comparing LC(T) and U
|
||||
{
|
||||
const DataTypePtr array_lc_nested_or_lc_nullable_nested = array_lc_inner_type_is_nullable
|
||||
? checkAndGetDataType<DataTypeNullable>(array_lc_inner_type.get())->getNestedType()
|
||||
: array_lc_inner_type;
|
||||
|
||||
if (arg_is_lc) // comparing LC(T) and LC(U)
|
||||
{
|
||||
const DataTypePtr arg_lc_nested_or_lc_nullable_nested = arg_lc_inner_type_is_nullable
|
||||
? checkAndGetDataType<DataTypeNullable>(arg_lc_inner_type.get())->getNestedType()
|
||||
: arg_lc_inner_type;
|
||||
|
||||
return allowNested(
|
||||
array_lc_nested_or_lc_nullable_nested,
|
||||
arg_lc_nested_or_lc_nullable_nested);
|
||||
}
|
||||
else if (arg_is_nullable) // Comparing LC(T) and Nullable(U)
|
||||
{
|
||||
if (!array_lc_inner_type_is_nullable)
|
||||
return false; // Can't compare Array(LC(U)) elem and Nullable(T);
|
||||
|
||||
return allowNested(
|
||||
array_lc_nested_or_lc_nullable_nested,
|
||||
arg_or_arg_nullable_nested);
|
||||
}
|
||||
else // Comparing LC(T) and U (U neither Nullable nor LC)
|
||||
return allowNested(array_lc_nested_or_lc_nullable_nested, arg);
|
||||
}
|
||||
|
||||
if (arg_is_lc) // Allow T and LC(U) if U and T are allowed (the low cardinality column will be converted).
|
||||
return allowNested(array_inner_type, arg_lc_inner_type);
|
||||
|
||||
return false;
|
||||
return ((isNativeNumber(inner_type_decayed) || isEnum(inner_type_decayed)) && isNativeNumber(arg_decayed))
|
||||
|| getLeastSupertype({inner_type_decayed, arg_decayed});
|
||||
}
|
||||
|
||||
#define INTEGRAL_TPL_PACK UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64
|
||||
@ -1044,33 +971,38 @@ private:
|
||||
if (!col)
|
||||
return nullptr;
|
||||
|
||||
const IColumn & col_nested = col->getData();
|
||||
DataTypePtr array_elements_type = assert_cast<const DataTypeArray &>(*arguments[0].type).getNestedType();
|
||||
const DataTypePtr & index_type = arguments[1].type;
|
||||
|
||||
DataTypePtr common_type = getLeastSupertype({array_elements_type, index_type});
|
||||
|
||||
ColumnPtr col_nested = castColumn({ col->getDataPtr(), array_elements_type, "" }, common_type);
|
||||
|
||||
const ColumnPtr right_ptr = arguments[1].column->convertToFullColumnIfLowCardinality();
|
||||
const IColumn & item_arg = *right_ptr.get();
|
||||
ColumnPtr item_arg = castColumn({ right_ptr, removeLowCardinality(index_type), "" }, common_type);
|
||||
|
||||
auto col_res = ResultColumnType::create();
|
||||
|
||||
auto [null_map_data, null_map_item] = getNullMaps(arguments);
|
||||
|
||||
if (item_arg.onlyNull())
|
||||
if (item_arg->onlyNull())
|
||||
Impl::Null<ConcreteAction>::process(
|
||||
col->getOffsets(),
|
||||
col_res->getData(),
|
||||
null_map_data);
|
||||
else if (isColumnConst(item_arg))
|
||||
else if (isColumnConst(*item_arg))
|
||||
Impl::Main<ConcreteAction, true>::vector(
|
||||
col_nested,
|
||||
*col_nested,
|
||||
col->getOffsets(),
|
||||
typeid_cast<const ColumnConst &>(item_arg).getDataColumn(),
|
||||
typeid_cast<const ColumnConst &>(*item_arg).getDataColumn(),
|
||||
col_res->getData(), /// TODO This is wrong.
|
||||
null_map_data,
|
||||
nullptr);
|
||||
else
|
||||
Impl::Main<ConcreteAction>::vector(
|
||||
col_nested,
|
||||
*col_nested,
|
||||
col->getOffsets(),
|
||||
item_arg,
|
||||
*item_arg,
|
||||
col_res->getData(),
|
||||
null_map_data,
|
||||
null_map_item);
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <ext/range.h>
|
||||
#include <ext/map.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -51,41 +52,13 @@ public:
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
size_t rows = input_rows_count;
|
||||
size_t num_args = arguments.size();
|
||||
|
||||
DataTypePtr common_type = nullptr;
|
||||
auto commonType = [&common_type, &arguments]()
|
||||
{
|
||||
if (common_type == nullptr)
|
||||
{
|
||||
DataTypes data_types;
|
||||
data_types.reserve(arguments.size());
|
||||
for (const auto & argument : arguments)
|
||||
data_types.push_back(argument.type);
|
||||
|
||||
common_type = getLeastSupertype(data_types);
|
||||
}
|
||||
|
||||
return common_type;
|
||||
};
|
||||
DataTypePtr common_type = getLeastSupertype(ext::map(arguments, [](auto & arg) { return arg.type; }));
|
||||
|
||||
Columns preprocessed_columns(num_args);
|
||||
|
||||
for (size_t i = 0; i < num_args; ++i)
|
||||
{
|
||||
const auto & argument = arguments[i];
|
||||
ColumnPtr preprocessed_column = argument.column;
|
||||
|
||||
const auto * argument_type = typeid_cast<const DataTypeArray *>(argument.type.get());
|
||||
const auto & nested_type = argument_type->getNestedType();
|
||||
|
||||
/// Converts Array(Nothing) or Array(Nullable(Nothing) to common type. Example: hasAll([Null, 1], [Null]) -> 1
|
||||
if (typeid_cast<const DataTypeNothing *>(removeNullable(nested_type).get()))
|
||||
preprocessed_column = castColumn(argument, commonType());
|
||||
|
||||
preprocessed_columns[i] = std::move(preprocessed_column);
|
||||
}
|
||||
preprocessed_columns[i] = castColumn(arguments[i], common_type);
|
||||
|
||||
std::vector<std::unique_ptr<GatherUtils::IArraySource>> sources;
|
||||
|
||||
@ -100,12 +73,12 @@ public:
|
||||
}
|
||||
|
||||
if (const auto * argument_column_array = typeid_cast<const ColumnArray *>(argument_column.get()))
|
||||
sources.emplace_back(GatherUtils::createArraySource(*argument_column_array, is_const, rows));
|
||||
sources.emplace_back(GatherUtils::createArraySource(*argument_column_array, is_const, input_rows_count));
|
||||
else
|
||||
throw Exception{"Arguments for function " + getName() + " must be arrays.", ErrorCodes::LOGICAL_ERROR};
|
||||
}
|
||||
|
||||
auto result_column = ColumnUInt8::create(rows);
|
||||
auto result_column = ColumnUInt8::create(input_rows_count);
|
||||
auto * result_column_ptr = typeid_cast<ColumnUInt8 *>(result_column.get());
|
||||
GatherUtils::sliceHas(*sources[0], *sources[1], search_type, *result_column_ptr);
|
||||
|
||||
|
@ -172,11 +172,12 @@ public:
|
||||
for (size_t group = 1; group <= groups_count; ++group)
|
||||
all_matches.push_back(matched_groups[group]);
|
||||
|
||||
/// Additional limit to fail fast on supposedly incorrect usage.
|
||||
static constexpr size_t MAX_GROUPS_PER_ROW = 1000000;
|
||||
|
||||
if (all_matches.size() > MAX_GROUPS_PER_ROW)
|
||||
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size in the result of function {}", getName());
|
||||
/// Additional limit to fail fast on supposedly incorrect usage, arbitrary value.
|
||||
static constexpr size_t MAX_MATCHES_PER_ROW = 1000;
|
||||
if (matches_per_row > MAX_MATCHES_PER_ROW)
|
||||
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE,
|
||||
"Too many matches per row (> {}) in the result of function {}",
|
||||
MAX_MATCHES_PER_ROW, getName());
|
||||
|
||||
pos = matched_groups[0].data() + std::max<size_t>(1, matched_groups[0].size());
|
||||
|
||||
|
@ -17,7 +17,7 @@ private:
|
||||
bool nextImpl() override;
|
||||
|
||||
public:
|
||||
ReadBufferFromIStream(std::istream & istr_, size_t size = DBMS_DEFAULT_BUFFER_SIZE);
|
||||
explicit ReadBufferFromIStream(std::istream & istr_, size_t size = DBMS_DEFAULT_BUFFER_SIZE);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/UUID.h>
|
||||
#include <Poco/Net/SocketAddress.h>
|
||||
#include <Common/UInt128.h>
|
||||
#include <common/types.h>
|
||||
|
@ -373,7 +373,7 @@ struct ContextSharedPart
|
||||
std::atomic_size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default)
|
||||
String format_schema_path; /// Path to a directory that contains schema files used by input formats.
|
||||
ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers
|
||||
std::optional<SystemLogs> system_logs; /// Used to log queries and operations on parts
|
||||
std::unique_ptr<SystemLogs> system_logs; /// Used to log queries and operations on parts
|
||||
std::optional<StorageS3Settings> storage_s3_settings; /// Settings of S3 storage
|
||||
|
||||
RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml
|
||||
@ -442,24 +442,32 @@ struct ContextSharedPart
|
||||
|
||||
DatabaseCatalog::shutdown();
|
||||
|
||||
/// Preemptive destruction is important, because these objects may have a refcount to ContextShared (cyclic reference).
|
||||
/// TODO: Get rid of this.
|
||||
std::unique_ptr<SystemLogs> delete_system_logs;
|
||||
{
|
||||
auto lock = std::lock_guard(mutex);
|
||||
|
||||
system_logs.reset();
|
||||
embedded_dictionaries.reset();
|
||||
external_dictionaries_loader.reset();
|
||||
models_repository_guard.reset();
|
||||
external_models_loader.reset();
|
||||
buffer_flush_schedule_pool.reset();
|
||||
schedule_pool.reset();
|
||||
distributed_schedule_pool.reset();
|
||||
message_broker_schedule_pool.reset();
|
||||
ddl_worker.reset();
|
||||
/// Preemptive destruction is important, because these objects may have a refcount to ContextShared (cyclic reference).
|
||||
/// TODO: Get rid of this.
|
||||
|
||||
/// Stop trace collector if any
|
||||
trace_collector.reset();
|
||||
/// Stop zookeeper connection
|
||||
zookeeper.reset();
|
||||
delete_system_logs = std::move(system_logs);
|
||||
embedded_dictionaries.reset();
|
||||
external_dictionaries_loader.reset();
|
||||
models_repository_guard.reset();
|
||||
external_models_loader.reset();
|
||||
buffer_flush_schedule_pool.reset();
|
||||
schedule_pool.reset();
|
||||
distributed_schedule_pool.reset();
|
||||
message_broker_schedule_pool.reset();
|
||||
ddl_worker.reset();
|
||||
|
||||
/// Stop trace collector if any
|
||||
trace_collector.reset();
|
||||
/// Stop zookeeper connection
|
||||
zookeeper.reset();
|
||||
}
|
||||
|
||||
/// Can be removed w/o context lock
|
||||
delete_system_logs.reset();
|
||||
}
|
||||
|
||||
bool hasTraceCollector() const
|
||||
@ -1910,7 +1918,7 @@ void Context::setCluster(const String & cluster_name, const std::shared_ptr<Clus
|
||||
void Context::initializeSystemLogs()
|
||||
{
|
||||
auto lock = getLock();
|
||||
shared->system_logs.emplace(getGlobalContext(), getConfigRef());
|
||||
shared->system_logs = std::make_unique<SystemLogs>(getGlobalContext(), getConfigRef());
|
||||
}
|
||||
|
||||
void Context::initializeTraceCollector()
|
||||
@ -2615,6 +2623,20 @@ PartUUIDsPtr Context::getPartUUIDs()
|
||||
return part_uuids;
|
||||
}
|
||||
|
||||
|
||||
ReadTaskCallback Context::getReadTaskCallback() const
|
||||
{
|
||||
if (!next_task_callback.has_value())
|
||||
throw Exception(fmt::format("Next task callback is not set for query {}", getInitialQueryId()), ErrorCodes::LOGICAL_ERROR);
|
||||
return next_task_callback.value();
|
||||
}
|
||||
|
||||
|
||||
void Context::setReadTaskCallback(ReadTaskCallback && callback)
|
||||
{
|
||||
next_task_callback = callback;
|
||||
}
|
||||
|
||||
PartUUIDsPtr Context::getIgnoredPartUUIDs()
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
@ -128,6 +128,9 @@ using InputInitializer = std::function<void(ContextPtr, const StoragePtr &)>;
|
||||
/// Callback for reading blocks of data from client for function input()
|
||||
using InputBlocksReader = std::function<Block(ContextPtr)>;
|
||||
|
||||
/// Used in distributed task processing
|
||||
using ReadTaskCallback = std::function<String()>;
|
||||
|
||||
/// An empty interface for an arbitrary object that may be attached by a shared pointer
|
||||
/// to query context, when using ClickHouse as a library.
|
||||
struct IHostContext
|
||||
@ -189,6 +192,9 @@ private:
|
||||
TemporaryTablesMapping external_tables_mapping;
|
||||
Scalars scalars;
|
||||
|
||||
/// Fields for distributed s3 function
|
||||
std::optional<ReadTaskCallback> next_task_callback;
|
||||
|
||||
/// Record entities accessed by current query, and store this information in system.query_log.
|
||||
struct QueryAccessInfo
|
||||
{
|
||||
@ -769,6 +775,10 @@ public:
|
||||
|
||||
PartUUIDsPtr getPartUUIDs();
|
||||
PartUUIDsPtr getIgnoredPartUUIDs();
|
||||
|
||||
ReadTaskCallback getReadTaskCallback() const;
|
||||
void setReadTaskCallback(ReadTaskCallback && callback);
|
||||
|
||||
private:
|
||||
std::unique_lock<std::recursive_mutex> getLock() const;
|
||||
|
||||
|
@ -26,9 +26,9 @@ struct DatabaseAndTableWithAlias
|
||||
UUID uuid = UUIDHelpers::Nil;
|
||||
|
||||
DatabaseAndTableWithAlias() = default;
|
||||
DatabaseAndTableWithAlias(const ASTPtr & identifier_node, const String & current_database = "");
|
||||
DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database = "");
|
||||
DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database = "");
|
||||
explicit DatabaseAndTableWithAlias(const ASTPtr & identifier_node, const String & current_database = "");
|
||||
explicit DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database = "");
|
||||
explicit DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database = "");
|
||||
|
||||
/// "alias." or "table." if alias is empty
|
||||
String getQualifiedNamePrefix(bool with_dot = true) const;
|
||||
@ -80,7 +80,7 @@ private:
|
||||
void addAdditionalColumns(NamesAndTypesList & target, const NamesAndTypesList & addition)
|
||||
{
|
||||
target.insert(target.end(), addition.begin(), addition.end());
|
||||
for (auto & col : addition)
|
||||
for (const auto & col : addition)
|
||||
names.insert(col.name);
|
||||
}
|
||||
|
||||
|
@ -152,6 +152,8 @@ public:
|
||||
void shutdown() override
|
||||
{
|
||||
stopFlushThread();
|
||||
if (table)
|
||||
table->shutdown();
|
||||
}
|
||||
|
||||
String getName() override
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
bool second_with_brackets;
|
||||
|
||||
public:
|
||||
ASTPair(bool second_with_brackets_)
|
||||
explicit ASTPair(bool second_with_brackets_)
|
||||
: second_with_brackets(second_with_brackets_)
|
||||
{
|
||||
}
|
||||
@ -49,7 +49,7 @@ public:
|
||||
/// Has brackets around arguments
|
||||
bool has_brackets;
|
||||
|
||||
ASTFunctionWithKeyValueArguments(bool has_brackets_ = true)
|
||||
explicit ASTFunctionWithKeyValueArguments(bool has_brackets_ = true)
|
||||
: has_brackets(has_brackets_)
|
||||
{
|
||||
}
|
||||
|
@ -45,7 +45,7 @@ protected:
|
||||
class ParserIdentifier : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserIdentifier(bool allow_query_parameter_ = false) : allow_query_parameter(allow_query_parameter_) {}
|
||||
explicit ParserIdentifier(bool allow_query_parameter_ = false) : allow_query_parameter(allow_query_parameter_) {}
|
||||
protected:
|
||||
const char * getName() const override { return "identifier"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
@ -59,7 +59,7 @@ protected:
|
||||
class ParserCompoundIdentifier : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserCompoundIdentifier(bool table_name_with_optional_uuid_ = false, bool allow_query_parameter_ = false)
|
||||
explicit ParserCompoundIdentifier(bool table_name_with_optional_uuid_ = false, bool allow_query_parameter_ = false)
|
||||
: table_name_with_optional_uuid(table_name_with_optional_uuid_), allow_query_parameter(allow_query_parameter_)
|
||||
{
|
||||
}
|
||||
@ -85,7 +85,7 @@ public:
|
||||
using ColumnTransformers = MultiEnum<ColumnTransformer, UInt8>;
|
||||
static constexpr auto AllTransformers = ColumnTransformers{ColumnTransformer::APPLY, ColumnTransformer::EXCEPT, ColumnTransformer::REPLACE};
|
||||
|
||||
ParserColumnsTransformers(ColumnTransformers allowed_transformers_ = AllTransformers, bool is_strict_ = false)
|
||||
explicit ParserColumnsTransformers(ColumnTransformers allowed_transformers_ = AllTransformers, bool is_strict_ = false)
|
||||
: allowed_transformers(allowed_transformers_)
|
||||
, is_strict(is_strict_)
|
||||
{}
|
||||
@ -103,7 +103,7 @@ class ParserAsterisk : public IParserBase
|
||||
{
|
||||
public:
|
||||
using ColumnTransformers = ParserColumnsTransformers::ColumnTransformers;
|
||||
ParserAsterisk(ColumnTransformers allowed_transformers_ = ParserColumnsTransformers::AllTransformers)
|
||||
explicit ParserAsterisk(ColumnTransformers allowed_transformers_ = ParserColumnsTransformers::AllTransformers)
|
||||
: allowed_transformers(allowed_transformers_)
|
||||
{}
|
||||
|
||||
@ -129,7 +129,7 @@ class ParserColumnsMatcher : public IParserBase
|
||||
{
|
||||
public:
|
||||
using ColumnTransformers = ParserColumnsTransformers::ColumnTransformers;
|
||||
ParserColumnsMatcher(ColumnTransformers allowed_transformers_ = ParserColumnsTransformers::AllTransformers)
|
||||
explicit ParserColumnsMatcher(ColumnTransformers allowed_transformers_ = ParserColumnsTransformers::AllTransformers)
|
||||
: allowed_transformers(allowed_transformers_)
|
||||
{}
|
||||
|
||||
@ -149,7 +149,7 @@ protected:
|
||||
class ParserFunction : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserFunction(bool allow_function_parameters_ = true, bool is_table_function_ = false)
|
||||
explicit ParserFunction(bool allow_function_parameters_ = true, bool is_table_function_ = false)
|
||||
: allow_function_parameters(allow_function_parameters_), is_table_function(is_table_function_)
|
||||
{
|
||||
}
|
||||
|
@ -14,7 +14,7 @@ namespace DB
|
||||
/// Contains extra information about read data.
|
||||
struct RowReadExtension
|
||||
{
|
||||
/// IRowInputStream.read() output. It contains non zero for columns that actually read from the source and zero otherwise.
|
||||
/// IRowInputFormat::read output. It contains non zero for columns that actually read from the source and zero otherwise.
|
||||
/// It's used to attach defaults for partially filled rows.
|
||||
std::vector<UInt8> read_columns;
|
||||
};
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Poco/Net/NetException.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Server/IServer.h>
|
||||
#include <string>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -21,9 +22,9 @@ private:
|
||||
void run() override {}
|
||||
};
|
||||
public:
|
||||
KeeperTCPHandlerFactory(IServer & server_)
|
||||
KeeperTCPHandlerFactory(IServer & server_, bool secure)
|
||||
: server(server_)
|
||||
, log(&Poco::Logger::get("KeeperTCPHandlerFactory"))
|
||||
, log(&Poco::Logger::get(std::string{"KeeperTCP"} + (secure ? "S" : "") + "HandlerFactory"))
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -25,6 +25,7 @@
|
||||
#include <Interpreters/OpenTelemetrySpanLog.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
|
||||
#include <Storages/StorageS3Cluster.h>
|
||||
#include <Core/ExternalTable.h>
|
||||
#include <Storages/ColumnDefault.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
@ -33,6 +34,7 @@
|
||||
|
||||
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
|
||||
|
||||
#include "Core/Protocol.h"
|
||||
#include "TCPHandler.h"
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
@ -55,6 +57,7 @@ namespace ErrorCodes
|
||||
extern const int SOCKET_TIMEOUT;
|
||||
extern const int UNEXPECTED_PACKET_FROM_CLIENT;
|
||||
extern const int SUPPORT_IS_DISABLED;
|
||||
extern const int UNKNOWN_PROTOCOL;
|
||||
}
|
||||
|
||||
TCPHandler::TCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_, bool parse_proxy_protocol_, std::string server_display_name_)
|
||||
@ -285,6 +288,14 @@ void TCPHandler::runImpl()
|
||||
|
||||
customizeContext(query_context);
|
||||
|
||||
/// This callback is needed for requesting read tasks inside pipeline for distributed processing
|
||||
query_context->setReadTaskCallback([this]() -> String
|
||||
{
|
||||
std::lock_guard lock(task_callback_mutex);
|
||||
sendReadTaskRequestAssumeLocked();
|
||||
return receiveReadTaskResponseAssumeLocked();
|
||||
});
|
||||
|
||||
bool may_have_embedded_data = client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_CLIENT_SUPPORT_EMBEDDED_DATA;
|
||||
/// Processing Query
|
||||
state.io = executeQuery(state.query, query_context, false, state.stage, may_have_embedded_data);
|
||||
@ -644,6 +655,8 @@ void TCPHandler::processOrdinaryQueryWithProcessors()
|
||||
Block block;
|
||||
while (executor.pull(block, query_context->getSettingsRef().interactive_delay / 1000))
|
||||
{
|
||||
std::lock_guard lock(task_callback_mutex);
|
||||
|
||||
if (isQueryCancelled())
|
||||
{
|
||||
/// A packet was received requesting to stop execution of the request.
|
||||
@ -755,6 +768,13 @@ void TCPHandler::sendPartUUIDs()
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void TCPHandler::sendReadTaskRequestAssumeLocked()
|
||||
{
|
||||
writeVarUInt(Protocol::Server::ReadTaskRequest, *out);
|
||||
out->next();
|
||||
}
|
||||
|
||||
void TCPHandler::sendProfileInfo(const BlockStreamProfileInfo & info)
|
||||
{
|
||||
writeVarUInt(Protocol::Server::ProfileInfo, *out);
|
||||
@ -963,10 +983,10 @@ bool TCPHandler::receivePacket()
|
||||
UInt64 packet_type = 0;
|
||||
readVarUInt(packet_type, *in);
|
||||
|
||||
// std::cerr << "Server got packet: " << Protocol::Client::toString(packet_type) << "\n";
|
||||
|
||||
switch (packet_type)
|
||||
{
|
||||
case Protocol::Client::ReadTaskResponse:
|
||||
throw Exception("ReadTaskResponse must be received only after requesting in callback", ErrorCodes::LOGICAL_ERROR);
|
||||
case Protocol::Client::IgnoredPartUUIDs:
|
||||
/// Part uuids packet if any comes before query.
|
||||
receiveIgnoredPartUUIDs();
|
||||
@ -1016,6 +1036,34 @@ void TCPHandler::receiveIgnoredPartUUIDs()
|
||||
query_context->getIgnoredPartUUIDs()->add(uuids);
|
||||
}
|
||||
|
||||
|
||||
String TCPHandler::receiveReadTaskResponseAssumeLocked()
|
||||
{
|
||||
UInt64 packet_type = 0;
|
||||
readVarUInt(packet_type, *in);
|
||||
if (packet_type != Protocol::Client::ReadTaskResponse)
|
||||
{
|
||||
if (packet_type == Protocol::Client::Cancel)
|
||||
{
|
||||
state.is_cancelled = true;
|
||||
return {};
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(fmt::format("Received {} packet after requesting read task",
|
||||
Protocol::Client::toString(packet_type)), ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
|
||||
}
|
||||
}
|
||||
UInt64 version;
|
||||
readVarUInt(version, *in);
|
||||
if (version != DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION)
|
||||
throw Exception("Protocol version for distributed processing mismatched", ErrorCodes::UNKNOWN_PROTOCOL);
|
||||
String response;
|
||||
readStringBinary(response, *in);
|
||||
return response;
|
||||
}
|
||||
|
||||
|
||||
void TCPHandler::receiveClusterNameAndSalt()
|
||||
{
|
||||
readStringBinary(cluster, *in);
|
||||
|
@ -89,7 +89,7 @@ struct QueryState
|
||||
*this = QueryState();
|
||||
}
|
||||
|
||||
bool empty()
|
||||
bool empty() const
|
||||
{
|
||||
return is_empty;
|
||||
}
|
||||
@ -150,6 +150,7 @@ private:
|
||||
String cluster;
|
||||
String cluster_secret;
|
||||
|
||||
std::mutex task_callback_mutex;
|
||||
|
||||
/// At the moment, only one ongoing query in the connection is supported at a time.
|
||||
QueryState state;
|
||||
@ -169,9 +170,11 @@ private:
|
||||
bool receivePacket();
|
||||
void receiveQuery();
|
||||
void receiveIgnoredPartUUIDs();
|
||||
String receiveReadTaskResponseAssumeLocked();
|
||||
bool receiveData(bool scalar);
|
||||
bool readDataNext(const size_t & poll_interval, const int & receive_timeout);
|
||||
void readData(const Settings & connection_settings);
|
||||
void receiveClusterNameAndSalt();
|
||||
std::tuple<size_t, int> getReadTimeouts(const Settings & connection_settings);
|
||||
|
||||
[[noreturn]] void receiveUnexpectedData();
|
||||
@ -198,12 +201,11 @@ private:
|
||||
void sendLogs();
|
||||
void sendEndOfStream();
|
||||
void sendPartUUIDs();
|
||||
void sendReadTaskRequestAssumeLocked();
|
||||
void sendProfileInfo(const BlockStreamProfileInfo & info);
|
||||
void sendTotals(const Block & totals);
|
||||
void sendExtremes(const Block & extremes);
|
||||
|
||||
void receiveClusterNameAndSalt();
|
||||
|
||||
/// Creates state.block_in/block_out for blocks read/write, depending on whether compression is enabled.
|
||||
void initBlockInput();
|
||||
void initBlockOutput(const Block & block);
|
||||
|
@ -1327,7 +1327,7 @@ String KeyCondition::toString() const
|
||||
* A range of tuples specifies some subset of this space.
|
||||
*
|
||||
* Hyperrectangles will be the subrange of an n-dimensional space that is a direct product of one-dimensional ranges.
|
||||
* In this case, the one-dimensional range can be:
|
||||
* In this case, the one-dimensional range can be:
|
||||
* a point, a segment, an open interval, a half-open interval;
|
||||
* unlimited on the left, unlimited on the right ...
|
||||
*
|
||||
|
@ -452,7 +452,8 @@ StorageDistributed::StorageDistributed(
|
||||
const DistributedSettings & distributed_settings_,
|
||||
bool attach,
|
||||
ClusterPtr owned_cluster_)
|
||||
: StorageDistributed(id_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_, storage_policy_name_, relative_data_path_, distributed_settings_, attach, std::move(owned_cluster_))
|
||||
: StorageDistributed(id_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_,
|
||||
storage_policy_name_, relative_data_path_, distributed_settings_, attach, std::move(owned_cluster_))
|
||||
{
|
||||
remote_table_function_ptr = std::move(remote_table_function_ptr_);
|
||||
}
|
||||
@ -473,20 +474,15 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(
|
||||
ClusterPtr optimized_cluster = getOptimizedCluster(local_context, metadata_snapshot, query_info.query);
|
||||
if (optimized_cluster)
|
||||
{
|
||||
LOG_DEBUG(
|
||||
log,
|
||||
"Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}",
|
||||
makeFormattedListOfShards(optimized_cluster));
|
||||
LOG_DEBUG(log, "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}",
|
||||
makeFormattedListOfShards(optimized_cluster));
|
||||
cluster = optimized_cluster;
|
||||
query_info.optimized_cluster = cluster;
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_DEBUG(
|
||||
log,
|
||||
"Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the "
|
||||
"cluster{}",
|
||||
has_sharding_key ? "" : " (no sharding key)");
|
||||
LOG_DEBUG(log, "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster{}",
|
||||
has_sharding_key ? "" : " (no sharding key)");
|
||||
}
|
||||
}
|
||||
|
||||
@ -755,7 +751,7 @@ void StorageDistributed::alter(const AlterCommands & params, ContextPtr local_co
|
||||
|
||||
void StorageDistributed::startup()
|
||||
{
|
||||
if (remote_database.empty() && !remote_table_function_ptr)
|
||||
if (remote_database.empty() && !remote_table_function_ptr && !getCluster()->maybeCrossReplication())
|
||||
LOG_WARNING(log, "Name of remote database is empty. Default database will be used implicitly.");
|
||||
|
||||
if (!storage_policy)
|
||||
|
@ -159,8 +159,7 @@ bool StorageMerge::isRemote() const
|
||||
bool StorageMerge::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, ContextPtr query_context, const StorageMetadataPtr & /*metadata_snapshot*/) const
|
||||
{
|
||||
/// It's beneficial if it is true for at least one table.
|
||||
StorageListWithLocks selected_tables = getSelectedTables(
|
||||
query_context->getCurrentQueryId(), query_context->getSettingsRef());
|
||||
StorageListWithLocks selected_tables = getSelectedTables(query_context);
|
||||
|
||||
size_t i = 0;
|
||||
for (const auto & table : selected_tables)
|
||||
@ -250,8 +249,7 @@ Pipe StorageMerge::read(
|
||||
/** First we make list of selected tables to find out its size.
|
||||
* This is necessary to correctly pass the recommended number of threads to each table.
|
||||
*/
|
||||
StorageListWithLocks selected_tables
|
||||
= getSelectedTables(query_info, has_table_virtual_column, local_context->getCurrentQueryId(), local_context->getSettingsRef());
|
||||
StorageListWithLocks selected_tables = getSelectedTables(local_context, query_info.query, has_table_virtual_column);
|
||||
|
||||
if (selected_tables.empty())
|
||||
/// FIXME: do we support sampling in this case?
|
||||
@ -427,34 +425,20 @@ Pipe StorageMerge::createSources(
|
||||
return pipe;
|
||||
}
|
||||
|
||||
|
||||
StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const String & query_id, const Settings & settings) const
|
||||
{
|
||||
StorageListWithLocks selected_tables;
|
||||
auto iterator = getDatabaseIterator(getContext());
|
||||
|
||||
while (iterator->isValid())
|
||||
{
|
||||
const auto & table = iterator->table();
|
||||
if (table && table.get() != this)
|
||||
selected_tables.emplace_back(
|
||||
table, table->lockForShare(query_id, settings.lock_acquire_timeout), iterator->name());
|
||||
|
||||
iterator->next();
|
||||
}
|
||||
|
||||
return selected_tables;
|
||||
}
|
||||
|
||||
|
||||
StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(
|
||||
const SelectQueryInfo & query_info, bool has_virtual_column, const String & query_id, const Settings & settings) const
|
||||
ContextPtr query_context,
|
||||
const ASTPtr & query /* = nullptr */,
|
||||
bool filter_by_virtual_column /* = false */) const
|
||||
{
|
||||
const ASTPtr & query = query_info.query;
|
||||
assert(!filter_by_virtual_column || query);
|
||||
|
||||
const Settings & settings = query_context->getSettingsRef();
|
||||
StorageListWithLocks selected_tables;
|
||||
DatabaseTablesIteratorPtr iterator = getDatabaseIterator(getContext());
|
||||
|
||||
auto virtual_column = ColumnString::create();
|
||||
MutableColumnPtr table_name_virtual_column;
|
||||
if (filter_by_virtual_column)
|
||||
table_name_virtual_column = ColumnString::create();
|
||||
|
||||
while (iterator->isValid())
|
||||
{
|
||||
@ -467,18 +451,20 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(
|
||||
|
||||
if (storage.get() != this)
|
||||
{
|
||||
selected_tables.emplace_back(
|
||||
storage, storage->lockForShare(query_id, settings.lock_acquire_timeout), iterator->name());
|
||||
virtual_column->insert(iterator->name());
|
||||
auto table_lock = storage->lockForShare(query_context->getCurrentQueryId(), settings.lock_acquire_timeout);
|
||||
selected_tables.emplace_back(storage, std::move(table_lock), iterator->name());
|
||||
if (filter_by_virtual_column)
|
||||
table_name_virtual_column->insert(iterator->name());
|
||||
}
|
||||
|
||||
iterator->next();
|
||||
}
|
||||
|
||||
if (has_virtual_column)
|
||||
if (filter_by_virtual_column)
|
||||
{
|
||||
Block virtual_columns_block = Block{ColumnWithTypeAndName(std::move(virtual_column), std::make_shared<DataTypeString>(), "_table")};
|
||||
VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, getContext());
|
||||
/// Filter names of selected tables if there is a condition on "_table" virtual column in WHERE clause
|
||||
Block virtual_columns_block = Block{ColumnWithTypeAndName(std::move(table_name_virtual_column), std::make_shared<DataTypeString>(), "_table")};
|
||||
VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, query_context);
|
||||
auto values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_table");
|
||||
|
||||
/// Remove unused tables from the list
|
||||
@ -488,7 +474,6 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(
|
||||
return selected_tables;
|
||||
}
|
||||
|
||||
|
||||
DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator(ContextPtr local_context) const
|
||||
{
|
||||
try
|
||||
|
@ -55,10 +55,8 @@ private:
|
||||
using StorageWithLockAndName = std::tuple<StoragePtr, TableLockHolder, String>;
|
||||
using StorageListWithLocks = std::list<StorageWithLockAndName>;
|
||||
|
||||
StorageListWithLocks getSelectedTables(const String & query_id, const Settings & settings) const;
|
||||
|
||||
StorageMerge::StorageListWithLocks getSelectedTables(
|
||||
const SelectQueryInfo & query_info, bool has_virtual_column, const String & query_id, const Settings & settings) const;
|
||||
ContextPtr query_context, const ASTPtr & query = nullptr, bool filter_by_virtual_column = false) const;
|
||||
|
||||
template <typename F>
|
||||
StoragePtr getFirstTable(F && predicate) const;
|
||||
|
@ -11,7 +11,7 @@ class StorageProxy : public IStorage
|
||||
{
|
||||
public:
|
||||
|
||||
StorageProxy(const StorageID & table_id_) : IStorage(table_id_) {}
|
||||
explicit StorageProxy(const StorageID & table_id_) : IStorage(table_id_) {}
|
||||
|
||||
virtual StoragePtr getNested() const = 0;
|
||||
|
||||
|
@ -45,154 +45,267 @@ namespace ErrorCodes
|
||||
extern const int UNEXPECTED_EXPRESSION;
|
||||
extern const int S3_ERROR;
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
class StorageS3Source::DisclosedGlobIterator::Impl
|
||||
{
|
||||
class StorageS3Source : public SourceWithProgress
|
||||
|
||||
public:
|
||||
Impl(Aws::S3::S3Client & client_, const S3::URI & globbed_uri_)
|
||||
: client(client_), globbed_uri(globbed_uri_)
|
||||
{
|
||||
public:
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
static Block getHeader(Block sample_block, bool with_path_column, bool with_file_column)
|
||||
if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos)
|
||||
throw Exception("Expression can not have wildcards inside bucket name", ErrorCodes::UNEXPECTED_EXPRESSION);
|
||||
|
||||
const String key_prefix = globbed_uri.key.substr(0, globbed_uri.key.find_first_of("*?{"));
|
||||
|
||||
/// We don't have to list bucket, because there is no asterics.
|
||||
if (key_prefix.size() == globbed_uri.key.size())
|
||||
{
|
||||
if (with_path_column)
|
||||
sample_block.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_path"});
|
||||
if (with_file_column)
|
||||
sample_block.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_file"});
|
||||
|
||||
return sample_block;
|
||||
buffer.emplace_back(globbed_uri.key);
|
||||
buffer_iter = buffer.begin();
|
||||
is_finished = true;
|
||||
return;
|
||||
}
|
||||
|
||||
StorageS3Source(
|
||||
bool need_path,
|
||||
bool need_file,
|
||||
const String & format,
|
||||
String name_,
|
||||
const Block & sample_block,
|
||||
ContextPtr context,
|
||||
const ColumnsDescription & columns,
|
||||
UInt64 max_block_size,
|
||||
const CompressionMethod compression_method,
|
||||
const std::shared_ptr<Aws::S3::S3Client> & client,
|
||||
const String & bucket,
|
||||
const String & key)
|
||||
: SourceWithProgress(getHeader(sample_block, need_path, need_file))
|
||||
, name(std::move(name_))
|
||||
, with_file_column(need_file)
|
||||
, with_path_column(need_path)
|
||||
, file_path(bucket + "/" + key)
|
||||
{
|
||||
read_buf = wrapReadBufferWithCompressionMethod(std::make_unique<ReadBufferFromS3>(client, bucket, key), compression_method);
|
||||
auto input_format = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size);
|
||||
reader = std::make_shared<InputStreamFromInputFormat>(input_format);
|
||||
request.SetBucket(globbed_uri.bucket);
|
||||
request.SetPrefix(key_prefix);
|
||||
matcher = std::make_unique<re2::RE2>(makeRegexpPatternFromGlobs(globbed_uri.key));
|
||||
fillInternalBufferAssumeLocked();
|
||||
}
|
||||
|
||||
if (columns.hasDefaults())
|
||||
reader = std::make_shared<AddingDefaultsBlockInputStream>(reader, columns, context);
|
||||
String next()
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return nextAssumeLocked();
|
||||
}
|
||||
|
||||
private:
|
||||
|
||||
String nextAssumeLocked()
|
||||
{
|
||||
if (buffer_iter != buffer.end())
|
||||
{
|
||||
auto answer = *buffer_iter;
|
||||
++buffer_iter;
|
||||
return answer;
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
Chunk generate() override
|
||||
{
|
||||
if (!reader)
|
||||
return {};
|
||||
|
||||
if (!initialized)
|
||||
{
|
||||
reader->readSuffix();
|
||||
initialized = true;
|
||||
}
|
||||
|
||||
if (auto block = reader->read())
|
||||
{
|
||||
auto columns = block.getColumns();
|
||||
UInt64 num_rows = block.rows();
|
||||
|
||||
if (with_path_column)
|
||||
columns.push_back(DataTypeString().createColumnConst(num_rows, file_path)->convertToFullColumnIfConst());
|
||||
if (with_file_column)
|
||||
{
|
||||
size_t last_slash_pos = file_path.find_last_of('/');
|
||||
columns.push_back(DataTypeString().createColumnConst(num_rows, file_path.substr(
|
||||
last_slash_pos + 1))->convertToFullColumnIfConst());
|
||||
}
|
||||
|
||||
return Chunk(std::move(columns), num_rows);
|
||||
}
|
||||
|
||||
reader.reset();
|
||||
|
||||
if (is_finished)
|
||||
return {};
|
||||
}
|
||||
|
||||
private:
|
||||
String name;
|
||||
std::unique_ptr<ReadBuffer> read_buf;
|
||||
BlockInputStreamPtr reader;
|
||||
bool initialized = false;
|
||||
bool with_file_column = false;
|
||||
bool with_path_column = false;
|
||||
String file_path;
|
||||
};
|
||||
fillInternalBufferAssumeLocked();
|
||||
|
||||
class StorageS3BlockOutputStream : public IBlockOutputStream
|
||||
return nextAssumeLocked();
|
||||
}
|
||||
|
||||
void fillInternalBufferAssumeLocked()
|
||||
{
|
||||
public:
|
||||
StorageS3BlockOutputStream(
|
||||
const String & format,
|
||||
const Block & sample_block_,
|
||||
ContextPtr context,
|
||||
const CompressionMethod compression_method,
|
||||
const std::shared_ptr<Aws::S3::S3Client> & client,
|
||||
const String & bucket,
|
||||
const String & key,
|
||||
size_t min_upload_part_size,
|
||||
size_t max_single_part_upload_size)
|
||||
: sample_block(sample_block_)
|
||||
{
|
||||
write_buf = wrapWriteBufferWithCompressionMethod(
|
||||
std::make_unique<WriteBufferFromS3>(client, bucket, key, min_upload_part_size, max_single_part_upload_size), compression_method, 3);
|
||||
writer = FormatFactory::instance().getOutputStreamParallelIfPossible(format, *write_buf, sample_block, context);
|
||||
}
|
||||
buffer.clear();
|
||||
|
||||
Block getHeader() const override
|
||||
{
|
||||
return sample_block;
|
||||
}
|
||||
outcome = client.ListObjectsV2(request);
|
||||
if (!outcome.IsSuccess())
|
||||
throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}",
|
||||
quoteString(request.GetBucket()), quoteString(request.GetPrefix()),
|
||||
backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage()));
|
||||
|
||||
void write(const Block & block) override
|
||||
{
|
||||
writer->write(block);
|
||||
}
|
||||
const auto & result_batch = outcome.GetResult().GetContents();
|
||||
|
||||
void writePrefix() override
|
||||
buffer.reserve(result_batch.size());
|
||||
for (const auto & row : result_batch)
|
||||
{
|
||||
writer->writePrefix();
|
||||
String key = row.GetKey();
|
||||
if (re2::RE2::FullMatch(key, *matcher))
|
||||
buffer.emplace_back(std::move(key));
|
||||
}
|
||||
/// Set iterator only after the whole batch is processed
|
||||
buffer_iter = buffer.begin();
|
||||
|
||||
void flush() override
|
||||
{
|
||||
writer->flush();
|
||||
}
|
||||
request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken());
|
||||
|
||||
void writeSuffix() override
|
||||
{
|
||||
writer->writeSuffix();
|
||||
writer->flush();
|
||||
write_buf->finalize();
|
||||
}
|
||||
/// It returns false when all objects were returned
|
||||
is_finished = !outcome.GetResult().GetIsTruncated();
|
||||
}
|
||||
|
||||
private:
|
||||
Block sample_block;
|
||||
std::unique_ptr<WriteBuffer> write_buf;
|
||||
BlockOutputStreamPtr writer;
|
||||
};
|
||||
std::mutex mutex;
|
||||
Strings buffer;
|
||||
Strings::iterator buffer_iter;
|
||||
Aws::S3::S3Client client;
|
||||
S3::URI globbed_uri;
|
||||
Aws::S3::Model::ListObjectsV2Request request;
|
||||
Aws::S3::Model::ListObjectsV2Outcome outcome;
|
||||
std::unique_ptr<re2::RE2> matcher;
|
||||
bool is_finished{false};
|
||||
};
|
||||
|
||||
StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator(Aws::S3::S3Client & client_, const S3::URI & globbed_uri_)
|
||||
: pimpl(std::make_shared<StorageS3Source::DisclosedGlobIterator::Impl>(client_, globbed_uri_)) {}
|
||||
|
||||
String StorageS3Source::DisclosedGlobIterator::next()
|
||||
{
|
||||
return pimpl->next();
|
||||
}
|
||||
|
||||
|
||||
Block StorageS3Source::getHeader(Block sample_block, bool with_path_column, bool with_file_column)
|
||||
{
|
||||
if (with_path_column)
|
||||
sample_block.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_path"});
|
||||
if (with_file_column)
|
||||
sample_block.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_file"});
|
||||
|
||||
return sample_block;
|
||||
}
|
||||
|
||||
StorageS3Source::StorageS3Source(
|
||||
bool need_path,
|
||||
bool need_file,
|
||||
const String & format_,
|
||||
String name_,
|
||||
const Block & sample_block_,
|
||||
ContextPtr context_,
|
||||
const ColumnsDescription & columns_,
|
||||
UInt64 max_block_size_,
|
||||
const String compression_hint_,
|
||||
const std::shared_ptr<Aws::S3::S3Client> & client_,
|
||||
const String & bucket_,
|
||||
std::shared_ptr<IteratorWrapper> file_iterator_)
|
||||
: SourceWithProgress(getHeader(sample_block_, need_path, need_file))
|
||||
, WithContext(context_)
|
||||
, name(std::move(name_))
|
||||
, bucket(bucket_)
|
||||
, format(format_)
|
||||
, columns_desc(columns_)
|
||||
, max_block_size(max_block_size_)
|
||||
, compression_hint(compression_hint_)
|
||||
, client(client_)
|
||||
, sample_block(sample_block_)
|
||||
, with_file_column(need_file)
|
||||
, with_path_column(need_path)
|
||||
, file_iterator(file_iterator_)
|
||||
{
|
||||
initialize();
|
||||
}
|
||||
|
||||
|
||||
bool StorageS3Source::initialize()
|
||||
{
|
||||
String current_key = (*file_iterator)();
|
||||
if (current_key.empty())
|
||||
return false;
|
||||
|
||||
file_path = bucket + "/" + current_key;
|
||||
|
||||
read_buf = wrapReadBufferWithCompressionMethod(
|
||||
std::make_unique<ReadBufferFromS3>(client, bucket, current_key), chooseCompressionMethod(current_key, compression_hint));
|
||||
auto input_format = FormatFactory::instance().getInput(format, *read_buf, sample_block, getContext(), max_block_size);
|
||||
reader = std::make_shared<InputStreamFromInputFormat>(input_format);
|
||||
|
||||
if (columns_desc.hasDefaults())
|
||||
reader = std::make_shared<AddingDefaultsBlockInputStream>(reader, columns_desc, getContext());
|
||||
|
||||
initialized = false;
|
||||
return true;
|
||||
}
|
||||
|
||||
String StorageS3Source::getName() const
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
Chunk StorageS3Source::generate()
|
||||
{
|
||||
if (!reader)
|
||||
return {};
|
||||
|
||||
if (!initialized)
|
||||
{
|
||||
reader->readPrefix();
|
||||
initialized = true;
|
||||
}
|
||||
|
||||
if (auto block = reader->read())
|
||||
{
|
||||
auto columns = block.getColumns();
|
||||
UInt64 num_rows = block.rows();
|
||||
|
||||
if (with_path_column)
|
||||
columns.push_back(DataTypeString().createColumnConst(num_rows, file_path)->convertToFullColumnIfConst());
|
||||
if (with_file_column)
|
||||
{
|
||||
size_t last_slash_pos = file_path.find_last_of('/');
|
||||
columns.push_back(DataTypeString().createColumnConst(num_rows, file_path.substr(
|
||||
last_slash_pos + 1))->convertToFullColumnIfConst());
|
||||
}
|
||||
|
||||
return Chunk(std::move(columns), num_rows);
|
||||
}
|
||||
|
||||
reader->readSuffix();
|
||||
reader.reset();
|
||||
read_buf.reset();
|
||||
|
||||
if (!initialize())
|
||||
return {};
|
||||
|
||||
return generate();
|
||||
}
|
||||
|
||||
|
||||
class StorageS3BlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
StorageS3BlockOutputStream(
|
||||
const String & format,
|
||||
const Block & sample_block_,
|
||||
ContextPtr context,
|
||||
const CompressionMethod compression_method,
|
||||
const std::shared_ptr<Aws::S3::S3Client> & client,
|
||||
const String & bucket,
|
||||
const String & key,
|
||||
size_t min_upload_part_size,
|
||||
size_t max_single_part_upload_size)
|
||||
: sample_block(sample_block_)
|
||||
{
|
||||
write_buf = wrapWriteBufferWithCompressionMethod(
|
||||
std::make_unique<WriteBufferFromS3>(client, bucket, key, min_upload_part_size, max_single_part_upload_size), compression_method, 3);
|
||||
writer = FormatFactory::instance().getOutputStreamParallelIfPossible(format, *write_buf, sample_block, context);
|
||||
}
|
||||
|
||||
Block getHeader() const override
|
||||
{
|
||||
return sample_block;
|
||||
}
|
||||
|
||||
void write(const Block & block) override
|
||||
{
|
||||
writer->write(block);
|
||||
}
|
||||
|
||||
void writePrefix() override
|
||||
{
|
||||
writer->writePrefix();
|
||||
}
|
||||
|
||||
void flush() override
|
||||
{
|
||||
writer->flush();
|
||||
}
|
||||
|
||||
void writeSuffix() override
|
||||
{
|
||||
writer->writeSuffix();
|
||||
writer->flush();
|
||||
write_buf->finalize();
|
||||
}
|
||||
|
||||
private:
|
||||
Block sample_block;
|
||||
std::unique_ptr<WriteBuffer> write_buf;
|
||||
BlockOutputStreamPtr writer;
|
||||
};
|
||||
|
||||
|
||||
StorageS3::StorageS3(
|
||||
const S3::URI & uri_,
|
||||
const String & access_key_id_,
|
||||
@ -205,84 +318,23 @@ StorageS3::StorageS3(
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
ContextPtr context_,
|
||||
const String & compression_method_)
|
||||
const String & compression_method_,
|
||||
bool distributed_processing_)
|
||||
: IStorage(table_id_)
|
||||
, WithContext(context_->getGlobalContext())
|
||||
, uri(uri_)
|
||||
, access_key_id(access_key_id_)
|
||||
, secret_access_key(secret_access_key_)
|
||||
, max_connections(max_connections_)
|
||||
, client_auth{uri_, access_key_id_, secret_access_key_, max_connections_, {}, {}} /// Client and settings will be updated later
|
||||
, format_name(format_name_)
|
||||
, min_upload_part_size(min_upload_part_size_)
|
||||
, max_single_part_upload_size(max_single_part_upload_size_)
|
||||
, compression_method(compression_method_)
|
||||
, name(uri_.storage_name)
|
||||
, distributed_processing(distributed_processing_)
|
||||
{
|
||||
getContext()->getRemoteHostFilter().checkURL(uri_.uri);
|
||||
context_->getGlobalContext()->getRemoteHostFilter().checkURL(uri_.uri);
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(columns_);
|
||||
storage_metadata.setConstraints(constraints_);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
updateAuthSettings(context_);
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
/* "Recursive" directory listing with matched paths as a result.
|
||||
* Have the same method in StorageFile.
|
||||
*/
|
||||
Strings listFilesWithRegexpMatching(Aws::S3::S3Client & client, const S3::URI & globbed_uri)
|
||||
{
|
||||
if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos)
|
||||
{
|
||||
throw Exception("Expression can not have wildcards inside bucket name", ErrorCodes::UNEXPECTED_EXPRESSION);
|
||||
}
|
||||
|
||||
const String key_prefix = globbed_uri.key.substr(0, globbed_uri.key.find_first_of("*?{"));
|
||||
if (key_prefix.size() == globbed_uri.key.size())
|
||||
{
|
||||
return {globbed_uri.key};
|
||||
}
|
||||
|
||||
Aws::S3::Model::ListObjectsV2Request request;
|
||||
request.SetBucket(globbed_uri.bucket);
|
||||
request.SetPrefix(key_prefix);
|
||||
|
||||
re2::RE2 matcher(makeRegexpPatternFromGlobs(globbed_uri.key));
|
||||
Strings result;
|
||||
Aws::S3::Model::ListObjectsV2Outcome outcome;
|
||||
int page = 0;
|
||||
do
|
||||
{
|
||||
++page;
|
||||
outcome = client.ListObjectsV2(request);
|
||||
if (!outcome.IsSuccess())
|
||||
{
|
||||
if (page > 1)
|
||||
throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, page {}, S3 exception: {}, message: {}",
|
||||
quoteString(request.GetBucket()), quoteString(request.GetPrefix()), page,
|
||||
backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage()));
|
||||
|
||||
throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}",
|
||||
quoteString(request.GetBucket()), quoteString(request.GetPrefix()),
|
||||
backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage()));
|
||||
}
|
||||
|
||||
for (const auto & row : outcome.GetResult().GetContents())
|
||||
{
|
||||
String key = row.GetKey();
|
||||
if (re2::RE2::FullMatch(key, matcher))
|
||||
result.emplace_back(std::move(key));
|
||||
}
|
||||
|
||||
request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken());
|
||||
}
|
||||
while (outcome.GetResult().GetIsTruncated());
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
updateClientAndAuthSettings(context_, client_auth);
|
||||
}
|
||||
|
||||
|
||||
@ -295,7 +347,7 @@ Pipe StorageS3::read(
|
||||
size_t max_block_size,
|
||||
unsigned num_streams)
|
||||
{
|
||||
updateAuthSettings(local_context);
|
||||
updateClientAndAuthSettings(local_context, client_auth);
|
||||
|
||||
Pipes pipes;
|
||||
bool need_path_column = false;
|
||||
@ -308,7 +360,26 @@ Pipe StorageS3::read(
|
||||
need_file_column = true;
|
||||
}
|
||||
|
||||
for (const String & key : listFilesWithRegexpMatching(*client, uri))
|
||||
std::shared_ptr<StorageS3Source::IteratorWrapper> iterator_wrapper{nullptr};
|
||||
if (distributed_processing)
|
||||
{
|
||||
iterator_wrapper = std::make_shared<StorageS3Source::IteratorWrapper>(
|
||||
[callback = local_context->getReadTaskCallback()]() -> String {
|
||||
return callback();
|
||||
});
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Iterate through disclosed globs and make a source for each file
|
||||
auto glob_iterator = std::make_shared<StorageS3Source::DisclosedGlobIterator>(*client_auth.client, client_auth.uri);
|
||||
iterator_wrapper = std::make_shared<StorageS3Source::IteratorWrapper>([glob_iterator]()
|
||||
{
|
||||
return glob_iterator->next();
|
||||
});
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < num_streams; ++i)
|
||||
{
|
||||
pipes.emplace_back(std::make_shared<StorageS3Source>(
|
||||
need_path_column,
|
||||
need_file_column,
|
||||
@ -318,63 +389,62 @@ Pipe StorageS3::read(
|
||||
local_context,
|
||||
metadata_snapshot->getColumns(),
|
||||
max_block_size,
|
||||
chooseCompressionMethod(uri.key, compression_method),
|
||||
client,
|
||||
uri.bucket,
|
||||
key));
|
||||
|
||||
compression_method,
|
||||
client_auth.client,
|
||||
client_auth.uri.bucket,
|
||||
iterator_wrapper));
|
||||
}
|
||||
auto pipe = Pipe::unitePipes(std::move(pipes));
|
||||
// It's possible to have many buckets read from s3, resize(num_streams) might open too many handles at the same time.
|
||||
// Using narrowPipe instead.
|
||||
|
||||
narrowPipe(pipe, num_streams);
|
||||
return pipe;
|
||||
}
|
||||
|
||||
BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context)
|
||||
{
|
||||
updateAuthSettings(local_context);
|
||||
updateClientAndAuthSettings(local_context, client_auth);
|
||||
return std::make_shared<StorageS3BlockOutputStream>(
|
||||
format_name,
|
||||
metadata_snapshot->getSampleBlock(),
|
||||
getContext(),
|
||||
chooseCompressionMethod(uri.key, compression_method),
|
||||
client,
|
||||
uri.bucket,
|
||||
uri.key,
|
||||
local_context,
|
||||
chooseCompressionMethod(client_auth.uri.key, compression_method),
|
||||
client_auth.client,
|
||||
client_auth.uri.bucket,
|
||||
client_auth.uri.key,
|
||||
min_upload_part_size,
|
||||
max_single_part_upload_size);
|
||||
}
|
||||
|
||||
void StorageS3::updateAuthSettings(ContextPtr local_context)
|
||||
void StorageS3::updateClientAndAuthSettings(ContextPtr ctx, StorageS3::ClientAuthentificaiton & upd)
|
||||
{
|
||||
auto settings = local_context->getStorageS3Settings().getSettings(uri.uri.toString());
|
||||
if (client && (!access_key_id.empty() || settings == auth_settings))
|
||||
auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString());
|
||||
if (upd.client && (!upd.access_key_id.empty() || settings == upd.auth_settings))
|
||||
return;
|
||||
|
||||
Aws::Auth::AWSCredentials credentials(access_key_id, secret_access_key);
|
||||
Aws::Auth::AWSCredentials credentials(upd.access_key_id, upd.secret_access_key);
|
||||
HeaderCollection headers;
|
||||
if (access_key_id.empty())
|
||||
if (upd.access_key_id.empty())
|
||||
{
|
||||
credentials = Aws::Auth::AWSCredentials(settings.access_key_id, settings.secret_access_key);
|
||||
headers = settings.headers;
|
||||
}
|
||||
|
||||
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
|
||||
local_context->getRemoteHostFilter(), local_context->getGlobalContext()->getSettingsRef().s3_max_redirects);
|
||||
ctx->getRemoteHostFilter(), ctx->getGlobalContext()->getSettingsRef().s3_max_redirects);
|
||||
|
||||
client_configuration.endpointOverride = uri.endpoint;
|
||||
client_configuration.maxConnections = max_connections;
|
||||
client_configuration.endpointOverride = upd.uri.endpoint;
|
||||
client_configuration.maxConnections = upd.max_connections;
|
||||
|
||||
client = S3::ClientFactory::instance().create(
|
||||
upd.client = S3::ClientFactory::instance().create(
|
||||
client_configuration,
|
||||
uri.is_virtual_hosted_style,
|
||||
upd.uri.is_virtual_hosted_style,
|
||||
credentials.GetAWSAccessKeyId(),
|
||||
credentials.GetAWSSecretKey(),
|
||||
settings.server_side_encryption_customer_key_base64,
|
||||
std::move(headers),
|
||||
settings.use_environment_credentials.value_or(getContext()->getConfigRef().getBool("s3.use_environment_credentials", false)));
|
||||
settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", false)));
|
||||
|
||||
auth_settings = std::move(settings);
|
||||
upd.auth_settings = std::move(settings);
|
||||
}
|
||||
|
||||
void registerStorageS3Impl(const String & name, StorageFactory & factory)
|
||||
@ -385,7 +455,8 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory)
|
||||
|
||||
if (engine_args.size() < 2 || engine_args.size() > 5)
|
||||
throw Exception(
|
||||
"Storage S3 requires 2 to 5 arguments: url, [access_key_id, secret_access_key], name of used format and [compression_method].", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
"Storage S3 requires 2 to 5 arguments: url, [access_key_id, secret_access_key], name of used format and [compression_method].",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
for (auto & engine_arg : engine_args)
|
||||
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext());
|
||||
|
@ -4,11 +4,20 @@
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <Core/Types.h>
|
||||
|
||||
#include <Compression/CompressionInfo.h>
|
||||
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/StorageS3Settings.h>
|
||||
|
||||
#include <Processors/Sources/SourceWithProgress.h>
|
||||
#include <Poco/URI.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
#include <IO/S3Common.h>
|
||||
#include <IO/CompressionMethod.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace Aws::S3
|
||||
{
|
||||
@ -18,6 +27,66 @@ namespace Aws::S3
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class StorageS3SequentialSource;
|
||||
class StorageS3Source : public SourceWithProgress, WithContext
|
||||
{
|
||||
public:
|
||||
class DisclosedGlobIterator
|
||||
{
|
||||
public:
|
||||
DisclosedGlobIterator(Aws::S3::S3Client &, const S3::URI &);
|
||||
String next();
|
||||
private:
|
||||
class Impl;
|
||||
/// shared_ptr to have copy constructor
|
||||
std::shared_ptr<Impl> pimpl;
|
||||
};
|
||||
|
||||
using IteratorWrapper = std::function<String()>;
|
||||
|
||||
static Block getHeader(Block sample_block, bool with_path_column, bool with_file_column);
|
||||
|
||||
StorageS3Source(
|
||||
bool need_path,
|
||||
bool need_file,
|
||||
const String & format,
|
||||
String name_,
|
||||
const Block & sample_block,
|
||||
ContextPtr context_,
|
||||
const ColumnsDescription & columns_,
|
||||
UInt64 max_block_size_,
|
||||
const String compression_hint_,
|
||||
const std::shared_ptr<Aws::S3::S3Client> & client_,
|
||||
const String & bucket,
|
||||
std::shared_ptr<IteratorWrapper> file_iterator_);
|
||||
|
||||
String getName() const override;
|
||||
|
||||
Chunk generate() override;
|
||||
|
||||
private:
|
||||
String name;
|
||||
String bucket;
|
||||
String file_path;
|
||||
String format;
|
||||
ColumnsDescription columns_desc;
|
||||
UInt64 max_block_size;
|
||||
String compression_hint;
|
||||
std::shared_ptr<Aws::S3::S3Client> client;
|
||||
Block sample_block;
|
||||
|
||||
|
||||
std::unique_ptr<ReadBuffer> read_buf;
|
||||
BlockInputStreamPtr reader;
|
||||
bool initialized = false;
|
||||
bool with_file_column = false;
|
||||
bool with_path_column = false;
|
||||
std::shared_ptr<IteratorWrapper> file_iterator;
|
||||
|
||||
/// Recreate ReadBuffer and BlockInputStream for each file.
|
||||
bool initialize();
|
||||
};
|
||||
|
||||
/**
|
||||
* This class represents table engine for external S3 urls.
|
||||
* It sends HTTP GET to server when select is called and
|
||||
@ -37,7 +106,8 @@ public:
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
ContextPtr context_,
|
||||
const String & compression_method_ = "");
|
||||
const String & compression_method_ = "",
|
||||
bool distributed_processing_ = false);
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -58,20 +128,30 @@ public:
|
||||
NamesAndTypesList getVirtuals() const override;
|
||||
|
||||
private:
|
||||
const S3::URI uri;
|
||||
const String access_key_id;
|
||||
const String secret_access_key;
|
||||
const UInt64 max_connections;
|
||||
|
||||
friend class StorageS3Cluster;
|
||||
friend class TableFunctionS3Cluster;
|
||||
|
||||
struct ClientAuthentificaiton
|
||||
{
|
||||
const S3::URI uri;
|
||||
const String access_key_id;
|
||||
const String secret_access_key;
|
||||
const UInt64 max_connections;
|
||||
std::shared_ptr<Aws::S3::S3Client> client;
|
||||
S3AuthSettings auth_settings;
|
||||
};
|
||||
|
||||
ClientAuthentificaiton client_auth;
|
||||
|
||||
String format_name;
|
||||
size_t min_upload_part_size;
|
||||
size_t max_single_part_upload_size;
|
||||
String compression_method;
|
||||
std::shared_ptr<Aws::S3::S3Client> client;
|
||||
String name;
|
||||
S3AuthSettings auth_settings;
|
||||
const bool distributed_processing;
|
||||
|
||||
void updateAuthSettings(ContextPtr context);
|
||||
static void updateClientAndAuthSettings(ContextPtr, ClientAuthentificaiton &);
|
||||
};
|
||||
|
||||
}
|
||||
|
166
src/Storages/StorageS3Cluster.cpp
Normal file
166
src/Storages/StorageS3Cluster.cpp
Normal file
@ -0,0 +1,166 @@
|
||||
#include "Storages/StorageS3Cluster.h"
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include "Common/Exception.h"
|
||||
#include <Common/Throttler.h>
|
||||
#include "Client/Connection.h"
|
||||
#include "Core/QueryProcessingStage.h"
|
||||
#include <Core/UUID.h>
|
||||
#include "DataStreams/RemoteBlockInputStream.h"
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <IO/ReadBufferFromS3.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromS3.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/getHeaderForProcessingStage.h>
|
||||
#include <Interpreters/SelectQueryOptions.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <DataStreams/AddingDefaultsBlockInputStream.h>
|
||||
#include <DataStreams/narrowBlockInputStreams.h>
|
||||
#include <Processors/Formats/InputStreamFromInputFormat.h>
|
||||
#include <Processors/Pipe.h>
|
||||
#include <Processors/Sources/SourceFromInputStream.h>
|
||||
#include "Processors/Sources/SourceWithProgress.h"
|
||||
#include <Processors/Sources/RemoteSource.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
#include <aws/core/auth/AWSCredentials.h>
|
||||
#include <aws/s3/S3Client.h>
|
||||
#include <aws/s3/model/ListObjectsV2Request.h>
|
||||
|
||||
#include <ios>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <thread>
|
||||
#include <cassert>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
StorageS3Cluster::StorageS3Cluster(
|
||||
const String & filename_,
|
||||
const String & access_key_id_,
|
||||
const String & secret_access_key_,
|
||||
const StorageID & table_id_,
|
||||
String cluster_name_,
|
||||
const String & format_name_,
|
||||
UInt64 max_connections_,
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
ContextPtr context_,
|
||||
const String & compression_method_)
|
||||
: IStorage(table_id_)
|
||||
, client_auth{S3::URI{Poco::URI{filename_}}, access_key_id_, secret_access_key_, max_connections_, {}, {}}
|
||||
, filename(filename_)
|
||||
, cluster_name(cluster_name_)
|
||||
, format_name(format_name_)
|
||||
, compression_method(compression_method_)
|
||||
{
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(columns_);
|
||||
storage_metadata.setConstraints(constraints_);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
StorageS3::updateClientAndAuthSettings(context_, client_auth);
|
||||
}
|
||||
|
||||
/// The code executes on initiator
|
||||
Pipe StorageS3Cluster::read(
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
ContextPtr context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
size_t /*max_block_size*/,
|
||||
unsigned /*num_streams*/)
|
||||
{
|
||||
StorageS3::updateClientAndAuthSettings(context, client_auth);
|
||||
|
||||
auto cluster = context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettings());
|
||||
S3::URI s3_uri(Poco::URI{filename});
|
||||
StorageS3::updateClientAndAuthSettings(context, client_auth);
|
||||
|
||||
auto iterator = std::make_shared<StorageS3Source::DisclosedGlobIterator>(*client_auth.client, client_auth.uri);
|
||||
auto callback = std::make_shared<StorageS3Source::IteratorWrapper>([iterator]() mutable -> String
|
||||
{
|
||||
return iterator->next();
|
||||
});
|
||||
|
||||
/// Calculate the header. This is significant, because some columns could be thrown away in some cases like query with count(*)
|
||||
Block header =
|
||||
InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
|
||||
|
||||
const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{};
|
||||
|
||||
Pipes pipes;
|
||||
connections.reserve(cluster->getShardCount());
|
||||
|
||||
const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState;
|
||||
|
||||
for (const auto & replicas : cluster->getShardsAddresses())
|
||||
{
|
||||
/// There will be only one replica, because we consider each replica as a shard
|
||||
for (const auto & node : replicas)
|
||||
{
|
||||
connections.emplace_back(std::make_shared<Connection>(
|
||||
node.host_name, node.port, context->getGlobalContext()->getCurrentDatabase(),
|
||||
node.user, node.password, node.cluster, node.cluster_secret,
|
||||
"S3ClusterInititiator",
|
||||
node.compression,
|
||||
node.secure
|
||||
));
|
||||
|
||||
/// For unknown reason global context is passed to IStorage::read() method
|
||||
/// So, task_identifier is passed as constructor argument. It is more obvious.
|
||||
auto remote_query_executor = std::make_shared<RemoteQueryExecutor>(
|
||||
*connections.back(), queryToString(query_info.query), header, context,
|
||||
/*throttler=*/nullptr, scalars, Tables(), processed_stage, callback);
|
||||
|
||||
pipes.emplace_back(std::make_shared<RemoteSource>(remote_query_executor, add_agg_info, false));
|
||||
}
|
||||
}
|
||||
|
||||
metadata_snapshot->check(column_names, getVirtuals(), getStorageID());
|
||||
return Pipe::unitePipes(std::move(pipes));
|
||||
}
|
||||
|
||||
QueryProcessingStage::Enum StorageS3Cluster::getQueryProcessingStage(
|
||||
ContextPtr context, QueryProcessingStage::Enum to_stage, SelectQueryInfo &) const
|
||||
{
|
||||
/// Initiator executes query on remote node.
|
||||
if (context->getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY)
|
||||
if (to_stage >= QueryProcessingStage::Enum::WithMergeableState)
|
||||
return QueryProcessingStage::Enum::WithMergeableState;
|
||||
|
||||
/// Follower just reads the data.
|
||||
return QueryProcessingStage::Enum::FetchColumns;
|
||||
}
|
||||
|
||||
|
||||
NamesAndTypesList StorageS3Cluster::getVirtuals() const
|
||||
{
|
||||
return NamesAndTypesList{
|
||||
{"_path", std::make_shared<DataTypeString>()},
|
||||
{"_file", std::make_shared<DataTypeString>()}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
#endif
|
63
src/Storages/StorageS3Cluster.h
Normal file
63
src/Storages/StorageS3Cluster.h
Normal file
@ -0,0 +1,63 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include "Client/Connection.h"
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <IO/S3Common.h>
|
||||
#include <Storages/StorageS3.h>
|
||||
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include "ext/shared_ptr_helper.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
struct ClientAuthentificationBuilder
|
||||
{
|
||||
String access_key_id;
|
||||
String secret_access_key;
|
||||
UInt64 max_connections;
|
||||
};
|
||||
|
||||
class StorageS3Cluster : public ext::shared_ptr_helper<StorageS3Cluster>, public IStorage
|
||||
{
|
||||
friend struct ext::shared_ptr_helper<StorageS3Cluster>;
|
||||
public:
|
||||
std::string getName() const override { return "S3Cluster"; }
|
||||
|
||||
Pipe read(const Names &, const StorageMetadataPtr &, SelectQueryInfo &,
|
||||
ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, unsigned /*num_streams*/) override;
|
||||
|
||||
QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, SelectQueryInfo &) const override;
|
||||
|
||||
NamesAndTypesList getVirtuals() const override;
|
||||
|
||||
protected:
|
||||
StorageS3Cluster(
|
||||
const String & filename_, const String & access_key_id_, const String & secret_access_key_, const StorageID & table_id_,
|
||||
String cluster_name_, const String & format_name_, UInt64 max_connections_, const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_, ContextPtr context_, const String & compression_method_);
|
||||
|
||||
private:
|
||||
/// Connections from initiator to other nodes
|
||||
std::vector<std::shared_ptr<Connection>> connections;
|
||||
StorageS3::ClientAuthentificaiton client_auth;
|
||||
|
||||
String filename;
|
||||
String cluster_name;
|
||||
String format_name;
|
||||
String compression_method;
|
||||
};
|
||||
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -31,7 +31,7 @@ public:
|
||||
unsigned num_streams) override;
|
||||
|
||||
protected:
|
||||
StorageSystemOne(const StorageID & table_id_);
|
||||
explicit StorageSystemOne(const StorageID & table_id_);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -18,7 +18,7 @@ namespace DB
|
||||
class TableFunctionRemote : public ITableFunction
|
||||
{
|
||||
public:
|
||||
TableFunctionRemote(const std::string & name_, bool secure_ = false);
|
||||
explicit TableFunctionRemote(const std::string & name_, bool secure_ = false);
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
|
@ -17,7 +17,6 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
@ -26,35 +25,53 @@ void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr con
|
||||
/// Parse args
|
||||
ASTs & args_func = ast_function->children;
|
||||
|
||||
const auto message = fmt::format(
|
||||
"The signature of table function {} could be the following:\n" \
|
||||
" - url, format, structure\n" \
|
||||
" - url, format, structure, compression_method\n" \
|
||||
" - url, access_key_id, secret_access_key, format, structure\n" \
|
||||
" - url, access_key_id, secret_access_key, format, structure, compression_method",
|
||||
getName());
|
||||
|
||||
if (args_func.size() != 1)
|
||||
throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
ASTs & args = args_func.at(0)->children;
|
||||
|
||||
if (args.size() < 3 || args.size() > 6)
|
||||
throw Exception("Table function '" + getName() + "' requires 3 to 6 arguments: url, [access_key_id, secret_access_key,] format, structure and [compression_method].",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
throw Exception(message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
for (auto & arg : args)
|
||||
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
|
||||
|
||||
/// Size -> argument indexes
|
||||
static auto size_to_args = std::map<size_t, std::map<String, size_t>>
|
||||
{
|
||||
{3, {{"format", 1}, {"structure", 2}}},
|
||||
{4, {{"format", 1}, {"structure", 2}, {"compression_method", 3}}},
|
||||
{5, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}}},
|
||||
{6, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}, {"compression_method", 5}}}
|
||||
};
|
||||
|
||||
/// This argument is always the first
|
||||
filename = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
if (args.size() < 5)
|
||||
{
|
||||
format = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
structure = args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
}
|
||||
else
|
||||
{
|
||||
access_key_id = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
secret_access_key = args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
format = args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
structure = args[4]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
}
|
||||
auto & args_to_idx = size_to_args[args.size()];
|
||||
|
||||
if (args.size() == 4 || args.size() == 6)
|
||||
compression_method = args.back()->as<ASTLiteral &>().value.safeGet<String>();
|
||||
if (args_to_idx.contains("format"))
|
||||
format = args[args_to_idx["format"]]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
if (args_to_idx.contains("structure"))
|
||||
structure = args[args_to_idx["structure"]]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
if (args_to_idx.contains("compression_method"))
|
||||
compression_method = args[args_to_idx["compression_method"]]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
if (args_to_idx.contains("access_key_id"))
|
||||
access_key_id = args[args_to_idx["access_key_id"]]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
if (args_to_idx.contains("secret_access_key"))
|
||||
secret_access_key = args[args_to_idx["secret_access_key"]]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
}
|
||||
|
||||
ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context) const
|
||||
|
144
src/TableFunctions/TableFunctionS3Cluster.cpp
Normal file
144
src/TableFunctions/TableFunctionS3Cluster.cpp
Normal file
@ -0,0 +1,144 @@
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <Storages/StorageS3Cluster.h>
|
||||
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataStreams/RemoteBlockInputStream.h>
|
||||
#include <IO/S3Common.h>
|
||||
#include <Storages/StorageS3.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ClientInfo.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/TableFunctionS3.h>
|
||||
#include <TableFunctions/TableFunctionS3Cluster.h>
|
||||
#include <TableFunctions/parseColumnsListForTableFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Processors/Sources/SourceFromInputStream.h>
|
||||
|
||||
#include "registerTableFunctions.h"
|
||||
|
||||
#include <memory>
|
||||
#include <thread>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
|
||||
void TableFunctionS3Cluster::parseArguments(const ASTPtr & ast_function, ContextPtr context)
|
||||
{
|
||||
/// Parse args
|
||||
ASTs & args_func = ast_function->children;
|
||||
|
||||
if (args_func.size() != 1)
|
||||
throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
ASTs & args = args_func.at(0)->children;
|
||||
|
||||
const auto message = fmt::format(
|
||||
"The signature of table function {} could be the following:\n" \
|
||||
" - cluster, url, format, structure\n" \
|
||||
" - cluster, url, format, structure, compression_method\n" \
|
||||
" - cluster, url, access_key_id, secret_access_key, format, structure\n" \
|
||||
" - cluster, url, access_key_id, secret_access_key, format, structure, compression_method",
|
||||
getName());
|
||||
|
||||
if (args.size() < 4 || args.size() > 7)
|
||||
throw Exception(message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
for (auto & arg : args)
|
||||
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
|
||||
|
||||
/// This arguments are always the first
|
||||
cluster_name = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
filename = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
/// Size -> argument indexes
|
||||
static auto size_to_args = std::map<size_t, std::map<String, size_t>>
|
||||
{
|
||||
{4, {{"format", 2}, {"structure", 3}}},
|
||||
{5, {{"format", 2}, {"structure", 3}, {"compression_method", 4}}},
|
||||
{6, {{"access_key_id", 2}, {"secret_access_key", 3}, {"format", 4}, {"structure", 5}}},
|
||||
{7, {{"access_key_id", 2}, {"secret_access_key", 3}, {"format", 4}, {"structure", 5}, {"compression_method", 6}}}
|
||||
};
|
||||
|
||||
auto & args_to_idx = size_to_args[args.size()];
|
||||
|
||||
if (args_to_idx.contains("format"))
|
||||
format = args[args_to_idx["format"]]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
if (args_to_idx.contains("structure"))
|
||||
structure = args[args_to_idx["structure"]]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
if (args_to_idx.contains("compression_method"))
|
||||
compression_method = args[args_to_idx["compression_method"]]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
if (args_to_idx.contains("access_key_id"))
|
||||
access_key_id = args[args_to_idx["access_key_id"]]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
if (args_to_idx.contains("secret_access_key"))
|
||||
secret_access_key = args[args_to_idx["secret_access_key"]]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
}
|
||||
|
||||
|
||||
ColumnsDescription TableFunctionS3Cluster::getActualTableStructure(ContextPtr context) const
|
||||
{
|
||||
return parseColumnsListFromString(structure, context);
|
||||
}
|
||||
|
||||
StoragePtr TableFunctionS3Cluster::executeImpl(
|
||||
const ASTPtr & /*function*/, ContextPtr context,
|
||||
const std::string & table_name, ColumnsDescription /*cached_columns*/) const
|
||||
{
|
||||
StoragePtr storage;
|
||||
if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY)
|
||||
{
|
||||
/// On worker node this filename won't contains globs
|
||||
Poco::URI uri (filename);
|
||||
S3::URI s3_uri (uri);
|
||||
/// Actually this parameters are not used
|
||||
UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size;
|
||||
UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size;
|
||||
UInt64 max_connections = context->getSettingsRef().s3_max_connections;
|
||||
storage = StorageS3::create(
|
||||
s3_uri, access_key_id, secret_access_key, StorageID(getDatabaseName(), table_name),
|
||||
format, min_upload_part_size, max_single_part_upload_size, max_connections,
|
||||
getActualTableStructure(context), ConstraintsDescription{},
|
||||
context, compression_method, /*distributed_processing=*/true);
|
||||
}
|
||||
else
|
||||
{
|
||||
storage = StorageS3Cluster::create(
|
||||
filename, access_key_id, secret_access_key, StorageID(getDatabaseName(), table_name),
|
||||
cluster_name, format, context->getSettingsRef().s3_max_connections,
|
||||
getActualTableStructure(context), ConstraintsDescription{},
|
||||
context, compression_method);
|
||||
}
|
||||
|
||||
storage->startup();
|
||||
|
||||
return storage;
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionS3Cluster(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionS3Cluster>();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
#endif
|
56
src/TableFunctions/TableFunctionS3Cluster.h
Normal file
56
src/TableFunctions/TableFunctionS3Cluster.h
Normal file
@ -0,0 +1,56 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
/**
|
||||
* s3Cluster(cluster_name, source, [access_key_id, secret_access_key,] format, structure)
|
||||
* A table function, which allows to process many files from S3 on a specific cluster
|
||||
* On initiator it creates a connection to _all_ nodes in cluster, discloses asterics
|
||||
* in S3 file path and dispatch each file dynamically.
|
||||
* On worker node it asks initiator about next task to process, processes it.
|
||||
* This is repeated until the tasks are finished.
|
||||
*/
|
||||
class TableFunctionS3Cluster : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "s3Cluster";
|
||||
std::string getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
bool hasStaticStructure() const override { return true; }
|
||||
|
||||
protected:
|
||||
StoragePtr executeImpl(
|
||||
const ASTPtr & ast_function,
|
||||
ContextPtr context,
|
||||
const std::string & table_name,
|
||||
ColumnsDescription cached_columns) const override;
|
||||
|
||||
const char * getStorageTypeName() const override { return "S3Cluster"; }
|
||||
|
||||
ColumnsDescription getActualTableStructure(ContextPtr) const override;
|
||||
void parseArguments(const ASTPtr &, ContextPtr) override;
|
||||
|
||||
String cluster_name;
|
||||
String filename;
|
||||
String format;
|
||||
String structure;
|
||||
String access_key_id;
|
||||
String secret_access_key;
|
||||
String compression_method = "auto";
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -21,6 +21,7 @@ void registerTableFunctions()
|
||||
|
||||
#if USE_AWS_S3
|
||||
registerTableFunctionS3(factory);
|
||||
registerTableFunctionS3Cluster(factory);
|
||||
registerTableFunctionCOS(factory);
|
||||
#endif
|
||||
|
||||
|
@ -21,6 +21,7 @@ void registerTableFunctionGenerate(TableFunctionFactory & factory);
|
||||
|
||||
#if USE_AWS_S3
|
||||
void registerTableFunctionS3(TableFunctionFactory & factory);
|
||||
void registerTableFunctionS3Cluster(TableFunctionFactory & factory);
|
||||
void registerTableFunctionCOS(TableFunctionFactory & factory);
|
||||
#endif
|
||||
|
||||
|
@ -0,0 +1 @@
|
||||
#!/usr/bin/env python3
|
@ -0,0 +1,42 @@
|
||||
<yandex>
|
||||
<keeper_server>
|
||||
<tcp_port>9181</tcp_port>
|
||||
<server_id>1</server_id>
|
||||
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
|
||||
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
|
||||
|
||||
<coordination_settings>
|
||||
<operation_timeout_ms>5000</operation_timeout_ms>
|
||||
<session_timeout_ms>10000</session_timeout_ms>
|
||||
<snapshot_distance>75</snapshot_distance>
|
||||
<raft_logs_level>trace</raft_logs_level>
|
||||
</coordination_settings>
|
||||
|
||||
<raft_configuration>
|
||||
<secure>true</secure>
|
||||
<server>
|
||||
<id>1</id>
|
||||
<hostname>node1</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>true</can_become_leader>
|
||||
<priority>3</priority>
|
||||
</server>
|
||||
<server>
|
||||
<id>2</id>
|
||||
<hostname>node2</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>true</can_become_leader>
|
||||
<start_as_follower>true</start_as_follower>
|
||||
<priority>2</priority>
|
||||
</server>
|
||||
<server>
|
||||
<id>3</id>
|
||||
<hostname>node3</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>true</can_become_leader>
|
||||
<start_as_follower>true</start_as_follower>
|
||||
<priority>1</priority>
|
||||
</server>
|
||||
</raft_configuration>
|
||||
</keeper_server>
|
||||
</yandex>
|
@ -0,0 +1,42 @@
|
||||
<yandex>
|
||||
<keeper_server>
|
||||
<tcp_port>9181</tcp_port>
|
||||
<server_id>2</server_id>
|
||||
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
|
||||
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
|
||||
|
||||
<coordination_settings>
|
||||
<operation_timeout_ms>5000</operation_timeout_ms>
|
||||
<session_timeout_ms>10000</session_timeout_ms>
|
||||
<snapshot_distance>75</snapshot_distance>
|
||||
<raft_logs_level>trace</raft_logs_level>
|
||||
</coordination_settings>
|
||||
|
||||
<raft_configuration>
|
||||
<secure>true</secure>
|
||||
<server>
|
||||
<id>1</id>
|
||||
<hostname>node1</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>true</can_become_leader>
|
||||
<priority>3</priority>
|
||||
</server>
|
||||
<server>
|
||||
<id>2</id>
|
||||
<hostname>node2</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>true</can_become_leader>
|
||||
<start_as_follower>true</start_as_follower>
|
||||
<priority>2</priority>
|
||||
</server>
|
||||
<server>
|
||||
<id>3</id>
|
||||
<hostname>node3</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>true</can_become_leader>
|
||||
<start_as_follower>true</start_as_follower>
|
||||
<priority>1</priority>
|
||||
</server>
|
||||
</raft_configuration>
|
||||
</keeper_server>
|
||||
</yandex>
|
@ -0,0 +1,42 @@
|
||||
<yandex>
|
||||
<keeper_server>
|
||||
<tcp_port>9181</tcp_port>
|
||||
<server_id>3</server_id>
|
||||
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
|
||||
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
|
||||
|
||||
<coordination_settings>
|
||||
<operation_timeout_ms>5000</operation_timeout_ms>
|
||||
<session_timeout_ms>10000</session_timeout_ms>
|
||||
<snapshot_distance>75</snapshot_distance>
|
||||
<raft_logs_level>trace</raft_logs_level>
|
||||
</coordination_settings>
|
||||
|
||||
<raft_configuration>
|
||||
<secure>true</secure>
|
||||
<server>
|
||||
<id>1</id>
|
||||
<hostname>node1</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>true</can_become_leader>
|
||||
<priority>3</priority>
|
||||
</server>
|
||||
<server>
|
||||
<id>2</id>
|
||||
<hostname>node2</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>true</can_become_leader>
|
||||
<start_as_follower>true</start_as_follower>
|
||||
<priority>2</priority>
|
||||
</server>
|
||||
<server>
|
||||
<id>3</id>
|
||||
<hostname>node3</hostname>
|
||||
<port>44444</port>
|
||||
<can_become_leader>true</can_become_leader>
|
||||
<start_as_follower>true</start_as_follower>
|
||||
<priority>1</priority>
|
||||
</server>
|
||||
</raft_configuration>
|
||||
</keeper_server>
|
||||
</yandex>
|
@ -0,0 +1,21 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIDazCCAlOgAwIBAgIUUiyhAav08YhTLfUIXLN/0Ln09n4wDQYJKoZIhvcNAQEL
|
||||
BQAwRTELMAkGA1UEBhMCQVUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoM
|
||||
GEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDAeFw0yMTA0MTIxMTQ1MjBaFw0yMTA1
|
||||
MTIxMTQ1MjBaMEUxCzAJBgNVBAYTAkFVMRMwEQYDVQQIDApTb21lLVN0YXRlMSEw
|
||||
HwYDVQQKDBhJbnRlcm5ldCBXaWRnaXRzIFB0eSBMdGQwggEiMA0GCSqGSIb3DQEB
|
||||
AQUAA4IBDwAwggEKAoIBAQDK0Ww4voPlkePBPS2MsEi7e1ePS+CDxTdDuOwWWEA7
|
||||
JiOyqIGqdyL6AE2EqjL3sSdVFVxytpGQWDuM6JHXdb01AnMngBuql9Jkiln7i267
|
||||
v54HtMWdm8o3rik/b/mB+kkn/sP715tI49Ybh/RobtvtK16ZgHr1ombkq6rXiom2
|
||||
8GmSmpYFwZtZsXtm2JwbZVayupQpWwdu3KrTXKBtVyKVvvWdgkf47DWYtWDS3vqE
|
||||
cShM1H97G4DvI+4RX1WtQevQ0yCx1aFTg4xMHFkpUxlP8iW6mQaQPqy9rnI57e3L
|
||||
RHc2I/B56xa43R3GmQ2S7bE4hvm1SrZDtVgrZLf4nvwNAgMBAAGjUzBRMB0GA1Ud
|
||||
DgQWBBQ4+o0x1FzK7nRbcnm2pNLwaywCdzAfBgNVHSMEGDAWgBQ4+o0x1FzK7nRb
|
||||
cnm2pNLwaywCdzAPBgNVHRMBAf8EBTADAQH/MA0GCSqGSIb3DQEBCwUAA4IBAQDE
|
||||
YmM8MH6RKcaqMqCBefWLj0LTcZ/Wm4G/eCFC51PkAIsf7thnzViemBHRXUSF8wzc
|
||||
1MBPD6II6OB1F0i7ntGjtlhnL2WcPYbo2Np59p7fo9SMbYwF49OZ40twsuKeeoAp
|
||||
pfow+y/EBZqa99MY2q6FU6FDA3Rpv0Sdk+/5PHdsSP6cgeMszFBUS0tCQEvEl83n
|
||||
FJUb0vjEX4x3J64XO/0DKXyCxFyF77OwHG2ZV5BeCpIhGXu+d/e221LJkGI2orKR
|
||||
kgsaUwrkS8HQt3Hd0gYpLI1Opx/JlRpB0VLYLzRGj7kDpbAcTj3SMEUp/FAZmlXR
|
||||
Iiebt73eE3rOWVFgyY9f
|
||||
-----END CERTIFICATE-----
|
@ -0,0 +1,19 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIDETCCAfkCFHL+gKBQnU0P73/nrFrGaVPauTPmMA0GCSqGSIb3DQEBCwUAMEUx
|
||||
CzAJBgNVBAYTAkFVMRMwEQYDVQQIDApTb21lLVN0YXRlMSEwHwYDVQQKDBhJbnRl
|
||||
cm5ldCBXaWRnaXRzIFB0eSBMdGQwHhcNMjEwNDEyMTE0NzI5WhcNMjEwNTEyMTE0
|
||||
NzI5WjBFMQswCQYDVQQGEwJBVTETMBEGA1UECAwKU29tZS1TdGF0ZTEhMB8GA1UE
|
||||
CgwYSW50ZXJuZXQgV2lkZ2l0cyBQdHkgTHRkMIIBIjANBgkqhkiG9w0BAQEFAAOC
|
||||
AQ8AMIIBCgKCAQEA1iPeYn1Vy4QnQi6uNVqQnFLr0u3qdrMjGEBNAOuGmtIdhIn8
|
||||
rMCzaehNr3y2YTMRbZAqmv28P/wOXpzR1uQaFlQzTOjmsn/HOZ9JX2hv5sBUv7SU
|
||||
UiPJS7UtptKDPbLv3N/v1dOXbY+vVyzo8U1Q9OS1J5yhYW6KtxP++hfSrOsFu669
|
||||
d1pqWFWaNBsmf0zF+ETvi6lywhyTFA1/PazcStP5GntcDL7eDvGq+DDsRC40oRpy
|
||||
S4xRQRSteCTtGGmWpx+Jmt+90wFnLgruUbWT0veCoLxLvz0tJUk3ueUVnMkrxBQG
|
||||
Fz+IWm+SQppNU5LlAcBcu9wJfo3h34BXp0NFNQIDAQABMA0GCSqGSIb3DQEBCwUA
|
||||
A4IBAQCUnvQsv+GsPwGnIWqH9iiFVhgDx5QbSTW94Fyqk8dcIJBzWAiCshmLBWPJ
|
||||
pfy4y2nxJbzovFsd9DA49pxqqILeLjue99yma2DVKeo+XDLDN3OX5faIMTBd7AnL
|
||||
0MKqW7gUSLRUZrNOvFciAY8xRezgBQQBo4mcmmMbAbk5wKndGY6ZZOcY+JwXlqGB
|
||||
5hyi6ishO8ciiZi3GMFNWWk9ViSfo27IqjKdSkQq1pr3FULvepd6SkdX+NvfZTAH
|
||||
rG+CSoFGiJcOBbhDkvpY32cAJEnJOA1vHpFxfnGP8/1haeVZHqSwH1cySD78HVtF
|
||||
fBs000wGHzBYWNI2KkwjNtYf06P4
|
||||
-----END CERTIFICATE-----
|
@ -0,0 +1,27 @@
|
||||
-----BEGIN RSA PRIVATE KEY-----
|
||||
MIIEowIBAAKCAQEA1iPeYn1Vy4QnQi6uNVqQnFLr0u3qdrMjGEBNAOuGmtIdhIn8
|
||||
rMCzaehNr3y2YTMRbZAqmv28P/wOXpzR1uQaFlQzTOjmsn/HOZ9JX2hv5sBUv7SU
|
||||
UiPJS7UtptKDPbLv3N/v1dOXbY+vVyzo8U1Q9OS1J5yhYW6KtxP++hfSrOsFu669
|
||||
d1pqWFWaNBsmf0zF+ETvi6lywhyTFA1/PazcStP5GntcDL7eDvGq+DDsRC40oRpy
|
||||
S4xRQRSteCTtGGmWpx+Jmt+90wFnLgruUbWT0veCoLxLvz0tJUk3ueUVnMkrxBQG
|
||||
Fz+IWm+SQppNU5LlAcBcu9wJfo3h34BXp0NFNQIDAQABAoIBAHYDso2o8V2F6XTp
|
||||
8QxqawQcFudaQztDonW9CjMVmks8vRPMUDqMwNP/OMEcBA8xa8tsBm8Ao3zH1suB
|
||||
tYuujkn8AYHDYVDCZvN0u6UfE3yiRpKYXJ2gJ1HX+d7UaYvZT6P0rmKzh+LTqxhq
|
||||
Ib7Kk3FDkirQgYgGueAH3x/JfUvaAGvFrq+HvvlhHOs7M7iFU4nJA8jNfBolpTnG
|
||||
v5MMI+f8/GHGreVICJUoclE+4V/4LDHUlrc3l1kQk0keeD6ECw/pl48TNL6ncXKu
|
||||
baez1rfKbMPjhLUy2q5UZa93oW+olchEOXs1nUNKUhIOOr0f0YweYhUHNTineVM9
|
||||
yTecMIkCgYEA7CFQMyeLVeBA6C9AHBe8Zf/k64cMPyr0lUz6548ulil580PNPbvW
|
||||
kd2vIKfUMgCO5lMA47ArL4bXZ7cjTvJmPYE1Yv8z+F0Tk03fnTrudHOSBEiGXAu3
|
||||
MPTxCDU7Se5Dwj0Fq81aFRtCHl8Rrss+WiBD8eRoxb/vwXKFc6VUAWMCgYEA6CjZ
|
||||
XrZz11lySBhjkyVXcdLj89hDZ+bPxA7b3VB7TfCxsn5xVck7U3TFkg5Z9XwEQ7Ob
|
||||
XFAPuwT9GKm7QPp6L8T2RltoJ3ys40UH1RtcNLz2aIo/xSP7lopPdAfWHef5r4y9
|
||||
kRw+Gh4NP/l5wefXsRz/D0jY3+t+QnwnhuCKbocCgYEAiR6bPOlkvzyXVH1DxEyA
|
||||
Sdb8b00f7nqaRyzJsrfxvJ9fQsWHpKa0ZkYOUW9ECLlMQjHHHXEK0vGBmqe9qDWY
|
||||
63RhtRgvbLVYDb018k7rc9I846Hd7AudmJ9UbIjE4hyrWlsnNOntur32ej6IvTEn
|
||||
Bx0fd5NEyDi6GGLRXiOOkbMCgYAressLE/yqDlR68CZl/o5cAPU0TAKDyRSMUYQX
|
||||
9OTC+hstpMSxHlkADlSaQBnVAf8CdvbX2R65FfwYzGEHkGGl5KuDDcd57b2rathG
|
||||
rzMbpXA4r/u1fkG2Nf0fbABL5ZA7so4mSTXQSmSM4LpO+I7K2vVh9XC4rzAcX4g/
|
||||
mHoUrQKBgBf3rxp5h9P3HWoZYjzBDo2FqXUjKLLjE9ed5e/VqecqfHIkmueuNHlN
|
||||
xifHr7lpsYu6IXkTnlK14pvLoPuwP59dCIOUYwAFz8RlH4MSUGNhYeGA8cqRrhmJ
|
||||
tYk3OKExuN/+O12kUPVTy6BMH1hBdRJP+7y7lapWsRhZt18y+8Za
|
||||
-----END RSA PRIVATE KEY-----
|
@ -0,0 +1,15 @@
|
||||
|
||||
<yandex>
|
||||
<openSSL>
|
||||
<server>
|
||||
<certificateFile>/etc/clickhouse-server/config.d/server.crt</certificateFile>
|
||||
<privateKeyFile>/etc/clickhouse-server/config.d/server.key</privateKeyFile>
|
||||
<caConfig>/etc/clickhouse-server/config.d/rootCA.pem</caConfig>
|
||||
<loadDefaultCAFile>true</loadDefaultCAFile>
|
||||
<verificationMode>none</verificationMode>
|
||||
<cacheSessions>true</cacheSessions>
|
||||
<disableProtocols>sslv2,sslv3</disableProtocols>
|
||||
<preferServerCiphers>true</preferServerCiphers>
|
||||
</server>
|
||||
</openSSL>
|
||||
</yandex>
|
58
tests/integration/test_keeper_internal_secure/test.py
Normal file
58
tests/integration/test_keeper_internal_secure/test.py
Normal file
@ -0,0 +1,58 @@
|
||||
#!/usr/bin/env python3
|
||||
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
import random
|
||||
import string
|
||||
import os
|
||||
import time
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node1 = cluster.add_instance('node1', main_configs=['configs/enable_secure_keeper1.xml', 'configs/ssl_conf.xml', 'configs/server.crt', 'configs/server.key', 'configs/rootCA.pem'])
|
||||
node2 = cluster.add_instance('node2', main_configs=['configs/enable_secure_keeper2.xml', 'configs/ssl_conf.xml', 'configs/server.crt', 'configs/server.key', 'configs/rootCA.pem'])
|
||||
node3 = cluster.add_instance('node3', main_configs=['configs/enable_secure_keeper3.xml', 'configs/ssl_conf.xml', 'configs/server.crt', 'configs/server.key', 'configs/rootCA.pem'])
|
||||
|
||||
from kazoo.client import KazooClient, KazooState
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
def get_fake_zk(nodename, timeout=30.0):
|
||||
_fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout)
|
||||
def reset_listener(state):
|
||||
nonlocal _fake_zk_instance
|
||||
print("Fake zk callback called for state", state)
|
||||
if state != KazooState.CONNECTED:
|
||||
_fake_zk_instance._reset()
|
||||
|
||||
_fake_zk_instance.add_listener(reset_listener)
|
||||
_fake_zk_instance.start()
|
||||
return _fake_zk_instance
|
||||
|
||||
def test_secure_raft_works(started_cluster):
|
||||
try:
|
||||
node1_zk = get_fake_zk("node1")
|
||||
node2_zk = get_fake_zk("node2")
|
||||
node3_zk = get_fake_zk("node3")
|
||||
|
||||
node1_zk.create("/test_node", b"somedata1")
|
||||
node2_zk.sync("/test_node")
|
||||
node3_zk.sync("/test_node")
|
||||
|
||||
assert node1_zk.exists("/test_node") is not None
|
||||
assert node2_zk.exists("/test_node") is not None
|
||||
assert node3_zk.exists("/test_node") is not None
|
||||
finally:
|
||||
try:
|
||||
for zk_conn in [node1_zk, node2_zk, node3_zk]:
|
||||
zk_conn.stop()
|
||||
zk_conn.close()
|
||||
except:
|
||||
pass
|
1
tests/integration/test_keeper_secure_client/__init__.py
Normal file
1
tests/integration/test_keeper_secure_client/__init__.py
Normal file
@ -0,0 +1 @@
|
||||
#!/usr/bin/env python3
|
@ -0,0 +1,8 @@
|
||||
-----BEGIN DH PARAMETERS-----
|
||||
MIIBCAKCAQEAua92DDli13gJ+//ZXyGaggjIuidqB0crXfhUlsrBk9BV1hH3i7fR
|
||||
XGP9rUdk2ubnB3k2ejBStL5oBrkHm9SzUFSQHqfDjLZjKoUpOEmuDc4cHvX1XTR5
|
||||
Pr1vf5cd0yEncJWG5W4zyUB8k++SUdL2qaeslSs+f491HBLDYn/h8zCgRbBvxhxb
|
||||
9qeho1xcbnWeqkN6Kc9bgGozA16P9NLuuLttNnOblkH+lMBf42BSne/TWt3AlGZf
|
||||
slKmmZcySUhF8aKfJnLKbkBCFqOtFRh8zBA9a7g+BT/lSANATCDPaAk1YVih2EKb
|
||||
dpc3briTDbRsiqg2JKMI7+VdULY9bh3EawIBAg==
|
||||
-----END DH PARAMETERS-----
|
@ -0,0 +1,24 @@
|
||||
<?xml version="1.0" encoding="utf-8"?>
|
||||
<yandex>
|
||||
<keeper_server>
|
||||
<tcp_port_secure>10181</tcp_port_secure>
|
||||
<server_id>1</server_id>
|
||||
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
|
||||
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
|
||||
|
||||
<coordination_settings>
|
||||
<operation_timeout_ms>10000</operation_timeout_ms>
|
||||
<session_timeout_ms>30000</session_timeout_ms>
|
||||
<raft_logs_level>trace</raft_logs_level>
|
||||
<force_sync>false</force_sync>
|
||||
</coordination_settings>
|
||||
|
||||
<raft_configuration>
|
||||
<server>
|
||||
<id>1</id>
|
||||
<hostname>localhost</hostname>
|
||||
<port>44444</port>
|
||||
</server>
|
||||
</raft_configuration>
|
||||
</keeper_server>
|
||||
</yandex>
|
@ -0,0 +1,19 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIC/TCCAeWgAwIBAgIJANjx1QSR77HBMA0GCSqGSIb3DQEBCwUAMBQxEjAQBgNV
|
||||
BAMMCWxvY2FsaG9zdDAgFw0xODA3MzAxODE2MDhaGA8yMjkyMDUxNDE4MTYwOFow
|
||||
FDESMBAGA1UEAwwJbG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIB
|
||||
CgKCAQEAs9uSo6lJG8o8pw0fbVGVu0tPOljSWcVSXH9uiJBwlZLQnhN4SFSFohfI
|
||||
4K8U1tBDTnxPLUo/V1K9yzoLiRDGMkwVj6+4+hE2udS2ePTQv5oaMeJ9wrs+5c9T
|
||||
4pOtlq3pLAdm04ZMB1nbrEysceVudHRkQbGHzHp6VG29Fw7Ga6YpqyHQihRmEkTU
|
||||
7UCYNA+Vk7aDPdMS/khweyTpXYZimaK9f0ECU3/VOeG3fH6Sp2X6FN4tUj/aFXEj
|
||||
sRmU5G2TlYiSIUMF2JPdhSihfk1hJVALrHPTU38SOL+GyyBRWdNcrIwVwbpvsvPg
|
||||
pryMSNxnpr0AK0dFhjwnupIv5hJIOQIDAQABo1AwTjAdBgNVHQ4EFgQUjPLb3uYC
|
||||
kcamyZHK4/EV8jAP0wQwHwYDVR0jBBgwFoAUjPLb3uYCkcamyZHK4/EV8jAP0wQw
|
||||
DAYDVR0TBAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEAM/ocuDvfPus/KpMVD51j
|
||||
4IdlU8R0vmnYLQ+ygzOAo7+hUWP5j0yvq4ILWNmQX6HNvUggCgFv9bjwDFhb/5Vr
|
||||
85ieWfTd9+LTjrOzTw4avdGwpX9G+6jJJSSq15tw5ElOIFb/qNA9O4dBiu8vn03C
|
||||
L/zRSXrARhSqTW5w/tZkUcSTT+M5h28+Lgn9ysx4Ff5vi44LJ1NnrbJbEAIYsAAD
|
||||
+UA+4MBFKx1r6hHINULev8+lCfkpwIaeS8RL+op4fr6kQPxnULw8wT8gkuc8I4+L
|
||||
P9gg/xDHB44T3ADGZ5Ib6O0DJaNiToO6rnoaaxs0KkotbvDWvRoxEytSbXKoYjYp
|
||||
0g==
|
||||
-----END CERTIFICATE-----
|
@ -0,0 +1,28 @@
|
||||
-----BEGIN PRIVATE KEY-----
|
||||
MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQCz25KjqUkbyjyn
|
||||
DR9tUZW7S086WNJZxVJcf26IkHCVktCeE3hIVIWiF8jgrxTW0ENOfE8tSj9XUr3L
|
||||
OguJEMYyTBWPr7j6ETa51LZ49NC/mhox4n3Cuz7lz1Pik62WreksB2bThkwHWdus
|
||||
TKxx5W50dGRBsYfMenpUbb0XDsZrpimrIdCKFGYSRNTtQJg0D5WTtoM90xL+SHB7
|
||||
JOldhmKZor1/QQJTf9U54bd8fpKnZfoU3i1SP9oVcSOxGZTkbZOViJIhQwXYk92F
|
||||
KKF+TWElUAusc9NTfxI4v4bLIFFZ01ysjBXBum+y8+CmvIxI3GemvQArR0WGPCe6
|
||||
ki/mEkg5AgMBAAECggEATrbIBIxwDJOD2/BoUqWkDCY3dGevF8697vFuZKIiQ7PP
|
||||
TX9j4vPq0DfsmDjHvAPFkTHiTQXzlroFik3LAp+uvhCCVzImmHq0IrwvZ9xtB43f
|
||||
7Pkc5P6h1l3Ybo8HJ6zRIY3TuLtLxuPSuiOMTQSGRL0zq3SQ5DKuGwkz+kVjHXUN
|
||||
MR2TECFwMHKQ5VLrC+7PMpsJYyOMlDAWhRfUalxC55xOXTpaN8TxNnwQ8K2ISVY5
|
||||
212Jz/a4hn4LdwxSz3Tiu95PN072K87HLWx3EdT6vW4Ge5P/A3y+smIuNAlanMnu
|
||||
plHBRtpATLiTxZt/n6npyrfQVbYjSH7KWhB8hBHtaQKBgQDh9Cq1c/KtqDtE0Ccr
|
||||
/r9tZNTUwBE6VP+3OJeKdEdtsfuxjOCkS1oAjgBJiSDOiWPh1DdoDeVZjPKq6pIu
|
||||
Mq12OE3Doa8znfCXGbkSzEKOb2unKZMJxzrz99kXt40W5DtrqKPNb24CNqTiY8Aa
|
||||
CjtcX+3weat82VRXvph6U8ltMwKBgQDLxjiQQzNoY7qvg7CwJCjf9qq8jmLK766g
|
||||
1FHXopqS+dTxDLM8eJSRrpmxGWJvNeNc1uPhsKsKgotqAMdBUQTf7rSTbt4MyoH5
|
||||
bUcRLtr+0QTK9hDWMOOvleqNXha68vATkohWYfCueNsC60qD44o8RZAS6UNy3ENq
|
||||
cM1cxqe84wKBgQDKkHutWnooJtajlTxY27O/nZKT/HA1bDgniMuKaz4R4Gr1PIez
|
||||
on3YW3V0d0P7BP6PWRIm7bY79vkiMtLEKdiKUGWeyZdo3eHvhDb/3DCawtau8L2K
|
||||
GZsHVp2//mS1Lfz7Qh8/L/NedqCQ+L4iWiPnZ3THjjwn3CoZ05ucpvrAMwKBgB54
|
||||
nay039MUVq44Owub3KDg+dcIU62U+cAC/9oG7qZbxYPmKkc4oL7IJSNecGHA5SbU
|
||||
2268RFdl/gLz6tfRjbEOuOHzCjFPdvAdbysanpTMHLNc6FefJ+zxtgk9sJh0C4Jh
|
||||
vxFrw9nTKKzfEl12gQ1SOaEaUIO0fEBGbe8ZpauRAoGAMAlGV+2/K4ebvAJKOVTa
|
||||
dKAzQ+TD2SJmeR1HZmKDYddNqwtZlzg3v4ZhCk4eaUmGeC1Bdh8MDuB3QQvXz4Dr
|
||||
vOIP4UVaOr+uM+7TgAgVnP4/K6IeJGzUDhX93pmpWhODfdu/oojEKVcpCojmEmS1
|
||||
KCBtmIrQLqzMpnBpLNuSY+Q=
|
||||
-----END PRIVATE KEY-----
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user