Merge branch 'master' into ch_opt_array_element

This commit is contained in:
taiyang-li 2024-01-10 15:49:43 +08:00
commit 231de4ac49
133 changed files with 3724 additions and 1291 deletions

View File

@ -33,7 +33,7 @@ curl https://clickhouse.com/ | sh
## Upcoming Events
Keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler <at> clickhouse <dot> com.
Keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `<at>` clickhouse `<dot>` com.
## Recent Recordings
* **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments"

View File

@ -41,6 +41,10 @@ readarray -t DISKS_PATHS < <(clickhouse extract-from-config --config-file "$CLIC
readarray -t DISKS_METADATA_PATHS < <(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key='storage_configuration.disks.*.metadata_path' || true)
CLICKHOUSE_USER="${CLICKHOUSE_USER:-default}"
CLICKHOUSE_PASSWORD_FILE="${CLICKHOUSE_PASSWORD_FILE:-}"
if [[ -n "${CLICKHOUSE_PASSWORD_FILE}" && -f "${CLICKHOUSE_PASSWORD_FILE}" ]]; then
CLICKHOUSE_PASSWORD="$(cat "${CLICKHOUSE_PASSWORD_FILE}")"
fi
CLICKHOUSE_PASSWORD="${CLICKHOUSE_PASSWORD:-}"
CLICKHOUSE_DB="${CLICKHOUSE_DB:-}"
CLICKHOUSE_ACCESS_MANAGEMENT="${CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT:-0}"

View File

@ -44,6 +44,9 @@ if [[ -n "$USE_S3_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_S3_STORAGE_FOR_MERGE_TR
# It is not needed, we will explicitly create tables on s3.
# We do not have statefull tests with s3 storage run in public repository, but this is needed for another repository.
rm /etc/clickhouse-server/config.d/s3_storage_policy_for_merge_tree_by_default.xml
rm /etc/clickhouse-server/config.d/storage_metadata_with_full_object_key.xml
rm /etc/clickhouse-server/config.d/s3_storage_policy_with_template_object_key.xml
fi
function start()

View File

@ -193,6 +193,7 @@ stop
# Let's enable S3 storage by default
export USE_S3_STORAGE_FOR_MERGE_TREE=1
export $RANDOMIZE_OBJECT_KEY_TYPE=1
export ZOOKEEPER_FAULT_INJECTION=1
configure

View File

@ -11,7 +11,7 @@ This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ec
``` sql
CREATE TABLE s3_queue_engine_table (name String, value UInt32)
ENGINE = S3Queue(path [, NOSIGN | aws_access_key_id, aws_secret_access_key,] format, [compression])
ENGINE = S3Queue(path, [NOSIGN, | aws_access_key_id, aws_secret_access_key,] format, [compression])
[SETTINGS]
[mode = 'unordered',]
[after_processing = 'keep',]

View File

@ -1143,6 +1143,8 @@ Optional parameters:
- `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`.
- `read_resource` — Resource name to be used for [scheduling](/docs/en/operations/workload-scheduling.md) of read requests to this disk. Default value is empty string (IO scheduling is not enabled for this disk).
- `write_resource` — Resource name to be used for [scheduling](/docs/en/operations/workload-scheduling.md) of write requests to this disk. Default value is empty string (IO scheduling is not enabled for this disk).
- `key_template` — Define the format with which the object keys are generated. By default, Clickhouse takes `root path` from `endpoint` option and adds random generated suffix. That suffix is a dir with 3 random symbols and a file name with 29 random symbols. With that option you have a full control how to the object keys are generated. Some usage scenarios require having random symbols in the prefix or in the middle of object key. For example: `[a-z]{3}-prefix-random/constant-part/random-middle-[a-z]{3}/random-suffix-[a-z]{29}`. The value is parsed with [`re2`](https://github.com/google/re2/wiki/Syntax). Only some subset of the syntax is supported. Check if your preferred format is supported before using that option. Disk isn't initialized if clickhouse is unable to generate a key by the value of `key_template`. It requires enabled feature flag [storage_metadata_write_full_object_key](/docs/en/operations/settings/settings#storage_metadata_write_full_object_key). It forbids declaring the `root path` in `endpoint` option. It requires definition of the option `key_compatibility_prefix`.
- `key_compatibility_prefix` — That option is required when option `key_template` is in use. In order to be able to read the objects keys which were stored in the metadata files with the metadata version lower that `VERSION_FULL_OBJECT_KEY`, the previous `root path` from the `endpoint` option should be set here.
### Configuring the cache

View File

@ -4773,6 +4773,24 @@ Type: Int64
Default: 0
## output_format_compression_level
Default compression level if query output is compressed. The setting is applied when `SELECT` query has `INTO OUTFILE` or when writing to table functions `file`, `url`, `hdfs`, `s3`, or `azureBlobStorage`.
Possible values: from `1` to `22`
Default: `3`
## output_format_compression_zstd_window_log
Can be used when the output compression method is `zstd`. If greater than `0`, this setting explicitly sets compression window size (power of `2`) and enables a long-range mode for zstd compression. This can help to achieve a better compression ratio.
Possible values: non-negative numbers. Note that if the value is too small or too big, `zstdlib` will throw an exception. Typical values are from `20` (window size = `1MB`) to `30` (window size = `1GB`).
Default: `0`
## rewrite_count_distinct_if_with_count_distinct_implementation
Allows you to rewrite `countDistcintIf` with [count_distinct_implementation](#count_distinct_implementation) setting.

View File

@ -11,6 +11,8 @@ Keys:
- `--query` — Format queries of any length and complexity.
- `--hilite` — Add syntax highlight with ANSI terminal escape sequences.
- `--oneline` — Format in single line.
- `--max_line_length` — Format in single line queries with length less than specified.
- `--comments` — Keep comments in the output.
- `--quiet` or `-q` — Just check syntax, no output on success.
- `--multiquery` or `-n` — Allow multiple queries in the same file.
- `--obfuscate` — Obfuscate instead of formatting.

View File

@ -24,7 +24,7 @@ A client application to interact with clickhouse-keeper by its native protocol.
## Example {#clickhouse-keeper-client-example}
```bash
./clickhouse-keeper-client -h localhost:9181 --connection-timeout 30 --session-timeout 30 --operation-timeout 30
./clickhouse-keeper-client -h localhost -p 9181 --connection-timeout 30 --session-timeout 30 --operation-timeout 30
Connected to ZooKeeper at [::1]:9181 with session_id 137
/ :) ls
keeper foo bar

View File

@ -1605,6 +1605,78 @@ Result:
Alias: levenshteinDistance
## damerauLevenshteinDistance
Calculates the [Damerau-Levenshtein distance](https://en.wikipedia.org/wiki/Damerau%E2%80%93Levenshtein_distance) between two byte strings.
**Syntax**
```sql
damerauLevenshteinDistance(string1, string2)
```
**Examples**
``` sql
SELECT damerauLevenshteinDistance('clickhouse', 'mouse');
```
Result:
``` text
┌─damerauLevenshteinDistance('clickhouse', 'mouse')─┐
│ 6 │
└───────────────────────────────────────────────────┘
```
## jaroSimilarity
Calculates the [Jaro similarity](https://en.wikipedia.org/wiki/Jaro%E2%80%93Winkler_distance#Jaro_similarity) between two byte strings.
**Syntax**
```sql
jaroSimilarity(string1, string2)
```
**Examples**
``` sql
SELECT jaroSimilarity('clickhouse', 'click');
```
Result:
``` text
┌─jaroSimilarity('clickhouse', 'click')─┐
│ 0.8333333333333333 │
└───────────────────────────────────────┘
```
## jaroWinklerSimilarity
Calculates the [Jaro-Winkler similarity](https://en.wikipedia.org/wiki/Jaro%E2%80%93Winkler_distance#Jaro%E2%80%93Winkler_similarity) between two byte strings.
**Syntax**
```sql
jaroWinklerSimilarity(string1, string2)
```
**Examples**
``` sql
SELECT jaroWinklerSimilarity('clickhouse', 'click');
```
Result:
``` text
┌─jaroWinklerSimilarity('clickhouse', 'click')─┐
│ 0.8999999999999999 │
└──────────────────────────────────────────────┘
```
## initcap
Convert the first letter of each word to upper case and the rest to lower case. Words are sequences of alphanumeric characters separated by non-alphanumeric characters.

View File

@ -57,3 +57,56 @@ Result:
│ 6 │
└─────────┘
```
## seriesDecomposeSTL
Decomposes a time series using STL [(Seasonal-Trend Decomposition Procedure Based on Loess)](https://www.wessa.net/download/stl.pdf) into a season, a trend and a residual component.
**Syntax**
``` sql
seriesDecomposeSTL(series, period);
```
**Arguments**
- `series` - An array of numeric values
- `period` - A positive integer
The number of data points in `series` should be at least twice the value of `period`.
**Returned value**
- An array of three arrays where the first array include seasonal components, the second array - trend,
and the third array - residue component.
Type: [Array](../../sql-reference/data-types/array.md).
**Examples**
Query:
``` sql
SELECT seriesDecomposeSTL([10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34], 3) AS print_0;
```
Result:
``` text
┌───────────print_0──────────────────────────────────────────────────────────────────────────────────────────────────────┐
│ [[
-13.529999, -3.1799996, 16.71, -13.53, -3.1799996, 16.71, -13.53, -3.1799996,
16.71, -13.530001, -3.18, 16.710001, -13.530001, -3.1800003, 16.710001, -13.530001,
-3.1800003, 16.710001, -13.530001, -3.1799994, 16.71, -13.529999, -3.1799994, 16.709997
],
[
23.63, 23.63, 23.630003, 23.630001, 23.630001, 23.630001, 23.630001, 23.630001,
23.630001, 23.630001, 23.630001, 23.63, 23.630001, 23.630001, 23.63, 23.630001,
23.630001, 23.63, 23.630001, 23.630001, 23.630001, 23.630001, 23.630001, 23.630003
],
[
0, 0.0000019073486, -0.0000019073486, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0.0000019073486, 0,
0
]] │
└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
```

View File

@ -3,6 +3,7 @@
#include <string_view>
#include <boost/program_options.hpp>
#include <IO/copyData.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFileDescriptor.h>
@ -14,6 +15,7 @@
#include <Parsers/obfuscateQueries.h>
#include <Parsers/parseQuery.h>
#include <Common/ErrorCodes.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/TerminalSize.h>
#include <Interpreters/Context.h>
@ -30,22 +32,49 @@
#include <DataTypes/DataTypeFactory.h>
#include <Formats/FormatFactory.h>
#include <Formats/registerFormats.h>
#include <Processors/Transforms/getSourceFromASTInsertQuery.h>
namespace DB::ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
namespace
{
void skipSpacesAndComments(const char*& pos, const char* end, bool print_comments)
{
do
{
/// skip spaces to avoid throw exception after last query
while (pos != end && std::isspace(*pos))
++pos;
const char * comment_begin = pos;
/// for skip comment after the last query and to not throw exception
if (end - pos > 2 && *pos == '-' && *(pos + 1) == '-')
{
pos += 2;
/// skip until the end of the line
while (pos != end && *pos != '\n')
++pos;
if (print_comments)
std::cout << std::string_view(comment_begin, pos - comment_begin) << "\n";
}
/// need to parse next sql
else
break;
} while (pos != end);
}
}
#pragma GCC diagnostic ignored "-Wunused-function"
#pragma GCC diagnostic ignored "-Wmissing-declarations"
extern const char * auto_time_zones[];
namespace DB
{
namespace ErrorCodes
{
extern const int INVALID_FORMAT_INSERT_QUERY_WITH_DATA;
}
}
int mainEntryClickHouseFormat(int argc, char ** argv)
{
using namespace DB;
@ -56,8 +85,10 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
desc.add_options()
("query", po::value<std::string>(), "query to format")
("help,h", "produce help message")
("comments", "keep comments in the output")
("hilite", "add syntax highlight with ANSI terminal escape sequences")
("oneline", "format in single line")
("max_line_length", po::value<size_t>()->default_value(0), "format in single line queries with length less than specified")
("quiet,q", "just check syntax, no output on success")
("multiquery,n", "allow multiple queries in the same file")
("obfuscate", "obfuscate instead of formatting")
@ -89,6 +120,8 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
bool oneline = options.count("oneline");
bool quiet = options.count("quiet");
bool multiple = options.count("multiquery");
bool print_comments = options.count("comments");
size_t max_line_length = options["max_line_length"].as<size_t>();
bool obfuscate = options.count("obfuscate");
bool backslash = options.count("backslash");
bool allow_settings_after_format_in_insert = options.count("allow_settings_after_format_in_insert");
@ -105,6 +138,19 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
return 2;
}
if (oneline && max_line_length)
{
std::cerr << "Options 'oneline' and 'max_line_length' are mutually exclusive." << std::endl;
return 2;
}
if (max_line_length > 255)
{
std::cerr << "Option 'max_line_length' must be less than 256." << std::endl;
return 2;
}
String query;
if (options.count("query"))
@ -125,7 +171,6 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
if (options.count("seed"))
{
std::string seed;
hash_func.update(options["seed"].as<std::string>());
}
@ -181,30 +226,75 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
{
const char * pos = query.data();
const char * end = pos + query.size();
skipSpacesAndComments(pos, end, print_comments);
ParserQuery parser(end, allow_settings_after_format_in_insert);
do
while (pos != end)
{
size_t approx_query_length = multiple ? find_first_symbols<';'>(pos, end) - pos : end - pos;
ASTPtr res = parseQueryAndMovePosition(
parser, pos, end, "query", multiple, cmd_settings.max_query_size, cmd_settings.max_parser_depth);
/// For insert query with data(INSERT INTO ... VALUES ...), that will lead to the formatting failure,
/// we should throw an exception early, and make exception message more readable.
if (const auto * insert_query = res->as<ASTInsertQuery>(); insert_query && insert_query->data)
std::unique_ptr<ReadBuffer> insert_query_payload = nullptr;
/// If the query is INSERT ... VALUES, then we will try to parse the data.
if (auto * insert_query = res->as<ASTInsertQuery>(); insert_query && insert_query->data)
{
throw Exception(DB::ErrorCodes::INVALID_FORMAT_INSERT_QUERY_WITH_DATA,
"Can't format ASTInsertQuery with data, since data will be lost");
if ("Values" != insert_query->format)
throw Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Can't format INSERT query with data format '{}'", insert_query->format);
/// Reset format to default to have `INSERT INTO table VALUES` instead of `INSERT INTO table VALUES FORMAT Values`
insert_query->format = {};
/// We assume that data ends with a newline character (same as client does)
const char * this_query_end = find_first_symbols<'\n'>(insert_query->data, end);
insert_query->end = this_query_end;
pos = this_query_end;
insert_query_payload = getReadBufferFromASTInsertQuery(res);
}
if (!quiet)
{
if (!backslash)
{
WriteBufferFromOStream res_buf(std::cout, 4096);
formatAST(*res, res_buf, hilite, oneline);
res_buf.finalize();
if (multiple)
std::cout << "\n;\n";
WriteBufferFromOwnString str_buf;
formatAST(*res, str_buf, hilite, oneline || approx_query_length < max_line_length);
if (insert_query_payload)
{
str_buf.write(' ');
copyData(*insert_query_payload, str_buf);
}
String res_string = str_buf.str();
const char * s_pos = res_string.data();
const char * s_end = s_pos + res_string.size();
/// remove trailing spaces
while (s_end > s_pos && isWhitespaceASCIIOneLine(*(s_end - 1)))
--s_end;
WriteBufferFromOStream res_cout(std::cout, 4096);
/// For multiline queries we print ';' at new line,
/// but for single line queries we print ';' at the same line
bool has_multiple_lines = false;
while (s_pos != s_end)
{
if (*s_pos == '\n')
has_multiple_lines = true;
res_cout.write(*s_pos++);
}
res_cout.finalize();
if (multiple && !insert_query_payload)
{
if (oneline || !has_multiple_lines)
std::cout << ";\n";
else
std::cout << "\n;\n";
}
else if (multiple && insert_query_payload)
/// Do not need to add ; because it's already in the insert_query_payload
std::cout << "\n";
std::cout << std::endl;
}
/// add additional '\' at the end of each line;
@ -232,27 +322,10 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
std::cout << std::endl;
}
}
do
{
/// skip spaces to avoid throw exception after last query
while (pos != end && std::isspace(*pos))
++pos;
/// for skip comment after the last query and to not throw exception
if (end - pos > 2 && *pos == '-' && *(pos + 1) == '-')
{
pos += 2;
/// skip until the end of the line
while (pos != end && *pos != '\n')
++pos;
}
/// need to parse next sql
else
break;
} while (pos != end);
} while (multiple && pos != end);
skipSpacesAndComments(pos, end, print_comments);
if (!multiple)
break;
}
}
}
catch (...)

View File

@ -43,6 +43,19 @@ void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, Atomic
}
}
void cancelOnDependencyFailure(const LoadJobPtr & self, const LoadJobPtr & dependency, std::exception_ptr & cancel)
{
cancel = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED,
"Load job '{}' -> {}",
self->name,
getExceptionMessage(dependency->exception(), /* with_stacktrace = */ false)));
}
void ignoreDependencyFailure(const LoadJobPtr &, const LoadJobPtr &, std::exception_ptr &)
{
// No-op
}
LoadStatus LoadJob::status() const
{
std::unique_lock lock{mutex};
@ -96,7 +109,10 @@ size_t LoadJob::canceled(const std::exception_ptr & ptr)
size_t LoadJob::finish()
{
func = {}; // To ensure job function is destructed before `AsyncLoader::wait()` return
// To ensure functions are destructed before `AsyncLoader::wait()` return
func = {};
dependency_failure = {};
finish_time = std::chrono::system_clock::now();
if (waiters > 0)
finished.notify_all();
@ -327,17 +343,19 @@ void AsyncLoader::schedule(const LoadJobSet & jobs_to_schedule)
if (dep_status == LoadStatus::FAILED || dep_status == LoadStatus::CANCELED)
{
// Dependency on already failed or canceled job -- it's okay. Cancel all dependent jobs.
std::exception_ptr e;
// Dependency on already failed or canceled job -- it's okay.
// Process as usual (may lead to cancel of all dependent jobs).
std::exception_ptr cancel;
NOEXCEPT_SCOPE({
ALLOW_ALLOCATIONS_IN_SCOPE;
e = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED,
"Load job '{}' -> {}",
job->name,
getExceptionMessage(dep->exception(), /* with_stacktrace = */ false)));
if (job->dependency_failure)
job->dependency_failure(job, dep, cancel);
});
finish(job, LoadStatus::CANCELED, e, lock);
break; // This job is now finished, stop its dependencies processing
if (cancel)
{
finish(job, LoadStatus::CANCELED, cancel, lock);
break; // This job is now finished, stop its dependencies processing
}
}
}
}
@ -515,63 +533,76 @@ String AsyncLoader::checkCycle(const LoadJobPtr & job, LoadJobSet & left, LoadJo
return {};
}
void AsyncLoader::finish(const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job, std::unique_lock<std::mutex> & lock)
void AsyncLoader::finish(const LoadJobPtr & job, LoadStatus status, std::exception_ptr reason, std::unique_lock<std::mutex> & lock)
{
chassert(scheduled_jobs.contains(job)); // Job was pending
// Notify waiters
size_t resumed_workers = 0; // Number of workers resumed in the execution pool of the job
if (status == LoadStatus::OK)
{
// Notify waiters
resumed_workers += job->ok();
resumed_workers = job->ok();
else if (status == LoadStatus::FAILED)
resumed_workers = job->failed(reason);
else if (status == LoadStatus::CANCELED)
resumed_workers = job->canceled(reason);
// Update dependent jobs and enqueue if ready
for (const auto & dep : scheduled_jobs[job].dependent_jobs)
// Adjust suspended workers count
if (resumed_workers)
{
Pool & pool = pools[job->executionPool()];
pool.suspended_workers -= resumed_workers;
}
Info & info = scheduled_jobs[job];
if (info.isReady())
{
// Job could be in ready queue (on cancel) -- must be dequeued
pools[job->pool_id].ready_queue.erase(info.ready_seqno);
info.ready_seqno = 0;
}
// To avoid container modification during recursion (during clean dependency graph edges below)
LoadJobSet dependent;
dependent.swap(info.dependent_jobs);
// Update dependent jobs
for (const auto & dpt : dependent)
{
if (auto dpt_info = scheduled_jobs.find(dpt); dpt_info != scheduled_jobs.end())
{
chassert(scheduled_jobs.contains(dep)); // All depended jobs must be pending
Info & dep_info = scheduled_jobs[dep];
dep_info.dependencies_left--;
if (!dep_info.isBlocked())
enqueue(dep_info, dep, lock);
dpt_info->second.dependencies_left--;
if (!dpt_info->second.isBlocked())
enqueue(dpt_info->second, dpt, lock);
if (status != LoadStatus::OK)
{
std::exception_ptr cancel;
NOEXCEPT_SCOPE({
ALLOW_ALLOCATIONS_IN_SCOPE;
if (dpt->dependency_failure)
dpt->dependency_failure(dpt, job, cancel);
});
// Recurse into dependent job if it should be canceled
if (cancel)
finish(dpt, LoadStatus::CANCELED, cancel, lock);
}
}
else
{
// Job has already been canceled. Do not enter twice into the same job during finish recursion.
// This happens in {A<-B; A<-C; B<-D; C<-D} graph for D if A is failed or canceled.
chassert(status == LoadStatus::CANCELED);
}
}
else
// Clean dependency graph edges pointing to canceled jobs
if (status != LoadStatus::OK)
{
// Notify waiters
if (status == LoadStatus::FAILED)
resumed_workers += job->failed(exception_from_job);
else if (status == LoadStatus::CANCELED)
resumed_workers += job->canceled(exception_from_job);
Info & info = scheduled_jobs[job];
if (info.isReady())
{
pools[job->pool_id].ready_queue.erase(info.ready_seqno);
info.ready_seqno = 0;
}
// Recurse into all dependent jobs
LoadJobSet dependent;
dependent.swap(info.dependent_jobs); // To avoid container modification during recursion
for (const auto & dep : dependent)
{
if (!scheduled_jobs.contains(dep))
continue; // Job has already been canceled
std::exception_ptr e;
NOEXCEPT_SCOPE({
ALLOW_ALLOCATIONS_IN_SCOPE;
e = std::make_exception_ptr(
Exception(ErrorCodes::ASYNC_LOAD_CANCELED,
"Load job '{}' -> {}",
dep->name,
getExceptionMessage(exception_from_job, /* with_stacktrace = */ false)));
});
finish(dep, LoadStatus::CANCELED, e, lock);
}
// Clean dependency graph edges pointing to canceled jobs
for (const auto & dep : job->dependencies)
{
if (auto dep_info = scheduled_jobs.find(dep); dep_info != scheduled_jobs.end())
dep_info->second.dependent_jobs.erase(job);
}
}
// Job became finished
@ -582,12 +613,6 @@ void AsyncLoader::finish(const LoadJobPtr & job, LoadStatus status, std::excepti
if (log_progress)
logAboutProgress(log, finished_jobs.size() - old_jobs, finished_jobs.size() + scheduled_jobs.size() - old_jobs, stopwatch);
});
if (resumed_workers)
{
Pool & pool = pools[job->executionPool()];
pool.suspended_workers -= resumed_workers;
}
}
void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool_id, std::unique_lock<std::mutex> & lock)
@ -612,6 +637,9 @@ void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool_id, std::un
}
job->pool_id.store(new_pool_id);
// TODO(serxa): we should adjust suspended_workers and suspended_waiters here.
// Otherwise suspended_workers we be left inconsistent. Fix it and add a test.
// Scenario: schedule a job A, wait for it from a job B in the same pool, prioritize A
// Recurse into dependencies
for (const auto & dep : job->dependencies)

View File

@ -1,6 +1,7 @@
#pragma once
#include <condition_variable>
#include <concepts>
#include <exception>
#include <memory>
#include <map>
@ -57,12 +58,13 @@ enum class LoadStatus
class LoadJob : private boost::noncopyable
{
public:
template <class Func, class LoadJobSetType>
LoadJob(LoadJobSetType && dependencies_, String name_, size_t pool_id_, Func && func_)
template <class LoadJobSetType, class Func, class DFFunc>
LoadJob(LoadJobSetType && dependencies_, String name_, size_t pool_id_, DFFunc && dependency_failure_, Func && func_)
: dependencies(std::forward<LoadJobSetType>(dependencies_))
, name(std::move(name_))
, execution_pool_id(pool_id_)
, pool_id(pool_id_)
, dependency_failure(std::forward<DFFunc>(dependency_failure_))
, func(std::forward<Func>(func_))
{}
@ -108,6 +110,14 @@ private:
std::atomic<UInt64> job_id{0};
std::atomic<size_t> execution_pool_id;
std::atomic<size_t> pool_id;
// Handler for failed or canceled dependencies.
// If job needs to be canceled on `dependency` failure, then function should set `cancel` to a specific reason.
// Note that implementation should be fast and cannot use AsyncLoader, because it is called under `AsyncLoader::mutex`.
// Note that `dependency_failure` is called only on pending jobs.
std::function<void(const LoadJobPtr & self, const LoadJobPtr & dependency, std::exception_ptr & cancel)> dependency_failure;
// Function to be called to execute the job.
std::function<void(AsyncLoader & loader, const LoadJobPtr & self)> func;
mutable std::mutex mutex;
@ -123,35 +133,54 @@ private:
std::atomic<TimePoint> finish_time{TimePoint{}};
};
struct EmptyJobFunc
{
void operator()(AsyncLoader &, const LoadJobPtr &) {}
};
// For LoadJob::dependency_failure. Cancels the job on the first dependency failure or cancel.
void cancelOnDependencyFailure(const LoadJobPtr & self, const LoadJobPtr & dependency, std::exception_ptr & cancel);
template <class Func = EmptyJobFunc>
LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, Func && func = EmptyJobFunc())
// For LoadJob::dependency_failure. Never cancels the job due to dependency failure or cancel.
void ignoreDependencyFailure(const LoadJobPtr & self, const LoadJobPtr & dependency, std::exception_ptr & cancel);
template <class F> concept LoadJobDependencyFailure = std::invocable<F, const LoadJobPtr &, const LoadJobPtr &, std::exception_ptr &>;
template <class F> concept LoadJobFunc = std::invocable<F, AsyncLoader &, const LoadJobPtr &>;
LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, LoadJobDependencyFailure auto && dependency_failure, LoadJobFunc auto && func)
{
return std::make_shared<LoadJob>(std::move(dependencies), std::move(name), 0, std::forward<Func>(func));
return std::make_shared<LoadJob>(std::move(dependencies), std::move(name), 0, std::forward<decltype(dependency_failure)>(dependency_failure), std::forward<decltype(func)>(func));
}
template <class Func = EmptyJobFunc>
LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, String name, Func && func = EmptyJobFunc())
LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, String name, LoadJobDependencyFailure auto && dependency_failure, LoadJobFunc auto && func)
{
return std::make_shared<LoadJob>(dependencies, std::move(name), 0, std::forward<Func>(func));
return std::make_shared<LoadJob>(dependencies, std::move(name), 0, std::forward<decltype(dependency_failure)>(dependency_failure), std::forward<decltype(func)>(func));
}
template <class Func = EmptyJobFunc>
LoadJobPtr makeLoadJob(LoadJobSet && dependencies, size_t pool_id, String name, Func && func = EmptyJobFunc())
LoadJobPtr makeLoadJob(LoadJobSet && dependencies, size_t pool_id, String name, LoadJobDependencyFailure auto && dependency_failure, LoadJobFunc auto && func)
{
return std::make_shared<LoadJob>(std::move(dependencies), std::move(name), pool_id, std::forward<Func>(func));
return std::make_shared<LoadJob>(std::move(dependencies), std::move(name), pool_id, std::forward<decltype(dependency_failure)>(dependency_failure), std::forward<decltype(func)>(func));
}
template <class Func = EmptyJobFunc>
LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, size_t pool_id, String name, Func && func = EmptyJobFunc())
LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, size_t pool_id, String name, LoadJobDependencyFailure auto && dependency_failure, LoadJobFunc auto && func)
{
return std::make_shared<LoadJob>(dependencies, std::move(name), pool_id, std::forward<Func>(func));
return std::make_shared<LoadJob>(dependencies, std::move(name), pool_id, std::forward<decltype(dependency_failure)>(dependency_failure), std::forward<decltype(func)>(func));
}
LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, LoadJobFunc auto && func)
{
return std::make_shared<LoadJob>(std::move(dependencies), std::move(name), 0, cancelOnDependencyFailure, std::forward<decltype(func)>(func));
}
LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, String name, LoadJobFunc auto && func)
{
return std::make_shared<LoadJob>(dependencies, std::move(name), 0, cancelOnDependencyFailure, std::forward<decltype(func)>(func));
}
LoadJobPtr makeLoadJob(LoadJobSet && dependencies, size_t pool_id, String name, LoadJobFunc auto && func)
{
return std::make_shared<LoadJob>(std::move(dependencies), std::move(name), pool_id, cancelOnDependencyFailure, std::forward<decltype(func)>(func));
}
LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, size_t pool_id, String name, LoadJobFunc auto && func)
{
return std::make_shared<LoadJob>(dependencies, std::move(name), pool_id, cancelOnDependencyFailure, std::forward<decltype(func)>(func));
}
// Represents a logically connected set of LoadJobs required to achieve some goals (final LoadJob in the set).
class LoadTask : private boost::noncopyable
@ -277,7 +306,7 @@ private:
{
size_t dependencies_left = 0; // Current number of dependencies on pending jobs.
UInt64 ready_seqno = 0; // Zero means that job is not in ready queue.
LoadJobSet dependent_jobs; // Set of jobs dependent on this job.
LoadJobSet dependent_jobs; // Set of jobs dependent on this job. Contains only scheduled jobs.
// Three independent states of a scheduled job.
bool isBlocked() const { return dependencies_left > 0; }
@ -371,7 +400,7 @@ public:
private:
void checkCycle(const LoadJobSet & jobs, std::unique_lock<std::mutex> & lock);
String checkCycle(const LoadJobPtr & job, LoadJobSet & left, LoadJobSet & visited, std::unique_lock<std::mutex> & lock);
void finish(const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job, std::unique_lock<std::mutex> & lock);
void finish(const LoadJobPtr & job, LoadStatus status, std::exception_ptr reason, std::unique_lock<std::mutex> & lock);
void gatherNotScheduled(const LoadJobPtr & job, LoadJobSet & jobs, std::unique_lock<std::mutex> & lock);
void prioritize(const LoadJobPtr & job, size_t new_pool_id, std::unique_lock<std::mutex> & lock);
void enqueue(Info & info, const LoadJobPtr & job, std::unique_lock<std::mutex> & lock);

View File

@ -1,10 +1,11 @@
#pragma once
#include <list>
#include <memory>
#include <mutex>
#include <optional>
#include <base/types.h>
#include <boost/core/noncopyable.hpp>
#include <mutex>
#include <memory>
#include <list>
namespace DB

View File

@ -242,7 +242,7 @@
M(FilesystemCacheDelayedCleanupElements, "Filesystem cache elements in background cleanup queue") \
M(FilesystemCacheHoldFileSegments, "Filesystem cache file segment which are currently hold as unreleasable") \
M(AsyncInsertCacheSize, "Number of async insert hash id in cache") \
M(S3Requests, "S3 requests") \
M(S3Requests, "S3 requests count") \
M(KeeperAliveConnections, "Number of alive connections") \
M(KeeperOutstandingRequets, "Number of outstanding requests") \
M(ThreadsInOvercommitTracker, "Number of waiting threads inside of OvercommitTracker") \

View File

@ -0,0 +1,494 @@
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
# pragma clang diagnostic ignored "-Wgnu-anonymous-struct"
# pragma clang diagnostic ignored "-Wnested-anon-types"
# pragma clang diagnostic ignored "-Wunused-parameter"
# pragma clang diagnostic ignored "-Wshadow-field-in-constructor"
# pragma clang diagnostic ignored "-Wdtor-name"
#endif
#include <re2/re2.h>
#include <re2/regexp.h>
#include <re2/walker-inl.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
#ifdef LOG_INFO
#undef LOG_INFO
#undef LOG_WARNING
#undef LOG_ERROR
#undef LOG_FATAL
#endif
#include "MatchGenerator.h"
#include <Common/Exception.h>
#include <Common/thread_local_rng.h>
#include <map>
#include <functional>
#include <magic_enum.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
}
}
namespace re2
{
class RandomStringPrepareWalker : public Regexp::Walker<Regexp *>
{
private:
static constexpr int ImplicitMax = 100;
using Children = std::vector<Regexp *>;
class Generators;
/// This function objects look much prettier than lambda expression when stack traces are printed
class NodeFunction
{
public:
virtual size_t operator() (char * out, size_t size) = 0;
virtual size_t getRequiredSize() = 0;
virtual ~NodeFunction() = default;
};
using NodeFunctionPtr = std::shared_ptr<NodeFunction>;
using NodeFuncs = std::vector<NodeFunctionPtr>;
static NodeFuncs getFuncs(const Children & children_, const Generators & generators_)
{
NodeFuncs result;
result.reserve(children_.size());
for (auto * child: children_)
{
result.push_back(generators_.at(child));
}
return result;
}
class Generators: public std::map<re2::Regexp *, NodeFunctionPtr> {};
class RegexpConcatFunction : public NodeFunction
{
public:
RegexpConcatFunction(const Children & children_, const Generators & generators_)
: children(getFuncs(children_, generators_))
{
}
size_t operator () (char * out, size_t size) override
{
size_t total_size = 0;
for (auto & child: children)
{
size_t consumed = child->operator()(out, size);
chassert(consumed <= size);
out += consumed;
size -= consumed;
total_size += consumed;
}
return total_size;
}
size_t getRequiredSize() override
{
size_t total_size = 0;
for (auto & child: children)
total_size += child->getRequiredSize();
return total_size;
}
private:
NodeFuncs children;
};
class RegexpAlternateFunction : public NodeFunction
{
public:
RegexpAlternateFunction(const Children & children_, const Generators & generators_)
: children(getFuncs(children_, generators_))
{
}
size_t operator () (char * out, size_t size) override
{
std::uniform_int_distribution<int> distribution(0, static_cast<int>(children.size()-1));
int chosen = distribution(thread_local_rng);
size_t consumed = children[chosen]->operator()(out, size);
chassert(consumed <= size);
return consumed;
}
size_t getRequiredSize() override
{
size_t total_size = 0;
for (auto & child: children)
total_size = std::max(total_size, child->getRequiredSize());
return total_size;
}
private:
NodeFuncs children;
};
class RegexpRepeatFunction : public NodeFunction
{
public:
RegexpRepeatFunction(Regexp * re_, const Generators & generators_, int min_repeat_, int max_repeat_)
: func(generators_.at(re_))
, min_repeat(min_repeat_)
, max_repeat(max_repeat_)
{
}
size_t operator () (char * out, size_t size) override
{
std::uniform_int_distribution<int> distribution(min_repeat, max_repeat);
int ntimes = distribution(thread_local_rng);
size_t total_size = 0;
for (int i = 0; i < ntimes; ++i)
{
size_t consumed =func->operator()(out, size);
chassert(consumed <= size);
out += consumed;
size -= consumed;
total_size += consumed;
}
return total_size;
}
size_t getRequiredSize() override
{
return max_repeat * func->getRequiredSize();
}
private:
NodeFunctionPtr func;
int min_repeat = 0;
int max_repeat = 0;
};
class RegexpCharClassFunction : public NodeFunction
{
using CharRanges = std::vector<std::pair<re2::Rune, re2::Rune>>;
public:
explicit RegexpCharClassFunction(Regexp * re_)
{
CharClass * cc = re_->cc();
chassert(cc);
if (cc->empty())
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "kRegexpCharClass is empty");
char_count = cc->size();
char_ranges.reserve(std::distance(cc->begin(), cc->end()));
for (const auto range: *cc)
{
char_ranges.emplace_back(range.lo, range.hi);
}
}
size_t operator () (char * out, size_t size) override
{
chassert(UTFmax <= size);
std::uniform_int_distribution<int> distribution(1, char_count);
int chosen = distribution(thread_local_rng);
int count_down = chosen;
auto it = char_ranges.begin();
for (; it != char_ranges.end(); ++it)
{
auto [lo, hi] = *it;
auto range_len = hi - lo + 1;
if (count_down <= range_len)
break;
count_down -= range_len;
}
if (it == char_ranges.end())
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR,
"Unable to choose the rune. Runes {}, ranges {}, chosen {}",
char_count, char_ranges.size(), chosen);
auto [lo, _] = *it;
Rune r = lo + count_down - 1;
return re2::runetochar(out, &r);
}
size_t getRequiredSize() override
{
return UTFmax;
}
private:
int char_count = 0;
CharRanges char_ranges;
};
class RegexpLiteralStringFunction : public NodeFunction
{
public:
explicit RegexpLiteralStringFunction(Regexp * re_)
{
if (re_->nrunes() == 0)
return;
char buffer[UTFmax];
for (int i = 0; i < re_->nrunes(); ++i)
{
int n = re2::runetochar(buffer, &re_->runes()[i]);
literal_string += String(buffer, n);
}
}
size_t operator () (char * out, size_t size) override
{
chassert(literal_string.size() <= size);
memcpy(out, literal_string.data(), literal_string.size());
return literal_string.size();
}
size_t getRequiredSize() override
{
return literal_string.size();
}
private:
String literal_string;
};
class RegexpLiteralFunction : public NodeFunction
{
public:
explicit RegexpLiteralFunction(Regexp * re_)
{
char buffer[UTFmax];
Rune r = re_->rune();
int n = re2::runetochar(buffer, &r);
literal = String(buffer, n);
}
size_t operator () (char * out, size_t size) override
{
chassert(literal.size() <= size);
memcpy(out, literal.data(), literal.size());
return literal.size();
}
size_t getRequiredSize() override
{
return literal.size();
}
private:
String literal;
};
class ThrowExceptionFunction : public NodeFunction
{
public:
explicit ThrowExceptionFunction(Regexp * re_)
: operation(magic_enum::enum_name(re_->op()))
{
}
size_t operator () (char *, size_t) override
{
throw DB::Exception(
DB::ErrorCodes::BAD_ARGUMENTS,
"RandomStringPrepareWalker: regexp node '{}' is not supported for generating a random match",
operation);
}
size_t getRequiredSize() override
{
return 0;
}
private:
String operation;
};
public:
std::function<String()> getGenerator()
{
if (root == nullptr)
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "no root has been set");
if (generators.empty())
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "no generators");
auto root_func = generators.at(root);
auto required_buffer_size = root_func->getRequiredSize();
auto generator_func = [=] ()
-> String
{
auto buffer = String(required_buffer_size, '\0');
size_t size = root_func->operator()(buffer.data(), buffer.size());
buffer.resize(size);
return buffer;
};
root = nullptr;
generators = {};
return std::move(generator_func);
}
private:
Children CopyChildrenArgs(Regexp ** children, int nchild)
{
Children result;
result.reserve(nchild);
for (int i = 0; i < nchild; ++i)
result.push_back(Copy(children[i]));
return result;
}
Regexp * ShortVisit(Regexp* /*re*/, Regexp * /*parent_arg*/) override
{
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "ShortVisit should not be called");
}
Regexp * PreVisit(Regexp * re, Regexp * parent_arg, bool* /*stop*/) override /*noexcept*/
{
if (parent_arg == nullptr)
{
chassert(root == nullptr);
chassert(re != nullptr);
root = re;
}
return re;
}
Regexp * PostVisit(Regexp * re, Regexp * /*parent_arg*/, Regexp * pre_arg,
Regexp ** child_args, int nchild_args) override /*noexcept*/
{
switch (re->op())
{
case kRegexpConcat: // Matches concatenation of sub_[0..nsub-1].
generators[re] = std::make_shared<RegexpConcatFunction>(CopyChildrenArgs(child_args, nchild_args), generators);
break;
case kRegexpAlternate: // Matches union of sub_[0..nsub-1].
generators[re] = std::make_shared<RegexpAlternateFunction>(CopyChildrenArgs(child_args, nchild_args), generators);
break;
case kRegexpQuest: // Matches sub_[0] zero or one times.
chassert(nchild_args == 1);
generators[re] = std::make_shared<RegexpRepeatFunction>(child_args[0], generators, 0, 1);
break;
case kRegexpStar: // Matches sub_[0] zero or more times.
chassert(nchild_args == 1);
generators[re] = std::make_shared<RegexpRepeatFunction>(child_args[0], generators, 0, ImplicitMax);
break;
case kRegexpPlus: // Matches sub_[0] one or more times.
chassert(nchild_args == 1);
generators[re] = std::make_shared<RegexpRepeatFunction>(child_args[0], generators, 1, ImplicitMax);
break;
case kRegexpCharClass: // Matches character class given by cc_.
chassert(nchild_args == 0);
generators[re] = std::make_shared<RegexpCharClassFunction>(re);
break;
case kRegexpLiteralString: // Matches runes_.
chassert(nchild_args == 0);
generators[re] = std::make_shared<RegexpLiteralStringFunction>(re);
break;
case kRegexpLiteral: // Matches rune_.
chassert(nchild_args == 0);
generators[re] = std::make_shared<RegexpLiteralFunction>(re);
break;
case kRegexpCapture: // Parenthesized (capturing) subexpression.
chassert(nchild_args == 1);
generators[re] = generators.at(child_args[0]);
break;
case kRegexpNoMatch: // Matches no strings.
case kRegexpEmptyMatch: // Matches empty string.
case kRegexpRepeat: // Matches sub_[0] at least min_ times, at most max_ times.
case kRegexpAnyChar: // Matches any character.
case kRegexpAnyByte: // Matches any byte [sic].
case kRegexpBeginLine: // Matches empty string at beginning of line.
case kRegexpEndLine: // Matches empty string at end of line.
case kRegexpWordBoundary: // Matches word boundary "\b".
case kRegexpNoWordBoundary: // Matches not-a-word boundary "\B".
case kRegexpBeginText: // Matches empty string at beginning of text.
case kRegexpEndText: // Matches empty string at end of text.
case kRegexpHaveMatch: // Forces match of entire expression
generators[re] = std::make_shared<ThrowExceptionFunction>(re);
}
return pre_arg;
}
Regexp * root = nullptr;
Generators generators;
};
}
namespace DB
{
void RandomStringGeneratorByRegexp::RegexpPtrDeleter::operator() (re2::Regexp * re) const noexcept
{
re->Decref();
}
RandomStringGeneratorByRegexp::RandomStringGeneratorByRegexp(const String & re_str)
{
re2::RE2::Options options;
options.set_case_sensitive(true);
options.set_encoding(re2::RE2::Options::EncodingLatin1);
auto flags = static_cast<re2::Regexp::ParseFlags>(options.ParseFlags());
re2::RegexpStatus status;
regexp.reset(re2::Regexp::Parse(re_str, flags, &status));
if (!regexp)
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS,
"Error parsing regexp '{}': {}",
re_str, status.Text());
regexp.reset(regexp->Simplify());
auto walker = re2::RandomStringPrepareWalker();
walker.Walk(regexp.get(), {});
generatorFunc = walker.getGenerator();
{
auto test_check = generate();
auto matched = RE2::FullMatch(test_check, re2::RE2(re_str));
if (!matched)
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS,
"Generator is unable to produce random string for regexp '{}': {}",
re_str, test_check);
}
}
String RandomStringGeneratorByRegexp::generate() const
{
chassert(generatorFunc);
return generatorFunc();
}
}

View File

@ -0,0 +1,31 @@
#pragma once
#include <base/types.h>
#include <memory>
namespace re2
{
class Regexp;
}
namespace DB
{
class RandomStringGeneratorByRegexp
{
public:
explicit RandomStringGeneratorByRegexp(const String & re_str);
String generate() const;
private:
struct RegexpPtrDeleter
{
void operator()(re2::Regexp * re) const noexcept;
};
using RegexpPtr = std::unique_ptr<re2::Regexp, RegexpPtrDeleter>;
RegexpPtr regexp;
std::function<String()> generatorFunc;
};
}

View File

@ -65,4 +65,5 @@ ObjectStorageKey ObjectStorageKey::createAsAbsolute(String key_)
object_key.is_relative = false;
return object_key;
}
}

View File

@ -0,0 +1,94 @@
#include "ObjectStorageKeyGenerator.h"
#include <Common/getRandomASCIIString.h>
#include <Common/MatchGenerator.h>
#include <fmt/format.h>
class GeneratorWithTemplate : public DB::IObjectStorageKeysGenerator
{
public:
explicit GeneratorWithTemplate(String key_template_)
: key_template(std::move(key_template_))
, re_gen(key_template)
{
}
DB::ObjectStorageKey generate(const String &) const override
{
return DB::ObjectStorageKey::createAsAbsolute(re_gen.generate());
}
private:
String key_template;
DB::RandomStringGeneratorByRegexp re_gen;
};
class GeneratorWithPrefix : public DB::IObjectStorageKeysGenerator
{
public:
explicit GeneratorWithPrefix(String key_prefix_)
: key_prefix(std::move(key_prefix_))
{}
DB::ObjectStorageKey generate(const String &) const override
{
/// Path to store the new S3 object.
/// Total length is 32 a-z characters for enough randomness.
/// First 3 characters are used as a prefix for
/// https://aws.amazon.com/premiumsupport/knowledge-center/s3-object-key-naming-pattern/
constexpr size_t key_name_total_size = 32;
constexpr size_t key_name_prefix_size = 3;
/// Path to store new S3 object.
String key = fmt::format("{}/{}",
DB::getRandomASCIIString(key_name_prefix_size),
DB::getRandomASCIIString(key_name_total_size - key_name_prefix_size));
/// what ever key_prefix value is, consider that key as relative
return DB::ObjectStorageKey::createAsRelative(key_prefix, key);
}
private:
String key_prefix;
};
class GeneratorAsIsWithPrefix : public DB::IObjectStorageKeysGenerator
{
public:
explicit GeneratorAsIsWithPrefix(String key_prefix_)
: key_prefix(std::move(key_prefix_))
{}
DB::ObjectStorageKey generate(const String & path) const override
{
return DB::ObjectStorageKey::createAsRelative(key_prefix, path);
}
private:
String key_prefix;
};
namespace DB
{
ObjectStorageKeysGeneratorPtr createObjectStorageKeysGeneratorAsIsWithPrefix(String key_prefix)
{
return std::make_shared<GeneratorAsIsWithPrefix>(std::move(key_prefix));
}
ObjectStorageKeysGeneratorPtr createObjectStorageKeysGeneratorByPrefix(String key_prefix)
{
return std::make_shared<GeneratorWithPrefix>(std::move(key_prefix));
}
ObjectStorageKeysGeneratorPtr createObjectStorageKeysGeneratorByTemplate(String key_template)
{
return std::make_shared<GeneratorWithTemplate>(std::move(key_template));
}
}

View File

@ -0,0 +1,22 @@
#pragma once
#include "ObjectStorageKey.h"
#include <memory>
namespace DB
{
class IObjectStorageKeysGenerator
{
public:
virtual ObjectStorageKey generate(const String & path) const = 0;
virtual ~IObjectStorageKeysGenerator() = default;
};
using ObjectStorageKeysGeneratorPtr = std::shared_ptr<IObjectStorageKeysGenerator>;
ObjectStorageKeysGeneratorPtr createObjectStorageKeysGeneratorAsIsWithPrefix(String key_prefix);
ObjectStorageKeysGeneratorPtr createObjectStorageKeysGeneratorByPrefix(String key_prefix);
ObjectStorageKeysGeneratorPtr createObjectStorageKeysGeneratorByTemplate(String key_template);
}

View File

@ -391,6 +391,9 @@ The server successfully detected this situation and will download merged part fr
M(DiskS3PutObject, "Number of DiskS3 API PutObject calls.") \
M(DiskS3GetObject, "Number of DiskS3 API GetObject calls.") \
\
M(S3Clients, "Number of created S3 clients.") \
M(TinyS3Clients, "Number of S3 clients copies which reuse an existing auth provider from another client.") \
\
M(EngineFileLikeReadFiles, "Number of files read in table engines working with files (like File/S3/URL/HDFS).") \
\
M(ReadBufferFromS3Microseconds, "Time spent on reading from S3.") \

View File

@ -296,6 +296,9 @@ constexpr std::pair<std::string_view, std::string_view> replacements[]
// Replace parts from @c replacements with shorter aliases
String demangleAndCollapseNames(std::string_view file, const char * const symbol_name)
{
if (!symbol_name)
return "?";
std::string_view file_copy = file;
if (auto trim_pos = file.find_last_of('/'); trim_pos != file.npos)
file_copy.remove_suffix(file.size() - trim_pos);

View File

@ -2,6 +2,8 @@
#include <gtest/gtest.h>
#include <array>
#include <atomic>
#include <exception>
#include <list>
#include <barrier>
#include <chrono>
@ -544,6 +546,99 @@ TEST(AsyncLoader, ScheduleJobWithCanceledDependencies)
}
}
TEST(AsyncLoader, IgnoreDependencyFailure)
{
AsyncLoaderTest t;
std::atomic<bool> success{false};
t.loader.start();
std::string_view error_message = "test job failure";
auto failed_job_func = [&] (AsyncLoader &, const LoadJobPtr &) {
throw Exception(ErrorCodes::ASYNC_LOAD_FAILED, "{}", error_message);
};
auto dependent_job_func = [&] (AsyncLoader &, const LoadJobPtr &) {
success.store(true);
};
auto failed_job = makeLoadJob({}, "failed_job", failed_job_func);
auto dependent_job = makeLoadJob({failed_job},
"dependent_job", ignoreDependencyFailure, dependent_job_func);
auto task = t.schedule({ failed_job, dependent_job });
t.loader.wait();
ASSERT_EQ(failed_job->status(), LoadStatus::FAILED);
ASSERT_EQ(dependent_job->status(), LoadStatus::OK);
ASSERT_EQ(success.load(), true);
}
TEST(AsyncLoader, CustomDependencyFailure)
{
AsyncLoaderTest t(16);
int error_count = 0;
std::atomic<size_t> good_count{0};
std::barrier canceled_sync(4);
t.loader.start();
std::string_view error_message = "test job failure";
auto evil_dep_func = [&] (AsyncLoader &, const LoadJobPtr &) {
throw Exception(ErrorCodes::ASYNC_LOAD_FAILED, "{}", error_message);
};
auto good_dep_func = [&] (AsyncLoader &, const LoadJobPtr &) {
good_count++;
};
auto late_dep_func = [&] (AsyncLoader &, const LoadJobPtr &) {
canceled_sync.arrive_and_wait(); // wait for fail (A) before this job is finished
};
auto collect_job_func = [&] (AsyncLoader &, const LoadJobPtr &) {
FAIL(); // job should be canceled, so we never get here
};
auto dependent_job_func = [&] (AsyncLoader &, const LoadJobPtr &) {
FAIL(); // job should be canceled, so we never get here
};
auto fail_after_two = [&] (const LoadJobPtr & self, const LoadJobPtr &, std::exception_ptr & cancel) {
if (++error_count == 2)
cancel = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED,
"Load job '{}' canceled: too many dependencies have failed",
self->name));
};
auto evil_dep1 = makeLoadJob({}, "evil_dep1", evil_dep_func);
auto evil_dep2 = makeLoadJob({}, "evil_dep2", evil_dep_func);
auto evil_dep3 = makeLoadJob({}, "evil_dep3", evil_dep_func);
auto good_dep1 = makeLoadJob({}, "good_dep1", good_dep_func);
auto good_dep2 = makeLoadJob({}, "good_dep2", good_dep_func);
auto good_dep3 = makeLoadJob({}, "good_dep3", good_dep_func);
auto late_dep1 = makeLoadJob({}, "late_dep1", late_dep_func);
auto late_dep2 = makeLoadJob({}, "late_dep2", late_dep_func);
auto late_dep3 = makeLoadJob({}, "late_dep3", late_dep_func);
auto collect_job = makeLoadJob({
evil_dep1, evil_dep2, evil_dep3,
good_dep1, good_dep2, good_dep3,
late_dep1, late_dep2, late_dep3
}, "collect_job", fail_after_two, collect_job_func);
auto dependent_job1 = makeLoadJob({ collect_job }, "dependent_job1", dependent_job_func);
auto dependent_job2 = makeLoadJob({ collect_job }, "dependent_job2", dependent_job_func);
auto dependent_job3 = makeLoadJob({ collect_job }, "dependent_job3", dependent_job_func);
auto task = t.schedule({ dependent_job1, dependent_job2, dependent_job3 }); // Other jobs should be discovery automatically
t.loader.wait(collect_job, true);
canceled_sync.arrive_and_wait(); // (A)
t.loader.wait();
ASSERT_EQ(late_dep1->status(), LoadStatus::OK);
ASSERT_EQ(late_dep2->status(), LoadStatus::OK);
ASSERT_EQ(late_dep3->status(), LoadStatus::OK);
ASSERT_EQ(collect_job->status(), LoadStatus::CANCELED);
ASSERT_EQ(dependent_job1->status(), LoadStatus::CANCELED);
ASSERT_EQ(dependent_job2->status(), LoadStatus::CANCELED);
ASSERT_EQ(dependent_job3->status(), LoadStatus::CANCELED);
ASSERT_EQ(good_count.load(), 3);
}
TEST(AsyncLoader, TestConcurrency)
{
AsyncLoaderTest t(10);

View File

@ -0,0 +1,101 @@
#include <Common/MatchGenerator.h>
#include <Common/ObjectStorageKeyGenerator.h>
#include <Common/Stopwatch.h>
#include <Common/Exception.h>
#include <gtest/gtest.h>
void routine(String s)
{
std::cerr << "case '"<< s << "'";
auto gen = DB::RandomStringGeneratorByRegexp(s);
[[maybe_unused]] auto res = gen.generate();
std::cerr << " result '"<< res << "'" << std::endl;
}
TEST(GenerateRandomString, Positive)
{
routine(".");
routine("[[:xdigit:]]");
routine("[0-9a-f]");
routine("[a-z]");
routine("prefix-[0-9a-f]-suffix");
routine("prefix-[a-z]-suffix");
routine("[0-9a-f]{3}");
routine("prefix-[0-9a-f]{3}-suffix");
routine("prefix-[a-z]{3}-suffix/[0-9a-f]{20}");
routine("left|right");
routine("[a-z]{0,3}");
routine("just constant string");
routine("[a-z]?");
routine("[a-z]*");
routine("[a-z]+");
routine("[^a-z]");
routine("[[:lower:]]{3}/suffix");
routine("prefix-(A|B|[0-9a-f]){3}");
routine("mergetree/[a-z]{3}/[a-z]{29}");
}
TEST(GenerateRandomString, Negative)
{
EXPECT_THROW(routine("[[:do_not_exists:]]"), DB::Exception);
EXPECT_THROW(routine("[:do_not_exis..."), DB::Exception);
EXPECT_THROW(routine("^abc"), DB::Exception);
}
TEST(GenerateRandomString, DifferentResult)
{
std::cerr << "100 different keys" << std::endl;
auto gen = DB::RandomStringGeneratorByRegexp("prefix-[a-z]{3}-suffix/[0-9a-f]{20}");
std::set<String> deduplicate;
for (int i = 0; i < 100; ++i)
ASSERT_TRUE(deduplicate.insert(gen.generate()).second);
std::cerr << "100 different keys: ok" << std::endl;
}
TEST(GenerateRandomString, FullRange)
{
std::cerr << "all possible letters" << std::endl;
auto gen = DB::RandomStringGeneratorByRegexp("[a-z]");
std::set<String> deduplicate;
int count = 'z' - 'a' + 1;
while (deduplicate.size() < count)
if (deduplicate.insert(gen.generate()).second)
std::cerr << " +1 ";
std::cerr << "all possible letters, ok" << std::endl;
}
UInt64 elapsed(DB::ObjectStorageKeysGeneratorPtr generator)
{
String path = "some_path";
Stopwatch watch;
for (int i = 0; i < 100000; ++i)
{
[[ maybe_unused ]] auto result = generator->generate(path).serialize();
}
return watch.elapsedMicroseconds();
}
TEST(ObjectStorageKey, Performance)
{
auto elapsed_old = elapsed(DB::createObjectStorageKeysGeneratorByPrefix(
"xx-xx-xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx/mergetree/"));
std::cerr << "old: " << elapsed_old << std::endl;
auto elapsed_new = elapsed(DB::createObjectStorageKeysGeneratorByTemplate(
"xx-xx-xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx/mergetree/[a-z]{3}/[a-z]{29}"));
std::cerr << "new: " << elapsed_new << std::endl;
if (elapsed_new > elapsed_old)
{
if (elapsed_new > elapsed_old)
std::cerr << "slow ratio: +" << float(elapsed_new) / elapsed_old << std::endl;
else
std::cerr << "fast ratio: " << float(elapsed_old) / elapsed_new << std::endl;
ASSERT_LT(elapsed_new, 1.2 * elapsed_old);
}
}

View File

@ -70,7 +70,7 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo
{
std::lock_guard client_lock{snapshot_s3_client_mutex};
// if client is not changed (same auth settings, same endpoint) we don't need to update
if (snapshot_s3_client && snapshot_s3_client->client && auth_settings == snapshot_s3_client->auth_settings
if (snapshot_s3_client && snapshot_s3_client->client && !snapshot_s3_client->auth_settings.hasUpdates(auth_settings)
&& snapshot_s3_client->uri.uri == new_uri.uri)
return;
}

View File

@ -4,6 +4,8 @@
#include <fmt/core.h>
#include <libnuraft/srv_config.hxx>
#include <optional>
namespace DB
{
// default- and copy-constructible version of nuraft::srv_config

View File

@ -204,6 +204,8 @@ class IColumn;
M(Bool, input_format_parallel_parsing, true, "Enable parallel parsing for some data formats.", 0) \
M(UInt64, min_chunk_bytes_for_parallel_parsing, (10 * 1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.", 0) \
M(Bool, output_format_parallel_formatting, true, "Enable parallel formatting for some data formats.", 0) \
M(UInt64, output_format_compression_level, 3, "Default compression level if query output is compressed. The setting is applied when `SELECT` query has `INTO OUTFILE` or when inserting to table function `file`, `url`, `hdfs`, `s3`, and `azureBlobStorage`.", 0) \
M(UInt64, output_format_compression_zstd_window_log, 0, "Can be used when the output compression method is `zstd`. If greater than `0`, this setting explicitly sets compression window size (power of `2`) and enables a long-range mode for zstd compression.", 0) \
\
M(UInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.", 0) \
M(UInt64, merge_tree_min_bytes_for_concurrent_read, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized.", 0) \

View File

@ -227,11 +227,17 @@ LoadTaskPtr DatabaseOrdinary::startupDatabaseAsync(
LoadJobSet startup_after,
LoadingStrictnessLevel /*mode*/)
{
// NOTE: this task is empty, but it is required for correct dependency handling (startup should be done after tables loading)
auto job = makeLoadJob(
std::move(startup_after),
TablesLoaderBackgroundStartupPoolId,
fmt::format("startup Ordinary database {}", getDatabaseName()));
fmt::format("startup Ordinary database {}", getDatabaseName()),
ignoreDependencyFailure,
[] (AsyncLoader &, const LoadJobPtr &)
{
// NOTE: this job is no-op, but it is required for correct dependency handling
// 1) startup should be done after tables loading
// 2) load or startup errors for tables should not lead to not starting up the whole database
});
return startup_database_task = makeLoadTask(async_loader, {job});
}

View File

@ -258,12 +258,6 @@ String DiskObjectStorage::getUniqueId(const String & path) const
bool DiskObjectStorage::checkUniqueId(const String & id) const
{
if (!id.starts_with(object_key_prefix))
{
LOG_DEBUG(log, "Blob with id {} doesn't start with blob storage prefix {}, Stack {}", id, object_key_prefix, StackTrace().toString());
return false;
}
auto object = StoredObject(id);
return object_storage->exists(object);
}

View File

@ -20,6 +20,7 @@ namespace ErrorCodes
void DiskObjectStorageMetadata::deserialize(ReadBuffer & buf)
{
readIntText(version, buf);
assertChar('\n', buf);
if (version < VERSION_ABSOLUTE_PATHS || version > VERSION_FULL_OBJECT_KEY)
throw Exception(
@ -27,8 +28,6 @@ void DiskObjectStorageMetadata::deserialize(ReadBuffer & buf)
"Unknown metadata file version. Path: {}. Version: {}. Maximum expected version: {}",
metadata_file_path, toString(version), toString(VERSION_FULL_OBJECT_KEY));
assertChar('\n', buf);
UInt32 keys_count;
readIntText(keys_count, buf);
assertChar('\t', buf);
@ -122,6 +121,7 @@ void DiskObjectStorageMetadata::serialize(WriteBuffer & buf, bool sync) const
chassert(write_version >= VERSION_ABSOLUTE_PATHS && write_version <= VERSION_FULL_OBJECT_KEY);
writeIntText(write_version, buf);
writeChar('\n', buf);
writeIntText(keys_with_meta.size(), buf);

View File

@ -19,7 +19,6 @@
#include <Disks/ObjectStorages/S3/diskSettings.h>
#include <Common/getRandomASCIIString.h>
#include <Common/ProfileEvents.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/logger_useful.h>
@ -556,27 +555,12 @@ std::unique_ptr<IObjectStorage> S3ObjectStorage::cloneObjectStorage(
return std::make_unique<S3ObjectStorage>(
std::move(new_client), std::move(new_s3_settings),
version_id, s3_capabilities, new_namespace,
endpoint, object_key_prefix, disk_name);
endpoint, key_generator, disk_name);
}
ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string &) const
ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string & path) const
{
/// Path to store the new S3 object.
/// Total length is 32 a-z characters for enough randomness.
/// First 3 characters are used as a prefix for
/// https://aws.amazon.com/premiumsupport/knowledge-center/s3-object-key-naming-pattern/
constexpr size_t key_name_total_size = 32;
constexpr size_t key_name_prefix_size = 3;
/// Path to store new S3 object.
String key = fmt::format("{}/{}",
getRandomASCIIString(key_name_prefix_size),
getRandomASCIIString(key_name_total_size - key_name_prefix_size));
/// what ever key_prefix value is, consider that key as relative
return ObjectStorageKey::createAsRelative(object_key_prefix, key);
return key_generator->generate(path);
}

View File

@ -9,6 +9,7 @@
#include <memory>
#include <Storages/StorageS3Settings.h>
#include <Common/MultiVersion.h>
#include <Common/ObjectStorageKeyGenerator.h>
namespace DB
@ -39,7 +40,6 @@ struct S3ObjectStorageSettings
bool read_only;
};
class S3ObjectStorage : public IObjectStorage
{
private:
@ -53,10 +53,10 @@ private:
const S3Capabilities & s3_capabilities_,
String bucket_,
String connection_string,
String object_key_prefix_,
ObjectStorageKeysGeneratorPtr key_generator_,
const String & disk_name_)
: bucket(std::move(bucket_))
, object_key_prefix(std::move(object_key_prefix_))
, key_generator(std::move(key_generator_))
, disk_name(disk_name_)
, client(std::move(client_))
, s3_settings(std::move(s3_settings_))
@ -179,7 +179,7 @@ private:
private:
std::string bucket;
String object_key_prefix;
ObjectStorageKeysGeneratorPtr key_generator;
std::string disk_name;
MultiVersion<S3::Client> client;
@ -199,11 +199,6 @@ private:
class S3PlainObjectStorage : public S3ObjectStorage
{
public:
ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override
{
return ObjectStorageKey::createAsRelative(object_key_prefix, path);
}
std::string getName() const override { return "S3PlainObjectStorage"; }
template <class ...Args>

View File

@ -91,6 +91,60 @@ private:
}
};
std::pair<String, ObjectStorageKeysGeneratorPtr> getPrefixAndKeyGenerator(
String type, const S3::URI & uri, const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
{
if (type == "s3_plain")
return {uri.key, createObjectStorageKeysGeneratorAsIsWithPrefix(uri.key)};
chassert(type == "s3");
bool storage_metadata_write_full_object_key = DiskObjectStorageMetadata::getWriteFullObjectKeySetting();
bool send_metadata = config.getBool(config_prefix + ".send_metadata", false);
if (send_metadata && storage_metadata_write_full_object_key)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Wrong configuration in {}. "
"s3 does not supports feature 'send_metadata' with feature 'storage_metadata_write_full_object_key'.",
config_prefix);
String object_key_compatibility_prefix = config.getString(config_prefix + ".key_compatibility_prefix", String());
String object_key_template = config.getString(config_prefix + ".key_template", String());
if (object_key_template.empty())
{
if (!object_key_compatibility_prefix.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Wrong configuration in {}. "
"Setting 'key_compatibility_prefix' can be defined only with setting 'key_template'.",
config_prefix);
return {uri.key, createObjectStorageKeysGeneratorByPrefix(uri.key)};
}
if (send_metadata)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Wrong configuration in {}. "
"s3 does not supports send_metadata with setting 'key_template'.",
config_prefix);
if (!storage_metadata_write_full_object_key)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Wrong configuration in {}. "
"Feature 'storage_metadata_write_full_object_key' has to be enabled in order to use setting 'key_template'.",
config_prefix);
if (!uri.key.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Wrong configuration in {}. "
"URI.key is forbidden with settings 'key_template', use setting 'key_compatibility_prefix' instead'. "
"URI.key: '{}', bucket: '{}'. ",
config_prefix,
uri.key, uri.bucket);
return {object_key_compatibility_prefix, createObjectStorageKeysGeneratorByTemplate(object_key_template)};
}
}
void registerDiskS3(DiskFactory & factory, bool global_skip_access_check)
@ -104,7 +158,8 @@ void registerDiskS3(DiskFactory & factory, bool global_skip_access_check)
{
String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint"));
S3::URI uri(endpoint);
if (!uri.key.ends_with('/'))
// an empty key remains empty
if (!uri.key.empty() && !uri.key.ends_with('/'))
uri.key.push_back('/');
S3Capabilities s3_capabilities = getCapabilitiesFromConfig(config, config_prefix);
@ -113,6 +168,8 @@ void registerDiskS3(DiskFactory & factory, bool global_skip_access_check)
String type = config.getString(config_prefix + ".type");
chassert(type == "s3" || type == "s3_plain");
auto [object_key_compatibility_prefix, object_key_generator] = getPrefixAndKeyGenerator(type, uri, config, config_prefix);
MetadataStoragePtr metadata_storage;
auto settings = getSettings(config, config_prefix, context);
auto client = getClient(config, config_prefix, context, *settings);
@ -128,20 +185,18 @@ void registerDiskS3(DiskFactory & factory, bool global_skip_access_check)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "s3_plain does not supports send_metadata");
s3_storage = std::make_shared<S3PlainObjectStorage>(
std::move(client), std::move(settings),
uri.version_id, s3_capabilities,
uri.bucket, uri.endpoint, uri.key, name);
metadata_storage = std::make_shared<MetadataStorageFromPlainObjectStorage>(s3_storage, uri.key);
std::move(client), std::move(settings), uri.version_id, s3_capabilities, uri.bucket, uri.endpoint, object_key_generator, name);
metadata_storage = std::make_shared<MetadataStorageFromPlainObjectStorage>(s3_storage, object_key_compatibility_prefix);
}
else
{
s3_storage = std::make_shared<S3ObjectStorage>(
std::move(client), std::move(settings),
uri.version_id, s3_capabilities,
uri.bucket, uri.endpoint, uri.key, name);
std::move(client), std::move(settings), uri.version_id, s3_capabilities, uri.bucket, uri.endpoint, object_key_generator, name);
auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context);
metadata_storage = std::make_shared<MetadataStorageFromDisk>(metadata_disk, uri.key);
metadata_storage = std::make_shared<MetadataStorageFromDisk>(metadata_disk, object_key_compatibility_prefix);
}
/// NOTE: should we still perform this check for clickhouse-disks?
@ -164,7 +219,7 @@ void registerDiskS3(DiskFactory & factory, bool global_skip_access_check)
DiskObjectStoragePtr s3disk = std::make_shared<DiskObjectStorage>(
name,
uri.key,
uri.key, /// might be empty
type == "s3" ? "DiskS3" : "DiskS3Plain",
std::move(metadata_storage),
std::move(s3_storage),

View File

@ -8,6 +8,8 @@
#include <Common/UTF8Helpers.h>
#include <Common/iota.h>
#include <numeric>
#ifdef __SSE4_2__
# include <nmmintrin.h>
#endif
@ -25,7 +27,7 @@ struct FunctionStringDistanceImpl
{
using ResultType = typename Op::ResultType;
static void constantConstant(const std::string & haystack, const std::string & needle, ResultType & res)
static void constantConstant(const String & haystack, const String & needle, ResultType & res)
{
res = Op::process(haystack.data(), haystack.size(), needle.data(), needle.size());
}
@ -51,7 +53,7 @@ struct FunctionStringDistanceImpl
}
static void constantVector(
const std::string & haystack,
const String & haystack,
const ColumnString::Chars & needle_data,
const ColumnString::Offsets & needle_offsets,
PaddedPODArray<ResultType> & res)
@ -70,7 +72,7 @@ struct FunctionStringDistanceImpl
static void vectorConstant(
const ColumnString::Chars & data,
const ColumnString::Offsets & offsets,
const std::string & needle,
const String & needle,
PaddedPODArray<ResultType> & res)
{
constantVector(needle, data, offsets, res);
@ -81,7 +83,7 @@ struct FunctionStringDistanceImpl
struct ByteHammingDistanceImpl
{
using ResultType = UInt64;
static ResultType inline process(
static ResultType process(
const char * __restrict haystack, size_t haystack_size, const char * __restrict needle, size_t needle_size)
{
UInt64 res = 0;
@ -115,7 +117,7 @@ template <bool is_utf8>
struct ByteJaccardIndexImpl
{
using ResultType = Float64;
static ResultType inline process(
static ResultType process(
const char * __restrict haystack, size_t haystack_size, const char * __restrict needle, size_t needle_size)
{
if (haystack_size == 0 || needle_size == 0)
@ -222,23 +224,23 @@ struct ByteJaccardIndexImpl
}
};
static constexpr size_t max_string_size = 1u << 16;
struct ByteEditDistanceImpl
{
using ResultType = UInt64;
static constexpr size_t max_string_size = 1u << 16;
static ResultType inline process(
static ResultType process(
const char * __restrict haystack, size_t haystack_size, const char * __restrict needle, size_t needle_size)
{
if (haystack_size == 0 || needle_size == 0)
return haystack_size + needle_size;
/// Safety threshold against DoS, since we use two array to calculate the distance.
/// Safety threshold against DoS, since we use two arrays to calculate the distance.
if (haystack_size > max_string_size || needle_size > max_string_size)
throw Exception(
ErrorCodes::TOO_LARGE_STRING_SIZE,
"The string size is too big for function editDistance, "
"should be at most {}", max_string_size);
"The string size is too big for function editDistance, should be at most {}", max_string_size);
PaddedPODArray<ResultType> distances0(haystack_size + 1, 0);
PaddedPODArray<ResultType> distances1(haystack_size + 1, 0);
@ -271,6 +273,180 @@ struct ByteEditDistanceImpl
}
};
struct ByteDamerauLevenshteinDistanceImpl
{
using ResultType = UInt64;
static ResultType process(
const char * __restrict haystack, size_t haystack_size, const char * __restrict needle, size_t needle_size)
{
/// Safety threshold against DoS
if (haystack_size > max_string_size || needle_size > max_string_size)
throw Exception(
ErrorCodes::TOO_LARGE_STRING_SIZE,
"The string size is too big for function damerauLevenshteinDistance, should be at most {}", max_string_size);
/// Shortcuts:
if (haystack_size == 0)
return needle_size;
if (needle_size == 0)
return haystack_size;
if (haystack_size == needle_size && memcmp(haystack, needle, haystack_size) == 0)
return 0;
/// Implements the algorithm for optimal string alignment distance from
/// https://en.wikipedia.org/wiki/Damerau%E2%80%93Levenshtein_distance#Optimal_string_alignment_distance
/// Dynamically allocate memory for the 2D array
/// Allocating a 2D array, for convenience starts is an array of pointers to the start of the rows.
std::vector<int> d((needle_size + 1) * (haystack_size + 1));
std::vector<int *> starts(haystack_size + 1);
/// Setting the pointers in starts to the beginning of (needle_size + 1)-long intervals.
/// Also initialize the row values based on the mentioned algorithm.
for (size_t i = 0; i <= haystack_size; ++i)
{
starts[i] = d.data() + (needle_size + 1) * i;
starts[i][0] = static_cast<int>(i);
}
for (size_t j = 0; j <= needle_size; ++j)
{
starts[0][j] = static_cast<int>(j);
}
for (size_t i = 1; i <= haystack_size; ++i)
{
for (size_t j = 1; j <= needle_size; ++j)
{
int cost = (haystack[i - 1] == needle[j - 1]) ? 0 : 1;
starts[i][j] = std::min(starts[i - 1][j] + 1, /// deletion
std::min(starts[i][j - 1] + 1, /// insertion
starts[i - 1][j - 1] + cost) /// substitution
);
if (i > 1 && j > 1 && haystack[i - 1] == needle[j - 2] && haystack[i - 2] == needle[j - 1])
starts[i][j] = std::min(starts[i][j], starts[i - 2][j - 2] + 1); /// transposition
}
}
return starts[haystack_size][needle_size];
}
};
struct ByteJaroSimilarityImpl
{
using ResultType = Float64;
static ResultType process(
const char * __restrict haystack, size_t haystack_size, const char * __restrict needle, size_t needle_size)
{
/// Safety threshold against DoS
if (haystack_size > max_string_size || needle_size > max_string_size)
throw Exception(
ErrorCodes::TOO_LARGE_STRING_SIZE,
"The string size is too big for function jaroSimilarity, should be at most {}", max_string_size);
/// Shortcuts:
if (haystack_size == 0)
return needle_size;
if (needle_size == 0)
return haystack_size;
if (haystack_size == needle_size && memcmp(haystack, needle, haystack_size) == 0)
return 1.0;
const int s1len = static_cast<int>(haystack_size);
const int s2len = static_cast<int>(needle_size);
/// Window size to search for matches in the other string
const int max_range = std::max(0, std::max(s1len, s2len) / 2 - 1);
std::vector<int> s1_matching(s1len, -1);
std::vector<int> s2_matching(s2len, -1);
/// Calculate matching characters
size_t matching_characters = 0;
for (int i = 0; i < s1len; i++)
{
/// Matching window
const int min_index = std::max(i - max_range, 0);
const int max_index = std::min(i + max_range + 1, s2len);
for (int j = min_index; j < max_index; j++)
{
if (s2_matching[j] == -1 && haystack[i] == needle[j])
{
s1_matching[i] = i;
s2_matching[j] = j;
matching_characters++;
break;
}
}
}
if (matching_characters == 0)
return 0.0;
/// Transpositions (one-way only)
double transpositions = 0.0;
for (size_t i = 0, s1i = 0, s2i = 0; i < matching_characters; i++)
{
while (s1_matching[s1i] == -1)
s1i++;
while (s2_matching[s2i] == -1)
s2i++;
if (haystack[s1i] != needle[s2i])
transpositions += 0.5;
s1i++;
s2i++;
}
double m = static_cast<double>(matching_characters);
double jaro_similarity = 1.0 / 3.0 * (m / static_cast<double>(s1len)
+ m / static_cast<double>(s2len)
+ (m - transpositions) / m);
return jaro_similarity;
}
};
struct ByteJaroWinklerSimilarityImpl
{
using ResultType = Float64;
static ResultType process(
const char * __restrict haystack, size_t haystack_size, const char * __restrict needle, size_t needle_size)
{
static constexpr int max_prefix_length = 4;
static constexpr double scaling_factor = 0.1;
static constexpr double boost_threshold = 0.7;
/// Safety threshold against DoS
if (haystack_size > max_string_size || needle_size > max_string_size)
throw Exception(
ErrorCodes::TOO_LARGE_STRING_SIZE,
"The string size is too big for function jaroWinklerSimilarity, should be at most {}", max_string_size);
const int s1len = static_cast<int>(haystack_size);
const int s2len = static_cast<int>(needle_size);
ResultType jaro_winkler_similarity = ByteJaroSimilarityImpl::process(haystack, haystack_size, needle, needle_size);
if (jaro_winkler_similarity > boost_threshold)
{
const int common_length = std::min(max_prefix_length, std::min(s1len, s2len));
int common_prefix = 0;
while (common_prefix < common_length && haystack[common_prefix] == needle[common_prefix])
common_prefix++;
jaro_winkler_similarity += common_prefix * scaling_factor * (1.0 - jaro_winkler_similarity);
}
return jaro_winkler_similarity;
}
};
struct NameByteHammingDistance
{
static constexpr auto name = "byteHammingDistance";
@ -283,6 +459,12 @@ struct NameEditDistance
};
using FunctionEditDistance = FunctionsStringSimilarity<FunctionStringDistanceImpl<ByteEditDistanceImpl>, NameEditDistance>;
struct NameDamerauLevenshteinDistance
{
static constexpr auto name = "damerauLevenshteinDistance";
};
using FunctionDamerauLevenshteinDistance = FunctionsStringSimilarity<FunctionStringDistanceImpl<ByteDamerauLevenshteinDistanceImpl>, NameDamerauLevenshteinDistance>;
struct NameJaccardIndex
{
static constexpr auto name = "stringJaccardIndex";
@ -295,6 +477,18 @@ struct NameJaccardIndexUTF8
};
using FunctionStringJaccardIndexUTF8 = FunctionsStringSimilarity<FunctionStringDistanceImpl<ByteJaccardIndexImpl<true>>, NameJaccardIndexUTF8>;
struct NameJaroSimilarity
{
static constexpr auto name = "jaroSimilarity";
};
using FunctionJaroSimilarity = FunctionsStringSimilarity<FunctionStringDistanceImpl<ByteJaroSimilarityImpl>, NameJaroSimilarity>;
struct NameJaroWinklerSimilarity
{
static constexpr auto name = "jaroWinklerSimilarity";
};
using FunctionJaroWinklerSimilarity = FunctionsStringSimilarity<FunctionStringDistanceImpl<ByteJaroWinklerSimilarityImpl>, NameJaroWinklerSimilarity>;
REGISTER_FUNCTION(StringDistance)
{
factory.registerFunction<FunctionByteHammingDistance>(
@ -305,9 +499,18 @@ REGISTER_FUNCTION(StringDistance)
FunctionDocumentation{.description = R"(Calculates the edit distance between two byte-strings.)"});
factory.registerAlias("levenshteinDistance", NameEditDistance::name);
factory.registerFunction<FunctionDamerauLevenshteinDistance>(
FunctionDocumentation{.description = R"(Calculates the Damerau-Levenshtein distance two between two byte-string.)"});
factory.registerFunction<FunctionStringJaccardIndex>(
FunctionDocumentation{.description = R"(Calculates the [Jaccard similarity index](https://en.wikipedia.org/wiki/Jaccard_index) between two byte strings.)"});
FunctionDocumentation{.description = R"(Calculates the Jaccard similarity index between two byte strings.)"});
factory.registerFunction<FunctionStringJaccardIndexUTF8>(
FunctionDocumentation{.description = R"(Calculates the [Jaccard similarity index](https://en.wikipedia.org/wiki/Jaccard_index) between two UTF8 strings.)"});
FunctionDocumentation{.description = R"(Calculates the Jaccard similarity index between two UTF8 strings.)"});
factory.registerFunction<FunctionJaroSimilarity>(
FunctionDocumentation{.description = R"(Calculates the Jaro similarity between two byte-string.)"});
factory.registerFunction<FunctionJaroWinklerSimilarity>(
FunctionDocumentation{.description = R"(Calculates the Jaro-Winkler similarity between two byte-string.)"});
}
}

View File

@ -3,6 +3,7 @@
#include <Functions/keyvaluepair/impl/Configuration.h>
#include <base/find_symbols.h>
#include <iterator>
#include <vector>
namespace DB

View File

@ -434,7 +434,7 @@ public:
};
FunctionArgumentDescriptors optional_args{
{optional_argument_names[0], &isNumber<IDataType>, isColumnConst, "const Number"},
{optional_argument_names[0], &isNumber<IDataType>, nullptr, "const Number"},
{optional_argument_names[1], &isNumber<IDataType>, isColumnConst, "const Number"},
{optional_argument_names[2], &isString<IDataType>, isColumnConst, "const String"}
};

View File

@ -143,7 +143,6 @@ public:
* depending on values of conditions.
*/
std::vector<Instruction> instructions;
instructions.reserve(arguments.size() / 2 + 1);
@ -238,7 +237,7 @@ public:
}
const auto & settings = context->getSettingsRef();
const WhichDataType which(result_type);
const WhichDataType which(removeNullable(result_type));
bool execute_multiif_columnar
= settings.allow_execute_multiif_columnar && !contains_short && (which.isInt() || which.isUInt() || which.isFloat());
@ -254,8 +253,12 @@ public:
if (which.is##TYPE()) \
{ \
MutableColumnPtr res = ColumnVector<TYPE>::create(rows); \
executeInstructionsColumnar<TYPE, INDEX>(instructions, rows, res); \
return std::move(res); \
MutableColumnPtr null_map = result_type->isNullable() ? ColumnUInt8::create(rows) : nullptr; \
executeInstructionsColumnar<TYPE, INDEX>(instructions, rows, res, null_map, result_type->isNullable()); \
if (!result_type->isNullable()) \
return std::move(res); \
else \
return ColumnNullable::create(std::move(res), std::move(null_map)); \
}
#define ENUMERATE_NUMERIC_TYPES(M, INDEX) \
@ -295,6 +298,7 @@ public:
}
private:
static void executeInstructions(std::vector<Instruction> & instructions, size_t rows, const MutableColumnPtr & res)
{
for (size_t i = 0; i < rows; ++i)
@ -374,17 +378,59 @@ private:
}
template <typename T, typename S>
static void executeInstructionsColumnar(std::vector<Instruction> & instructions, size_t rows, const MutableColumnPtr & res)
static void executeInstructionsColumnar(std::vector<Instruction> & instructions, size_t rows, const MutableColumnPtr & res, const MutableColumnPtr & null_map, bool nullable)
{
PaddedPODArray<S> inserts(rows, static_cast<S>(instructions.size()));
calculateInserts(instructions, rows, inserts);
PaddedPODArray<T> & res_data = assert_cast<ColumnVector<T> &>(*res).getData();
for (size_t row_i = 0; row_i < rows; ++row_i)
if (!nullable)
{
auto & instruction = instructions[inserts[row_i]];
auto ref = instruction.source->getDataAt(row_i);
res_data[row_i] = *reinterpret_cast<const T*>(ref.data);
for (size_t row_i = 0; row_i < rows; ++row_i)
{
auto & instruction = instructions[inserts[row_i]];
auto ref = instruction.source->getDataAt(row_i);
res_data[row_i] = *reinterpret_cast<const T*>(ref.data);
}
}
else
{
PaddedPODArray<UInt8> & null_map_data = assert_cast<ColumnUInt8 &>(*null_map).getData();
std::vector<const T*> data_cols(instructions.size());
std::vector<const UInt8 *> null_map_cols(instructions.size());
ColumnPtr shared_null_map_col = nullptr;
for (size_t i = 0; i < instructions.size(); ++i)
{
if (instructions[i].source->isNullable())
{
const ColumnNullable * nullable_col;
if (!instructions[i].source_is_constant)
nullable_col = assert_cast<const ColumnNullable *>(instructions[i].source.get());
else
{
const ColumnPtr data_column = assert_cast<const ColumnConst &>(*instructions[i].source).getDataColumnPtr();
nullable_col = assert_cast<const ColumnNullable *>(data_column.get());
}
null_map_cols[i] = assert_cast<const ColumnUInt8 &>(*nullable_col->getNullMapColumnPtr()).getData().data();
data_cols[i] = assert_cast<const ColumnVector<T> &>(*nullable_col->getNestedColumnPtr()).getData().data();
}
else
{
if (!shared_null_map_col)
{
shared_null_map_col = ColumnUInt8::create(rows, 0);
}
null_map_cols[i] = assert_cast<const ColumnUInt8 &>(*shared_null_map_col).getData().data();
data_cols[i] = assert_cast<const ColumnVector<T> &>(*instructions[i].source).getData().data();
}
}
for (size_t row_i = 0; row_i < rows; ++row_i)
{
auto & instruction = instructions[inserts[row_i]];
size_t index = instruction.source_is_constant ? 0 : row_i;
res_data[row_i] = *(data_cols[inserts[row_i]] + index);
null_map_data[row_i] = *(null_map_cols[inserts[row_i]] + index);
}
}
}

View File

@ -0,0 +1,238 @@
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wold-style-cast"
#pragma clang diagnostic ignored "-Wshadow"
#pragma clang diagnostic ignored "-Wimplicit-float-conversion"
#endif
#include <Functions/stl.hpp>
#ifdef __clang__
#pragma clang diagnostic pop
#endif
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_COLUMN;
}
// Decompose time series data based on STL(Seasonal-Trend Decomposition Procedure Based on Loess)
class FunctionSeriesDecomposeSTL : public IFunction
{
public:
static constexpr auto name = "seriesDecomposeSTL";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionSeriesDecomposeSTL>(); }
std::string getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
FunctionArgumentDescriptors args{
{"time_series", &isArray<IDataType>, nullptr, "Array"},
{"period", &isNativeUInt<IDataType>, nullptr, "Unsigned Integer"},
};
validateFunctionArgumentTypes(*this, arguments, args);
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeArray>(std::make_shared<DataTypeFloat32>()));
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override
{
ColumnPtr array_ptr = arguments[0].column;
const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get());
if (!array)
{
const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(arguments[0].column.get());
if (!const_array)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}",
arguments[0].column->getName(), getName());
array_ptr = const_array->convertToFullColumn();
array = assert_cast<const ColumnArray *>(array_ptr.get());
}
const IColumn & src_data = array->getData();
const ColumnArray::Offsets & src_offsets = array->getOffsets();
auto res = ColumnFloat32::create();
auto & res_data = res->getData();
ColumnArray::ColumnOffsets::MutablePtr res_col_offsets = ColumnArray::ColumnOffsets::create();
auto & res_col_offsets_data = res_col_offsets->getData();
auto root_offsets = ColumnArray::ColumnOffsets::create();
auto & root_offsets_data = root_offsets->getData();
ColumnArray::Offset prev_src_offset = 0;
for (size_t i = 0; i < src_offsets.size(); ++i)
{
UInt64 period;
auto period_ptr = arguments[1].column->convertToFullColumnIfConst();
if (checkAndGetColumn<ColumnUInt8>(period_ptr.get())
|| checkAndGetColumn<ColumnUInt16>(period_ptr.get())
|| checkAndGetColumn<ColumnUInt32>(period_ptr.get())
|| checkAndGetColumn<ColumnUInt64>(period_ptr.get()))
period = period_ptr->getUInt(i);
else
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal column {} of second argument of function {}",
arguments[1].column->getName(),
getName());
std::vector<Float32> seasonal;
std::vector<Float32> trend;
std::vector<Float32> residue;
ColumnArray::Offset curr_offset = src_offsets[i];
if (executeNumber<UInt8>(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue)
|| executeNumber<UInt16>(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue)
|| executeNumber<UInt32>(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue)
|| executeNumber<UInt64>(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue)
|| executeNumber<Int8>(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue)
|| executeNumber<Int16>(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue)
|| executeNumber<Int32>(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue)
|| executeNumber<Int64>(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue)
|| executeNumber<Float32>(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue)
|| executeNumber<Float64>(src_data, period, prev_src_offset, curr_offset, seasonal, trend, residue))
{
res_data.insert(seasonal.begin(), seasonal.end());
res_col_offsets_data.push_back(res_data.size());
res_data.insert(trend.begin(), trend.end());
res_col_offsets_data.push_back(res_data.size());
res_data.insert(residue.begin(), residue.end());
res_col_offsets_data.push_back(res_data.size());
root_offsets_data.push_back(res_col_offsets->size());
prev_src_offset = curr_offset;
}
else
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal column {} of first argument of function {}",
arguments[0].column->getName(),
getName());
}
ColumnArray::MutablePtr nested_array_col = ColumnArray::create(std::move(res), std::move(res_col_offsets));
return ColumnArray::create(std::move(nested_array_col), std::move(root_offsets));
}
template <typename T>
bool executeNumber(
const IColumn & src_data,
UInt64 period,
ColumnArray::Offset start,
ColumnArray::Offset end,
std::vector<Float32> & seasonal,
std::vector<Float32> & trend,
std::vector<Float32> & residue) const
{
const ColumnVector<T> * src_data_concrete = checkAndGetColumn<ColumnVector<T>>(&src_data);
if (!src_data_concrete)
return false;
const PaddedPODArray<T> & src_vec = src_data_concrete->getData();
chassert(start <= end);
size_t len = end - start;
if (len < 4)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "At least four data points are needed for function {}", getName());
if (period > (len / 2))
throw Exception(
ErrorCodes::BAD_ARGUMENTS, "The series should have data of at least two period lengths for function {}", getName());
std::vector<float> src(src_vec.begin() + start, src_vec.begin() + end);
auto res = stl::params().fit(src, period);
if (res.seasonal.empty())
return false;
seasonal = std::move(res.seasonal);
trend = std::move(res.trend);
residue = std::move(res.remainder);
return true;
}
};
REGISTER_FUNCTION(seriesDecomposeSTL)
{
factory.registerFunction<FunctionSeriesDecomposeSTL>(FunctionDocumentation{
.description = R"(
Decomposes a time series using STL [(Seasonal-Trend Decomposition Procedure Based on Loess)](https://www.wessa.net/download/stl.pdf) into a season, a trend and a residual component.
**Syntax**
``` sql
seriesDecomposeSTL(series, period);
```
**Arguments**
- `series` - An array of numeric values
- `period` - A positive number
The number of data points in `series` should be at least twice the value of `period`.
**Returned value**
- An array of three arrays where the first array include seasonal components, the second array - trend, and the third array - residue component.
Type: [Array](../../sql-reference/data-types/array.md).
**Examples**
Query:
``` sql
SELECT seriesDecomposeSTL([10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34, 10.1, 20.45, 40.34], 3) AS print_0;
```
Result:
``` text
print_0
[[
-13.529999, -3.1799996, 16.71, -13.53, -3.1799996, 16.71, -13.53, -3.1799996,
16.71, -13.530001, -3.18, 16.710001, -13.530001, -3.1800003, 16.710001, -13.530001,
-3.1800003, 16.710001, -13.530001, -3.1799994, 16.71, -13.529999, -3.1799994, 16.709997
],
[
23.63, 23.63, 23.630003, 23.630001, 23.630001, 23.630001, 23.630001, 23.630001,
23.630001, 23.630001, 23.630001, 23.63, 23.630001, 23.630001, 23.63, 23.630001,
23.630001, 23.63, 23.630001, 23.630001, 23.630001, 23.630001, 23.630001, 23.630003
],
[
0, 0.0000019073486, -0.0000019073486, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -0.0000019073486, 0,
0
]]
```)",
.categories{"Time series analysis"}});
}
}

513
src/Functions/stl.hpp Normal file
View File

@ -0,0 +1,513 @@
// Dump of https://github.com/ankane/stl-cpp/blob/3b1b3a3e9335cda26c8b0797d8b8d24ac8e350ad/include/stl.hpp.
// Added to ClickHouse source code and not referenced as a submodule because its easier maintain and modify/customize.
/*!
* STL C++ v0.1.3
* https://github.com/ankane/stl-cpp
* Unlicense OR MIT License
*
* Ported from https://www.netlib.org/a/stl
*
* Cleveland, R. B., Cleveland, W. S., McRae, J. E., & Terpenning, I. (1990).
* STL: A Seasonal-Trend Decomposition Procedure Based on Loess.
* Journal of Official Statistics, 6(1), 3-33.
*/
#pragma once
#include <algorithm>
#include <cmath>
#include <numeric>
#include <optional>
#include <stdexcept>
#include <vector>
namespace stl {
bool est(const float* y, size_t n, size_t len, int ideg, float xs, float* ys, size_t nleft, size_t nright, float* w, bool userw, const float* rw) {
auto range = ((float) n) - 1.0;
auto h = std::max(xs - ((float) nleft), ((float) nright) - xs);
if (len > n) {
h += (float) ((len - n) / 2);
}
auto h9 = 0.999 * h;
auto h1 = 0.001 * h;
// compute weights
auto a = 0.0;
for (auto j = nleft; j <= nright; j++) {
w[j - 1] = 0.0;
auto r = fabs(((float) j) - xs);
if (r <= h9) {
if (r <= h1) {
w[j - 1] = 1.0;
} else {
w[j - 1] = pow(1.0 - pow(r / h, 3), 3);
}
if (userw) {
w[j - 1] *= rw[j - 1];
}
a += w[j - 1];
}
}
if (a <= 0.0) {
return false;
} else { // weighted least squares
for (auto j = nleft; j <= nright; j++) { // make sum of w(j) == 1
w[j - 1] /= a;
}
if (h > 0.0 && ideg > 0) { // use linear fit
auto a = 0.0;
for (auto j = nleft; j <= nright; j++) { // weighted center of x values
a += w[j - 1] * ((float) j);
}
auto b = xs - a;
auto c = 0.0;
for (auto j = nleft; j <= nright; j++) {
c += w[j - 1] * pow(((float) j) - a, 2);
}
if (sqrt(c) > 0.001 * range) {
b /= c;
// points are spread out enough to compute slope
for (auto j = nleft; j <= nright; j++) {
w[j - 1] *= b * (((float) j) - a) + 1.0;
}
}
}
*ys = 0.0;
for (auto j = nleft; j <= nright; j++) {
*ys += w[j - 1] * y[j - 1];
}
return true;
}
}
void ess(const float* y, size_t n, size_t len, int ideg, size_t njump, bool userw, const float* rw, float* ys, float* res) {
if (n < 2) {
ys[0] = y[0];
return;
}
size_t nleft = 0;
size_t nright = 0;
auto newnj = std::min(njump, n - 1);
if (len >= n) {
nleft = 1;
nright = n;
for (size_t i = 1; i <= n; i += newnj) {
auto ok = est(y, n, len, ideg, (float) i, &ys[i - 1], nleft, nright, res, userw, rw);
if (!ok) {
ys[i - 1] = y[i - 1];
}
}
} else if (newnj == 1) { // newnj equal to one, len less than n
auto nsh = (len + 1) / 2;
nleft = 1;
nright = len;
for (size_t i = 1; i <= n; i++) { // fitted value at i
if (i > nsh && nright != n) {
nleft += 1;
nright += 1;
}
auto ok = est(y, n, len, ideg, (float) i, &ys[i - 1], nleft, nright, res, userw, rw);
if (!ok) {
ys[i - 1] = y[i - 1];
}
}
} else { // newnj greater than one, len less than n
auto nsh = (len + 1) / 2;
for (size_t i = 1; i <= n; i += newnj) { // fitted value at i
if (i < nsh) {
nleft = 1;
nright = len;
} else if (i >= n - nsh + 1) {
nleft = n - len + 1;
nright = n;
} else {
nleft = i - nsh + 1;
nright = len + i - nsh;
}
auto ok = est(y, n, len, ideg, (float) i, &ys[i - 1], nleft, nright, res, userw, rw);
if (!ok) {
ys[i - 1] = y[i - 1];
}
}
}
if (newnj != 1) {
for (size_t i = 1; i <= n - newnj; i += newnj) {
auto delta = (ys[i + newnj - 1] - ys[i - 1]) / ((float) newnj);
for (auto j = i + 1; j <= i + newnj - 1; j++) {
ys[j - 1] = ys[i - 1] + delta * ((float) (j - i));
}
}
auto k = ((n - 1) / newnj) * newnj + 1;
if (k != n) {
auto ok = est(y, n, len, ideg, (float) n, &ys[n - 1], nleft, nright, res, userw, rw);
if (!ok) {
ys[n - 1] = y[n - 1];
}
if (k != n - 1) {
auto delta = (ys[n - 1] - ys[k - 1]) / ((float) (n - k));
for (auto j = k + 1; j <= n - 1; j++) {
ys[j - 1] = ys[k - 1] + delta * ((float) (j - k));
}
}
}
}
}
void ma(const float* x, size_t n, size_t len, float* ave) {
auto newn = n - len + 1;
auto flen = (float) len;
auto v = 0.0;
// get the first average
for (size_t i = 0; i < len; i++) {
v += x[i];
}
ave[0] = v / flen;
if (newn > 1) {
auto k = len;
auto m = 0;
for (size_t j = 1; j < newn; j++) {
// window down the array
v = v - x[m] + x[k];
ave[j] = v / flen;
k += 1;
m += 1;
}
}
}
void fts(const float* x, size_t n, size_t np, float* trend, float* work) {
ma(x, n, np, trend);
ma(trend, n - np + 1, np, work);
ma(work, n - 2 * np + 2, 3, trend);
}
void rwts(const float* y, size_t n, const float* fit, float* rw) {
for (size_t i = 0; i < n; i++) {
rw[i] = fabs(y[i] - fit[i]);
}
auto mid1 = (n - 1) / 2;
auto mid2 = n / 2;
// sort
std::sort(rw, rw + n);
auto cmad = 3.0 * (rw[mid1] + rw[mid2]); // 6 * median abs resid
auto c9 = 0.999 * cmad;
auto c1 = 0.001 * cmad;
for (size_t i = 0; i < n; i++) {
auto r = fabs(y[i] - fit[i]);
if (r <= c1) {
rw[i] = 1.0;
} else if (r <= c9) {
rw[i] = pow(1.0 - pow(r / cmad, 2), 2);
} else {
rw[i] = 0.0;
}
}
}
void ss(const float* y, size_t n, size_t np, size_t ns, int isdeg, size_t nsjump, bool userw, float* rw, float* season, float* work1, float* work2, float* work3, float* work4) {
for (size_t j = 1; j <= np; j++) {
size_t k = (n - j) / np + 1;
for (size_t i = 1; i <= k; i++) {
work1[i - 1] = y[(i - 1) * np + j - 1];
}
if (userw) {
for (size_t i = 1; i <= k; i++) {
work3[i - 1] = rw[(i - 1) * np + j - 1];
}
}
ess(work1, k, ns, isdeg, nsjump, userw, work3, work2 + 1, work4);
auto xs = 0.0;
auto nright = std::min(ns, k);
auto ok = est(work1, k, ns, isdeg, xs, &work2[0], 1, nright, work4, userw, work3);
if (!ok) {
work2[0] = work2[1];
}
xs = k + 1;
size_t nleft = std::max(1, (int) k - (int) ns + 1);
ok = est(work1, k, ns, isdeg, xs, &work2[k + 1], nleft, k, work4, userw, work3);
if (!ok) {
work2[k + 1] = work2[k];
}
for (size_t m = 1; m <= k + 2; m++) {
season[(m - 1) * np + j - 1] = work2[m - 1];
}
}
}
void onestp(const float* y, size_t n, size_t np, size_t ns, size_t nt, size_t nl, int isdeg, int itdeg, int ildeg, size_t nsjump, size_t ntjump, size_t nljump, size_t ni, bool userw, float* rw, float* season, float* trend, float* work1, float* work2, float* work3, float* work4, float* work5) {
for (size_t j = 0; j < ni; j++) {
for (size_t i = 0; i < n; i++) {
work1[i] = y[i] - trend[i];
}
ss(work1, n, np, ns, isdeg, nsjump, userw, rw, work2, work3, work4, work5, season);
fts(work2, n + 2 * np, np, work3, work1);
ess(work3, n, nl, ildeg, nljump, false, work4, work1, work5);
for (size_t i = 0; i < n; i++) {
season[i] = work2[np + i] - work1[i];
}
for (size_t i = 0; i < n; i++) {
work1[i] = y[i] - season[i];
}
ess(work1, n, nt, itdeg, ntjump, userw, rw, trend, work3);
}
}
void stl(const float* y, size_t n, size_t np, size_t ns, size_t nt, size_t nl, int isdeg, int itdeg, int ildeg, size_t nsjump, size_t ntjump, size_t nljump, size_t ni, size_t no, float* rw, float* season, float* trend) {
if (ns < 3) {
throw std::invalid_argument("seasonal_length must be at least 3");
}
if (nt < 3) {
throw std::invalid_argument("trend_length must be at least 3");
}
if (nl < 3) {
throw std::invalid_argument("low_pass_length must be at least 3");
}
if (np < 2) {
throw std::invalid_argument("period must be at least 2");
}
if (isdeg != 0 && isdeg != 1) {
throw std::invalid_argument("seasonal_degree must be 0 or 1");
}
if (itdeg != 0 && itdeg != 1) {
throw std::invalid_argument("trend_degree must be 0 or 1");
}
if (ildeg != 0 && ildeg != 1) {
throw std::invalid_argument("low_pass_degree must be 0 or 1");
}
if (ns % 2 != 1) {
throw std::invalid_argument("seasonal_length must be odd");
}
if (nt % 2 != 1) {
throw std::invalid_argument("trend_length must be odd");
}
if (nl % 2 != 1) {
throw std::invalid_argument("low_pass_length must be odd");
}
auto work1 = std::vector<float>(n + 2 * np);
auto work2 = std::vector<float>(n + 2 * np);
auto work3 = std::vector<float>(n + 2 * np);
auto work4 = std::vector<float>(n + 2 * np);
auto work5 = std::vector<float>(n + 2 * np);
auto userw = false;
size_t k = 0;
while (true) {
onestp(y, n, np, ns, nt, nl, isdeg, itdeg, ildeg, nsjump, ntjump, nljump, ni, userw, rw, season, trend, work1.data(), work2.data(), work3.data(), work4.data(), work5.data());
k += 1;
if (k > no) {
break;
}
for (size_t i = 0; i < n; i++) {
work1[i] = trend[i] + season[i];
}
rwts(y, n, work1.data(), rw);
userw = true;
}
if (no <= 0) {
for (size_t i = 0; i < n; i++) {
rw[i] = 1.0;
}
}
}
float var(const std::vector<float>& series) {
auto mean = std::accumulate(series.begin(), series.end(), 0.0) / series.size();
std::vector<float> tmp;
tmp.reserve(series.size());
for (auto v : series) {
tmp.push_back(pow(v - mean, 2));
}
return std::accumulate(tmp.begin(), tmp.end(), 0.0) / (series.size() - 1);
}
float strength(const std::vector<float>& component, const std::vector<float>& remainder) {
std::vector<float> sr;
sr.reserve(remainder.size());
for (size_t i = 0; i < remainder.size(); i++) {
sr.push_back(component[i] + remainder[i]);
}
return std::max(0.0, 1.0 - var(remainder) / var(sr));
}
class StlResult {
public:
std::vector<float> seasonal;
std::vector<float> trend;
std::vector<float> remainder;
std::vector<float> weights;
inline float seasonal_strength() {
return strength(seasonal, remainder);
}
inline float trend_strength() {
return strength(trend, remainder);
}
};
class StlParams {
std::optional<size_t> ns_ = std::nullopt;
std::optional<size_t> nt_ = std::nullopt;
std::optional<size_t> nl_ = std::nullopt;
int isdeg_ = 0;
int itdeg_ = 1;
std::optional<int> ildeg_ = std::nullopt;
std::optional<size_t> nsjump_ = std::nullopt;
std::optional<size_t> ntjump_ = std::nullopt;
std::optional<size_t> nljump_ = std::nullopt;
std::optional<size_t> ni_ = std::nullopt;
std::optional<size_t> no_ = std::nullopt;
bool robust_ = false;
public:
inline StlParams seasonal_length(size_t ns) {
this->ns_ = ns;
return *this;
}
inline StlParams trend_length(size_t nt) {
this->nt_ = nt;
return *this;
}
inline StlParams low_pass_length(size_t nl) {
this->nl_ = nl;
return *this;
}
inline StlParams seasonal_degree(int isdeg) {
this->isdeg_ = isdeg;
return *this;
}
inline StlParams trend_degree(int itdeg) {
this->itdeg_ = itdeg;
return *this;
}
inline StlParams low_pass_degree(int ildeg) {
this->ildeg_ = ildeg;
return *this;
}
inline StlParams seasonal_jump(size_t nsjump) {
this->nsjump_ = nsjump;
return *this;
}
inline StlParams trend_jump(size_t ntjump) {
this->ntjump_ = ntjump;
return *this;
}
inline StlParams low_pass_jump(size_t nljump) {
this->nljump_ = nljump;
return *this;
}
inline StlParams inner_loops(bool ni) {
this->ni_ = ni;
return *this;
}
inline StlParams outer_loops(bool no) {
this->no_ = no;
return *this;
}
inline StlParams robust(bool robust) {
this->robust_ = robust;
return *this;
}
StlResult fit(const float* y, size_t n, size_t np);
StlResult fit(const std::vector<float>& y, size_t np);
};
StlParams params() {
return StlParams();
}
StlResult StlParams::fit(const float* y, size_t n, size_t np) {
if (n < 2 * np) {
throw std::invalid_argument("series has less than two periods");
}
auto ns = this->ns_.value_or(np);
auto isdeg = this->isdeg_;
auto itdeg = this->itdeg_;
auto res = StlResult {
std::vector<float>(n),
std::vector<float>(n),
std::vector<float>(),
std::vector<float>(n)
};
auto ildeg = this->ildeg_.value_or(itdeg);
auto newns = std::max(ns, (size_t) 3);
if (newns % 2 == 0) {
newns += 1;
}
auto newnp = std::max(np, (size_t) 2);
auto nt = (size_t) ceil((1.5 * newnp) / (1.0 - 1.5 / (float) newns));
nt = this->nt_.value_or(nt);
nt = std::max(nt, (size_t) 3);
if (nt % 2 == 0) {
nt += 1;
}
auto nl = this->nl_.value_or(newnp);
if (nl % 2 == 0 && !this->nl_.has_value()) {
nl += 1;
}
auto ni = this->ni_.value_or(this->robust_ ? 1 : 2);
auto no = this->no_.value_or(this->robust_ ? 15 : 0);
auto nsjump = this->nsjump_.value_or((size_t) ceil(((float) newns) / 10.0));
auto ntjump = this->ntjump_.value_or((size_t) ceil(((float) nt) / 10.0));
auto nljump = this->nljump_.value_or((size_t) ceil(((float) nl) / 10.0));
stl(y, n, newnp, newns, nt, nl, isdeg, itdeg, ildeg, nsjump, ntjump, nljump, ni, no, res.weights.data(), res.seasonal.data(), res.trend.data());
res.remainder.reserve(n);
for (size_t i = 0; i < n; i++) {
res.remainder.push_back(y[i] - res.seasonal[i] - res.trend[i]);
}
return res;
}
StlResult StlParams::fit(const std::vector<float>& y, size_t np) {
return StlParams::fit(y.data(), y.size(), np);
}
}

View File

@ -170,7 +170,7 @@ std::unique_ptr<ReadBuffer> wrapReadBufferWithCompressionMethod(
}
std::unique_ptr<WriteBuffer> wrapWriteBufferWithCompressionMethod(
std::unique_ptr<WriteBuffer> nested, CompressionMethod method, int level, size_t buf_size, char * existing_memory, size_t alignment)
std::unique_ptr<WriteBuffer> nested, CompressionMethod method, int level, int zstd_window_log, size_t buf_size, char * existing_memory, size_t alignment)
{
if (method == DB::CompressionMethod::Gzip || method == CompressionMethod::Zlib)
return std::make_unique<ZlibDeflatingWriteBuffer>(std::move(nested), method, level, buf_size, existing_memory, alignment);
@ -183,7 +183,7 @@ std::unique_ptr<WriteBuffer> wrapWriteBufferWithCompressionMethod(
return std::make_unique<LZMADeflatingWriteBuffer>(std::move(nested), level, buf_size, existing_memory, alignment);
if (method == CompressionMethod::Zstd)
return std::make_unique<ZstdDeflatingWriteBuffer>(std::move(nested), level, buf_size, existing_memory, alignment);
return std::make_unique<ZstdDeflatingWriteBuffer>(std::move(nested), level, zstd_window_log, buf_size, existing_memory, alignment);
if (method == CompressionMethod::Lz4)
return std::make_unique<Lz4DeflatingWriteBuffer>(std::move(nested), level, buf_size, existing_memory, alignment);

View File

@ -66,6 +66,7 @@ std::unique_ptr<WriteBuffer> wrapWriteBufferWithCompressionMethod(
std::unique_ptr<WriteBuffer> nested,
CompressionMethod method,
int level,
int zstd_window_log = 0,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr,
size_t alignment = 0);

View File

@ -3,7 +3,6 @@
#if USE_AWS_S3
#include <aws/core/client/CoreErrors.h>
#include <aws/core/client/DefaultRetryStrategy.h>
#include <aws/s3/model/HeadBucketRequest.h>
#include <aws/s3/model/GetObjectRequest.h>
#include <aws/s3/model/HeadObjectRequest.h>
@ -15,7 +14,6 @@
#include <Poco/Net/NetException.h>
#include <IO/S3Common.h>
#include <IO/S3/Requests.h>
#include <IO/S3/PocoHTTPClientFactory.h>
#include <IO/S3/AWSLogger.h>
@ -37,6 +35,9 @@ namespace ProfileEvents
extern const Event DiskS3WriteRequestsErrors;
extern const Event DiskS3ReadRequestsErrors;
extern const Event S3Clients;
extern const Event TinyS3Clients;
}
namespace DB
@ -199,6 +200,8 @@ Client::Client(
cache = std::make_shared<ClientCache>();
ClientCacheRegistry::instance().registerClient(cache);
ProfileEvents::increment(ProfileEvents::S3Clients);
}
Client::Client(
@ -219,6 +222,22 @@ Client::Client(
{
cache = std::make_shared<ClientCache>(*other.cache);
ClientCacheRegistry::instance().registerClient(cache);
ProfileEvents::increment(ProfileEvents::TinyS3Clients);
}
Client::~Client()
{
try
{
ClientCacheRegistry::instance().unregisterClient(cache.get());
}
catch (...)
{
tryLogCurrentException(log);
throw;
}
}
Aws::Auth::AWSCredentials Client::getCredentials() const

View File

@ -142,18 +142,7 @@ public:
Client(Client && other) = delete;
Client & operator=(Client &&) = delete;
~Client() override
{
try
{
ClientCacheRegistry::instance().unregisterClient(cache.get());
}
catch (...)
{
tryLogCurrentException(log);
throw;
}
}
~Client() override;
/// Returns the initial endpoint.
const String & getInitialEndpoint() const { return initial_endpoint; }
@ -170,7 +159,7 @@ public:
class RetryStrategy : public Aws::Client::RetryStrategy
{
public:
RetryStrategy(uint32_t maxRetries_ = 10, uint32_t scaleFactor_ = 25, uint32_t maxDelayMs_ = 90000);
explicit RetryStrategy(uint32_t maxRetries_ = 10, uint32_t scaleFactor_ = 25, uint32_t maxDelayMs_ = 90000);
/// NOLINTNEXTLINE(google-runtime-int)
bool ShouldRetry(const Aws::Client::AWSError<Aws::Client::CoreErrors>& error, long attemptedRetries) const override;

View File

@ -6,21 +6,12 @@
#if USE_AWS_S3
# include <Common/quoteString.h>
# include <IO/WriteBufferFromString.h>
# include <IO/HTTPHeaderEntries.h>
# include <Storages/StorageS3Settings.h>
# include <IO/S3/PocoHTTPClientFactory.h>
# include <IO/S3/PocoHTTPClient.h>
# include <IO/S3/Client.h>
# include <IO/S3/URI.h>
# include <IO/S3/Requests.h>
# include <IO/S3/Credentials.h>
# include <Common/quoteString.h>
# include <Common/logger_useful.h>
# include <fstream>
namespace ProfileEvents
{
@ -147,6 +138,12 @@ AuthSettings AuthSettings::loadFromConfig(const std::string & config_elem, const
};
}
bool AuthSettings::hasUpdates(const AuthSettings & other) const
{
AuthSettings copy = *this;
copy.updateFrom(other);
return *this != copy;
}
void AuthSettings::updateFrom(const AuthSettings & from)
{
@ -175,7 +172,7 @@ void AuthSettings::updateFrom(const AuthSettings & from)
expiration_window_seconds = from.expiration_window_seconds;
if (from.no_sign_request.has_value())
no_sign_request = *from.no_sign_request;
no_sign_request = from.no_sign_request;
}
}

View File

@ -92,9 +92,11 @@ struct AuthSettings
std::optional<uint64_t> expiration_window_seconds;
std::optional<bool> no_sign_request;
bool operator==(const AuthSettings & other) const = default;
bool hasUpdates(const AuthSettings & other) const;
void updateFrom(const AuthSettings & from);
private:
bool operator==(const AuthSettings & other) const = default;
};
}

View File

@ -1,30 +1,51 @@
#include <IO/ZstdDeflatingWriteBuffer.h>
#include <Common/Exception.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ZSTD_ENCODER_FAILED;
extern const int ILLEGAL_CODEC_PARAMETER;
}
static void setZstdParameter(ZSTD_CCtx * cctx, ZSTD_cParameter param, int value)
{
auto ret = ZSTD_CCtx_setParameter(cctx, param, value);
if (ZSTD_isError(ret))
throw Exception(
ErrorCodes::ZSTD_ENCODER_FAILED,
"zstd stream encoder option setting failed: error code: {}; zstd version: {}",
ret,
ZSTD_VERSION_STRING);
}
ZstdDeflatingWriteBuffer::ZstdDeflatingWriteBuffer(
std::unique_ptr<WriteBuffer> out_, int compression_level, size_t buf_size, char * existing_memory, size_t alignment)
std::unique_ptr<WriteBuffer> out_, int compression_level, int window_log, size_t buf_size, char * existing_memory, size_t alignment)
: WriteBufferWithOwnMemoryDecorator(std::move(out_), buf_size, existing_memory, alignment)
{
cctx = ZSTD_createCCtx();
if (cctx == nullptr)
throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED, "zstd stream encoder init failed: zstd version: {}", ZSTD_VERSION_STRING);
size_t ret = ZSTD_CCtx_setParameter(cctx, ZSTD_c_compressionLevel, compression_level);
if (ZSTD_isError(ret))
throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED,
"zstd stream encoder option setting failed: error code: {}; zstd version: {}",
ret, ZSTD_VERSION_STRING);
ret = ZSTD_CCtx_setParameter(cctx, ZSTD_c_checksumFlag, 1);
if (ZSTD_isError(ret))
throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED,
"zstd stream encoder option setting failed: error code: {}; zstd version: {}",
ret, ZSTD_VERSION_STRING);
setZstdParameter(cctx, ZSTD_c_compressionLevel, compression_level);
if (window_log > 0)
{
ZSTD_bounds window_log_bounds = ZSTD_cParam_getBounds(ZSTD_c_windowLog);
if (ZSTD_isError(window_log_bounds.error))
throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER, "ZSTD windowLog parameter is not supported {}",
std::string(ZSTD_getErrorName(window_log_bounds.error)));
if (window_log > window_log_bounds.upperBound || window_log < window_log_bounds.lowerBound)
throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER,
"ZSTD codec can't have window log more than {} and lower than {}, given {}",
toString(window_log_bounds.upperBound),
toString(window_log_bounds.lowerBound), toString(window_log));
setZstdParameter(cctx, ZSTD_c_enableLongDistanceMatching, 1);
setZstdParameter(cctx, ZSTD_c_windowLog, window_log);
}
setZstdParameter(cctx, ZSTD_c_checksumFlag, 1);
input = {nullptr, 0, 0};
output = {nullptr, 0, 0};

View File

@ -17,6 +17,7 @@ public:
ZstdDeflatingWriteBuffer(
std::unique_ptr<WriteBuffer> out_,
int compression_level,
int window_log = 0,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr,
size_t alignment = 0);

View File

@ -1,8 +1,9 @@
#pragma once
#include <functional>
#include <string>
#include <Core/Defines.h>
#include <Interpreters/Cache/FileCache_fwd.h>
#include <string>
namespace Poco { namespace Util { class AbstractConfiguration; } } // NOLINT(cppcoreguidelines-virtual-class-destructor)

View File

@ -46,6 +46,9 @@ ConcurrentHashJoin::ConcurrentHashJoin(ContextPtr context_, std::shared_ptr<Tabl
auto inner_hash_join = std::make_shared<InternalHashJoin>();
inner_hash_join->data = std::make_unique<HashJoin>(table_join_, right_sample_block, any_take_last_row_, 0, fmt::format("concurrent{}", i));
/// Non zero `max_joined_block_rows` allows to process block partially and return not processed part.
/// TODO: It's not handled properly in ConcurrentHashJoin case, so we set it to 0 to disable this feature.
inner_hash_join->data->setMaxJoinedBlockRows(0);
hash_joins.emplace_back(std::move(inner_hash_join));
}
}

View File

@ -1583,9 +1583,7 @@ bool Context::hasScalar(const String & name) const
void Context::addQueryAccessInfo(
const String & quoted_database_name,
const String & full_quoted_table_name,
const Names & column_names,
const String & projection_name,
const String & view_name)
const Names & column_names)
{
if (isGlobalContext())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info");
@ -1593,12 +1591,9 @@ void Context::addQueryAccessInfo(
std::lock_guard lock(query_access_info.mutex);
query_access_info.databases.emplace(quoted_database_name);
query_access_info.tables.emplace(full_quoted_table_name);
for (const auto & column_name : column_names)
query_access_info.columns.emplace(full_quoted_table_name + "." + backQuoteIfNeed(column_name));
if (!projection_name.empty())
query_access_info.projections.emplace(full_quoted_table_name + "." + backQuoteIfNeed(projection_name));
if (!view_name.empty())
query_access_info.views.emplace(view_name);
}
void Context::addQueryAccessInfo(const Names & partition_names)
@ -1611,6 +1606,15 @@ void Context::addQueryAccessInfo(const Names & partition_names)
query_access_info.partitions.emplace(partition_name);
}
void Context::addViewAccessInfo(const String & view_name)
{
if (isGlobalContext())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info");
std::lock_guard<std::mutex> lock(query_access_info.mutex);
query_access_info.views.emplace(view_name);
}
void Context::addQueryAccessInfo(const QualifiedProjectionName & qualified_projection_name)
{
if (!qualified_projection_name)

View File

@ -693,13 +693,14 @@ public:
void addSpecialScalar(const String & name, const Block & block);
const QueryAccessInfo & getQueryAccessInfo() const { return query_access_info; }
void addQueryAccessInfo(
const String & quoted_database_name,
const String & full_quoted_table_name,
const Names & column_names,
const String & projection_name = {},
const String & view_name = {});
const Names & column_names);
void addQueryAccessInfo(const Names & partition_names);
void addViewAccessInfo(const String & view_name);
struct QualifiedProjectionName
{
@ -707,8 +708,8 @@ public:
String projection_name;
explicit operator bool() const { return !projection_name.empty(); }
};
void addQueryAccessInfo(const QualifiedProjectionName & qualified_projection_name);
void addQueryAccessInfo(const QualifiedProjectionName & qualified_projection_name);
/// Supported factories for records in query_log
enum class QueryLogFactories

View File

@ -1050,7 +1050,7 @@ static std::unique_ptr<QueryPlan> buildJoinedPlan(
join_element.table_expression,
context,
original_right_column_names,
query_options.copy().setWithAllColumns().ignoreProjections(false).ignoreAlias(false));
query_options.copy().setWithAllColumns().ignoreAlias(false));
auto joined_plan = std::make_unique<QueryPlan>();
interpreter->buildQueryPlan(*joined_plan);
{

View File

@ -243,6 +243,7 @@ HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_s
, asof_inequality(table_join->getAsofInequality())
, data(std::make_shared<RightTableData>())
, right_sample_block(right_sample_block_)
, max_joined_block_rows(table_join->maxJoinedBlockRows())
, instance_log_id(!instance_id_.empty() ? "(" + instance_id_ + ") " : "")
, log(&Poco::Logger::get("HashJoin"))
{
@ -1401,7 +1402,7 @@ NO_INLINE size_t joinRightColumns(
{
if constexpr (join_features.need_replication)
{
if (unlikely(current_offset > max_joined_block_rows))
if (unlikely(current_offset >= max_joined_block_rows))
{
added_columns.offsets_to_replicate->resize_assume_reserved(i);
added_columns.filter.resize_assume_reserved(i);
@ -1690,7 +1691,7 @@ Block HashJoin::joinBlockImpl(
bool has_required_right_keys = (required_right_keys.columns() != 0);
added_columns.need_filter = join_features.need_filter || has_required_right_keys;
added_columns.max_joined_block_rows = table_join->maxJoinedBlockRows();
added_columns.max_joined_block_rows = max_joined_block_rows;
if (!added_columns.max_joined_block_rows)
added_columns.max_joined_block_rows = std::numeric_limits<size_t>::max();
else
@ -1771,7 +1772,6 @@ Block HashJoin::joinBlockImpl(
void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const
{
size_t max_joined_block_rows = table_join->maxJoinedBlockRows();
size_t start_left_row = 0;
size_t start_right_block = 0;
if (not_processed)

View File

@ -396,6 +396,8 @@ public:
void shrinkStoredBlocksToFit(size_t & total_bytes_in_join);
void setMaxJoinedBlockRows(size_t value) { max_joined_block_rows = value; }
private:
template<bool> friend class NotJoinedHash;
@ -433,6 +435,9 @@ private:
/// Left table column names that are sources for required_right_keys columns
std::vector<String> required_right_keys_sources;
/// Maximum number of rows in result block. If it is 0, then no limits.
size_t max_joined_block_rows = 0;
/// When tracked memory consumption is more than a threshold, we will shrink to fit stored blocks.
bool shrink_blocks = false;
Int64 memory_usage_before_adding_blocks = 0;

View File

@ -390,8 +390,6 @@ InterpreterSelectQuery::InterpreterSelectQuery(
if (!prepared_sets)
prepared_sets = std::make_shared<PreparedSets>();
query_info.ignore_projections = options.ignore_projections;
query_info.is_projection_query = options.is_projection_query;
query_info.is_internal = options.is_internal;
initSettings();
@ -417,7 +415,6 @@ InterpreterSelectQuery::InterpreterSelectQuery(
}
query_info.query = query_ptr->clone();
query_info.original_query = query_ptr->clone();
if (settings.count_distinct_optimization)
{
@ -856,9 +853,6 @@ InterpreterSelectQuery::InterpreterSelectQuery(
analysis_result.required_columns = required_columns;
}
if (query_info.projection)
storage_snapshot->addProjection(query_info.projection->desc);
/// Blocks used in expression analysis contains size 1 const columns for constant folding and
/// null non-const columns to avoid useless memory allocations. However, a valid block sample
/// requires all columns to be of size 0, thus we need to sanitize the block here.
@ -965,10 +959,7 @@ void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan)
executeImpl(query_plan, std::move(input_pipe));
/// We must guarantee that result structure is the same as in getSampleBlock()
///
/// But if it's a projection query, plan header does not match result_header.
/// TODO: add special stage for InterpreterSelectQuery?
if (!options.is_projection_query && !blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, result_header))
if (!blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, result_header))
{
auto convert_actions_dag = ActionsDAG::makeConvertingActions(
query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(),
@ -1476,12 +1467,6 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
if (expressions.hasHaving() && query.group_by_with_totals && (query.group_by_with_rollup || query.group_by_with_cube))
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING");
if (query_info.projection && query_info.projection->desc->type == ProjectionDescription::Type::Aggregate)
{
query_info.projection->aggregate_overflow_row = aggregate_overflow_row;
query_info.projection->aggregate_final = aggregate_final;
}
if (options.only_analyze)
{
auto read_nothing = std::make_unique<ReadNothingStep>(source_header);
@ -1550,11 +1535,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
LOG_TRACE(log, "{} -> {}", QueryProcessingStage::toString(from_stage), QueryProcessingStage::toString(options.to_stage));
}
if (query_info.projection && query_info.projection->input_order_info && query_info.input_order_info)
throw Exception(ErrorCodes::LOGICAL_ERROR, "InputOrderInfo is set for projection and for query");
InputOrderInfoPtr input_order_info_for_order;
if (!expressions.need_aggregate)
input_order_info_for_order = query_info.projection ? query_info.projection->input_order_info : query_info.input_order_info;
input_order_info_for_order = query_info.input_order_info;
if (options.to_stage > QueryProcessingStage::FetchColumns)
{
@ -1615,7 +1598,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
{
// If there is a storage that supports prewhere, this will always be nullptr
// Thus, we don't actually need to check if projection is active.
if (!query_info.projection && expressions.filter_info)
if (expressions.filter_info)
{
auto row_level_security_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(),
@ -1789,7 +1772,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
}
}
if (!query_info.projection && expressions.hasWhere())
if (expressions.hasWhere())
executeWhere(query_plan, expressions.before_where, expressions.remove_where_filter);
if (expressions.need_aggregate)
@ -2057,15 +2040,13 @@ static void executeMergeAggregatedImpl(
query_plan.addStep(std::move(merging_aggregated));
}
void InterpreterSelectQuery::addEmptySourceToQueryPlan(
QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info, const ContextPtr & context_)
void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info)
{
Pipe pipe(std::make_shared<NullSource>(source_header));
PrewhereInfoPtr prewhere_info_ptr = query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info;
if (prewhere_info_ptr)
if (query_info.prewhere_info)
{
auto & prewhere_info = *prewhere_info_ptr;
auto & prewhere_info = *query_info.prewhere_info;
if (prewhere_info.row_level_filter)
{
@ -2088,50 +2069,6 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(
auto read_from_pipe = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
read_from_pipe->setStepDescription("Read from NullSource");
query_plan.addStep(std::move(read_from_pipe));
if (query_info.projection)
{
if (query_info.projection->before_where)
{
auto where_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(),
query_info.projection->before_where,
query_info.projection->where_column_name,
query_info.projection->remove_where_filter);
where_step->setStepDescription("WHERE");
query_plan.addStep(std::move(where_step));
}
if (query_info.projection->desc->type == ProjectionDescription::Type::Aggregate)
{
if (query_info.projection->before_aggregation)
{
auto expression_before_aggregation
= std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), query_info.projection->before_aggregation);
expression_before_aggregation->setStepDescription("Before GROUP BY");
query_plan.addStep(std::move(expression_before_aggregation));
}
// Let's just choose the safe option since we don't know the value of `to_stage` here.
const bool should_produce_results_in_order_of_bucket_number = true;
// It is used to determine if we should use memory bound merging strategy. Maybe it makes sense for projections, but so far this case is just left untouched.
SortDescription group_by_sort_description;
executeMergeAggregatedImpl(
query_plan,
query_info.projection->aggregate_overflow_row,
query_info.projection->aggregate_final,
false,
false,
context_->getSettingsRef(),
query_info.projection->aggregation_keys,
query_info.projection->aggregate_descriptions,
should_produce_results_in_order_of_bucket_number,
std::move(group_by_sort_description));
}
}
}
RowPolicyFilterPtr InterpreterSelectQuery::getRowPolicyFilter() const
@ -2575,80 +2512,47 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
/// Create optimizer with prepared actions.
/// Maybe we will need to calc input_order_info later, e.g. while reading from StorageMerge.
if ((optimize_read_in_order || optimize_aggregation_in_order)
&& (!query_info.projection || query_info.projection->complete))
if (optimize_read_in_order)
{
if (optimize_read_in_order)
{
if (query_info.projection)
{
query_info.projection->order_optimizer = std::make_shared<ReadInOrderOptimizer>(
// TODO Do we need a projection variant for this field?
query,
analysis_result.order_by_elements_actions,
getSortDescription(query, context),
query_info.syntax_analyzer_result);
}
else
{
query_info.order_optimizer = std::make_shared<ReadInOrderOptimizer>(
query,
analysis_result.order_by_elements_actions,
getSortDescription(query, context),
query_info.syntax_analyzer_result);
}
}
else if (optimize_aggregation_in_order)
{
if (query_info.projection)
{
query_info.projection->order_optimizer = std::make_shared<ReadInOrderOptimizer>(
query,
query_info.projection->group_by_elements_actions,
query_info.projection->group_by_elements_order_descr,
query_info.syntax_analyzer_result);
}
else
{
query_info.order_optimizer = std::make_shared<ReadInOrderOptimizer>(
query,
analysis_result.group_by_elements_actions,
getSortDescriptionFromGroupBy(query),
query_info.syntax_analyzer_result);
}
}
query_info.order_optimizer = std::make_shared<ReadInOrderOptimizer>(
query,
analysis_result.order_by_elements_actions,
getSortDescription(query, context),
query_info.syntax_analyzer_result);
/// If we don't have filtration, we can pushdown limit to reading stage for optimizations.
UInt64 limit = (query.hasFiltration() || query.groupBy()) ? 0 : getLimitForSorting(query, context);
if (query_info.projection)
query_info.projection->input_order_info
= query_info.projection->order_optimizer->getInputOrder(query_info.projection->desc->metadata, context, limit);
else
query_info.input_order_info = query_info.order_optimizer->getInputOrder(metadata_snapshot, context, limit);
UInt64 limit = query.hasFiltration() ? 0 : getLimitForSorting(query, context);
query_info.input_order_info = query_info.order_optimizer->getInputOrder(metadata_snapshot, context, limit);
}
else if (optimize_aggregation_in_order)
{
query_info.order_optimizer = std::make_shared<ReadInOrderOptimizer>(
query,
analysis_result.group_by_elements_actions,
getSortDescriptionFromGroupBy(query),
query_info.syntax_analyzer_result);
query_info.input_order_info = query_info.order_optimizer->getInputOrder(metadata_snapshot, context, /*limit=*/ 0);
}
query_info.storage_limits = std::make_shared<StorageLimitsList>(storage_limits);
query_info.settings_limit_offset_done = options.settings_limit_offset_done;
storage->read(query_plan, required_columns, storage_snapshot, query_info, context, processing_stage, max_block_size, max_streams);
if (context->hasQueryContext() && !options.is_internal)
{
const String view_name{};
auto local_storage_id = storage->getStorageID();
context->getQueryContext()->addQueryAccessInfo(
backQuoteIfNeed(local_storage_id.getDatabaseName()),
local_storage_id.getFullTableName(),
required_columns,
query_info.projection ? query_info.projection->desc->name : "",
view_name);
required_columns);
}
/// Create step which reads from empty source if storage has no data.
if (!query_plan.isInitialized())
{
auto header = storage_snapshot->getSampleBlockForColumns(required_columns);
addEmptySourceToQueryPlan(query_plan, header, query_info, context);
addEmptySourceToQueryPlan(query_plan, header, query_info);
}
}
else
@ -2757,13 +2661,8 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac
expression_before_aggregation->setStepDescription("Before GROUP BY");
query_plan.addStep(std::move(expression_before_aggregation));
if (options.is_projection_query)
return;
AggregateDescriptions aggregates = query_analyzer->aggregates();
const Settings & settings = context->getSettingsRef();
const auto & keys = query_analyzer->aggregationKeys().getNames();
auto aggregator_params = getAggregatorParams(
@ -2827,13 +2726,6 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac
void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final, bool has_grouping_sets)
{
/// If aggregate projection was chosen for table, avoid adding MergeAggregated.
/// It is already added by storage (because of performance issues).
/// TODO: We should probably add another one processing stage for storage?
/// WithMergeableStateAfterAggregation is not ok because, e.g., it skips sorting after aggregation.
if (query_info.projection && query_info.projection->desc->type == ProjectionDescription::Type::Aggregate)
return;
const Settings & settings = context->getSettingsRef();
/// Used to determine if we should use memory bound merging strategy.

View File

@ -117,7 +117,7 @@ public:
bool hasAggregation() const { return query_analyzer->hasAggregation(); }
static void addEmptySourceToQueryPlan(
QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info, const ContextPtr & context_);
QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info);
Names getRequiredColumns() { return required_columns; }

View File

@ -190,7 +190,7 @@ bool isStorageTouchedByMutations(
if (context->getSettingsRef().allow_experimental_analyzer)
{
auto select_query_tree = prepareQueryAffectedQueryTree(commands, storage.shared_from_this(), context);
InterpreterSelectQueryAnalyzer interpreter(select_query_tree, context, SelectQueryOptions().ignoreLimits().ignoreProjections());
InterpreterSelectQueryAnalyzer interpreter(select_query_tree, context, SelectQueryOptions().ignoreLimits());
io = interpreter.execute();
}
else
@ -200,7 +200,7 @@ bool isStorageTouchedByMutations(
/// For some reason it may copy context and give it into ExpressionTransform
/// after that we will use context from destroyed stack frame in our stream.
interpreter_select_query.emplace(
select_query, context, storage_from_part, metadata_snapshot, SelectQueryOptions().ignoreLimits().ignoreProjections());
select_query, context, storage_from_part, metadata_snapshot, SelectQueryOptions().ignoreLimits());
io = interpreter_select_query->execute();
}
@ -404,7 +404,7 @@ MutationsInterpreter::MutationsInterpreter(
, available_columns(std::move(available_columns_))
, context(Context::createCopy(context_))
, settings(std::move(settings_))
, select_limits(SelectQueryOptions().analyze(!settings.can_execute).ignoreLimits().ignoreProjections())
, select_limits(SelectQueryOptions().analyze(!settings.can_execute).ignoreLimits())
{
prepare(!settings.can_execute);
}

View File

@ -33,14 +33,6 @@ struct SelectQueryOptions
bool remove_duplicates = false;
bool ignore_quota = false;
bool ignore_limits = false;
/// This flag is needed to analyze query ignoring table projections.
/// It is needed because we build another one InterpreterSelectQuery while analyzing projections.
/// It helps to avoid infinite recursion.
bool ignore_projections = false;
/// This flag is also used for projection analysis.
/// It is needed because lazy normal projections require special planning in FetchColumns stage, such as adding WHERE transform.
/// It is also used to avoid adding aggregating step when aggregate projection is chosen.
bool is_projection_query = false;
/// This flag is needed for projection description.
/// Otherwise, keys for GROUP BY may be removed as constants.
bool ignore_ast_optimizations = false;
@ -119,18 +111,6 @@ struct SelectQueryOptions
return *this;
}
SelectQueryOptions & ignoreProjections(bool value = true)
{
ignore_projections = value;
return *this;
}
SelectQueryOptions & projectionQuery(bool value = true)
{
is_projection_query = value;
return *this;
}
SelectQueryOptions & ignoreAlias(bool value = true)
{
ignore_alias = value;

View File

@ -1434,11 +1434,12 @@ void executeQuery(
const auto & compression_method_node = ast_query_with_output->compression->as<ASTLiteral &>();
compression_method = compression_method_node.value.safeGet<std::string>();
}
const auto & settings = context->getSettingsRef();
compressed_buffer = wrapWriteBufferWithCompressionMethod(
std::make_unique<WriteBufferFromFile>(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT),
chooseCompressionMethod(out_file, compression_method),
/* compression level = */ 3
/* compression level = */ static_cast<int>(settings.output_format_compression_level),
/* zstd_window_log = */ static_cast<int>(settings.output_format_compression_zstd_window_log)
);
}

View File

@ -846,9 +846,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
query_context->getQueryContext()->addQueryAccessInfo(
backQuoteIfNeed(local_storage_id.getDatabaseName()),
local_storage_id.getFullTableName(),
columns_names,
{},
{});
columns_names);
}
}

View File

@ -455,8 +455,7 @@ QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const NamesAndTyp
SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context)
{
SelectQueryInfo select_query_info;
select_query_info.original_query = queryNodeToSelectQuery(query_tree);
select_query_info.query = select_query_info.original_query;
select_query_info.query = queryNodeToSelectQuery(query_tree);
select_query_info.query_tree = query_tree;
select_query_info.planner_context = planner_context;
return select_query_info;

View File

@ -603,18 +603,21 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(
memcpy(buf, istr.position(), bytes_to_copy);
buf[bytes_to_copy] = 0;
char * pos_double = buf;
/// Skip leading zeroes - we don't want any funny octal business
char * non_zero_buf = find_first_not_symbols<'0'>(buf, buf + bytes_to_copy);
char * pos_double = non_zero_buf;
errno = 0;
Float64 float_value = std::strtod(buf, &pos_double);
if (pos_double == buf || errno == ERANGE || float_value < 0)
Float64 float_value = std::strtod(non_zero_buf, &pos_double);
if (pos_double == non_zero_buf || errno == ERANGE || float_value < 0)
return false;
if (negative)
float_value = -float_value;
char * pos_integer = buf;
char * pos_integer = non_zero_buf;
errno = 0;
UInt64 uint_value = std::strtoull(buf, &pos_integer, 0);
UInt64 uint_value = std::strtoull(non_zero_buf, &pos_integer, 0);
if (pos_integer == pos_double && errno != ERANGE && (!negative || uint_value <= (1ULL << 63)))
{
istr.position() += pos_integer - buf;

View File

@ -448,6 +448,7 @@ PODArray<char> & compress(PODArray<char> & source, PODArray<char> & scratch, Com
std::move(dest_buf),
method,
/*level*/ 3,
/*zstd_window_log*/ 0,
source.size(),
/*existing_memory*/ source.data());
chassert(compressed_buf->position() == source.data());

View File

@ -1080,10 +1080,7 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node,
/// If we don't have filtration, we can pushdown limit to reading stage for optimizations.
UInt64 limit = (select_query->hasFiltration() || select_query->groupBy()) ? 0 : InterpreterSelectQuery::getLimitForSorting(*select_query, context);
auto order_info = order_optimizer->getInputOrder(
query_info.projection ? query_info.projection->desc->metadata : read_from_merge_tree->getStorageMetadata(),
context,
limit);
auto order_info = order_optimizer->getInputOrder(read_from_merge_tree->getStorageMetadata(), context, limit);
if (order_info)
{

View File

@ -642,6 +642,7 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes &
return false;
}
Context::QualifiedProjectionName projection_name;
chassert(best_candidate != nullptr);
QueryPlanStepPtr projection_reading;
@ -654,23 +655,19 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes &
// candidates.minmax_projection->block.dumpStructure());
Pipe pipe(std::make_shared<SourceFromSingleChunk>(std::move(candidates.minmax_projection->block)));
projection_reading = std::make_unique<ReadFromPreparedSource>(
std::move(pipe),
context,
query_info.is_internal
? Context::QualifiedProjectionName{}
: Context::QualifiedProjectionName
{
.storage_id = reading->getMergeTreeData().getStorageID(),
.projection_name = candidates.minmax_projection->candidate.projection->name,
});
projection_reading = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
has_ordinary_parts = false;
projection_name = Context::QualifiedProjectionName
{
.storage_id = reading->getMergeTreeData().getStorageID(),
.projection_name = candidates.minmax_projection->candidate.projection->name,
};
}
else
{
auto storage_snapshot = reading->getStorageSnapshot();
auto proj_snapshot = std::make_shared<StorageSnapshot>(
storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns);
auto proj_snapshot = std::make_shared<StorageSnapshot>(storage_snapshot->storage, storage_snapshot->metadata);
proj_snapshot->addProjection(best_candidate->projection);
auto query_info_copy = query_info;
@ -693,23 +690,29 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes &
{
auto header = proj_snapshot->getSampleBlockForColumns(best_candidate->dag->getRequiredColumnsNames());
Pipe pipe(std::make_shared<NullSource>(std::move(header)));
projection_reading = std::make_unique<ReadFromPreparedSource>(
std::move(pipe),
context,
query_info.is_internal
? Context::QualifiedProjectionName{}
: Context::QualifiedProjectionName
{
.storage_id = reading->getMergeTreeData().getStorageID(),
.projection_name = best_candidate->projection->name,
});
projection_reading = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
}
projection_name = Context::QualifiedProjectionName
{
.storage_id = reading->getMergeTreeData().getStorageID(),
.projection_name = best_candidate->projection->name,
};
has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr;
if (has_ordinary_parts)
reading->setAnalyzedResult(std::move(best_candidate->merge_tree_ordinary_select_result_ptr));
}
if (!query_info.is_internal && context->hasQueryContext())
{
context->getQueryContext()->addQueryAccessInfo(Context::QualifiedProjectionName
{
.storage_id = reading->getMergeTreeData().getStorageID(),
.projection_name = best_candidate->projection->name,
});
}
// LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection reading header {}",
// projection_reading->getOutputStream().header.dumpStructure());

View File

@ -196,8 +196,7 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
}
auto storage_snapshot = reading->getStorageSnapshot();
auto proj_snapshot = std::make_shared<StorageSnapshot>(
storage_snapshot->storage, storage_snapshot->metadata, storage_snapshot->object_columns); //, storage_snapshot->data);
auto proj_snapshot = std::make_shared<StorageSnapshot>(storage_snapshot->storage, storage_snapshot->metadata);
proj_snapshot->addProjection(best_candidate->projection);
auto query_info_copy = query_info;
@ -219,16 +218,16 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
if (!projection_reading)
{
Pipe pipe(std::make_shared<NullSource>(proj_snapshot->getSampleBlockForColumns(required_columns)));
projection_reading = std::make_unique<ReadFromPreparedSource>(
std::move(pipe),
context,
query_info.is_internal
? Context::QualifiedProjectionName{}
: Context::QualifiedProjectionName
{
.storage_id = reading->getMergeTreeData().getStorageID(),
.projection_name = best_candidate->projection->name,
});
projection_reading = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
}
if (!query_info.is_internal && context->hasQueryContext())
{
context->getQueryContext()->addQueryAccessInfo(Context::QualifiedProjectionName
{
.storage_id = reading->getMergeTreeData().getStorageID(),
.projection_name = best_candidate->projection->name,
});
}
bool has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr;

View File

@ -124,12 +124,6 @@ static MergeTreeReaderSettings getMergeTreeReaderSettings(
};
}
static const PrewhereInfoPtr & getPrewhereInfoFromQueryInfo(const SelectQueryInfo & query_info)
{
return query_info.projection ? query_info.projection->prewhere_info
: query_info.prewhere_info;
}
static bool checkAllPartsOnRemoteFS(const RangesInDataParts & parts)
{
for (const auto & part : parts)
@ -256,7 +250,7 @@ ReadFromMergeTree::ReadFromMergeTree(
bool enable_parallel_reading)
: SourceStepWithFilter(DataStream{.header = MergeTreeSelectProcessor::transformHeader(
storage_snapshot_->getSampleBlockForColumns(real_column_names_),
getPrewhereInfoFromQueryInfo(query_info_),
query_info_.prewhere_info,
data_.getPartitionValueType(),
virt_column_names_)})
, reader_settings(getMergeTreeReaderSettings(context_, query_info_))
@ -266,7 +260,7 @@ ReadFromMergeTree::ReadFromMergeTree(
, virt_column_names(std::move(virt_column_names_))
, data(data_)
, query_info(query_info_)
, prewhere_info(getPrewhereInfoFromQueryInfo(query_info))
, prewhere_info(query_info_.prewhere_info)
, actions_settings(ExpressionActionsSettings::fromContext(context_))
, storage_snapshot(std::move(storage_snapshot_))
, metadata_for_reading(storage_snapshot->getMetadataForQuery())
@ -321,7 +315,7 @@ ReadFromMergeTree::ReadFromMergeTree(
*output_stream,
storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(),
getSortDirection(),
query_info.getInputOrderInfo(),
query_info.input_order_info,
prewhere_info);
}
@ -1632,10 +1626,10 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
result.total_marks_pk = total_marks_pk;
result.selected_rows = sum_rows;
const auto & input_order_info = query_info.getInputOrderInfo();
if (input_order_info)
result.read_type = (input_order_info->direction > 0) ? ReadType::InOrder
: ReadType::InReverseOrder;
if (query_info.input_order_info)
result.read_type = (query_info.input_order_info->direction > 0)
? ReadType::InOrder
: ReadType::InReverseOrder;
return std::make_shared<MergeTreeDataSelectAnalysisResult>(MergeTreeDataSelectAnalysisResult{.result = std::move(result)});
}
@ -1651,12 +1645,7 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction,
if (direction != 1 && query_info.isFinal())
return false;
auto order_info = std::make_shared<InputOrderInfo>(SortDescription{}, prefix_size, direction, limit);
if (query_info.projection)
query_info.projection->input_order_info = order_info;
else
query_info.input_order_info = order_info;
query_info.input_order_info = std::make_shared<InputOrderInfo>(SortDescription{}, prefix_size, direction, limit);
reader_settings.read_in_order = true;
/// In case or read-in-order, don't create too many reading streams.
@ -1678,7 +1667,7 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction,
}
if (!sort_description.empty())
{
const size_t used_prefix_of_sorting_key_size = order_info->used_prefix_of_sorting_key_size;
const size_t used_prefix_of_sorting_key_size = query_info.input_order_info->used_prefix_of_sorting_key_size;
if (sort_description.size() > used_prefix_of_sorting_key_size)
sort_description.resize(used_prefix_of_sorting_key_size);
output_stream->sort_description = std::move(sort_description);
@ -1708,7 +1697,7 @@ void ReadFromMergeTree::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info
*output_stream,
storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(),
getSortDirection(),
query_info.getInputOrderInfo(),
query_info.input_order_info,
prewhere_info);
}
@ -1803,8 +1792,6 @@ Pipe ReadFromMergeTree::spreadMarkRanges(
RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection)
{
const bool final = isQueryWithFinal();
const auto & input_order_info = query_info.getInputOrderInfo();
Names column_names_to_read = result.column_names_to_read;
NameSet names(column_names_to_read.begin(), column_names_to_read.end());
@ -1845,10 +1832,10 @@ Pipe ReadFromMergeTree::spreadMarkRanges(
return spreadMarkRangesAmongStreamsFinal(std::move(parts_with_ranges), num_streams, result.column_names_to_read, column_names_to_read, result_projection);
}
else if (input_order_info)
else if (query_info.input_order_info)
{
return spreadMarkRangesAmongStreamsWithOrder(
std::move(parts_with_ranges), num_streams, column_names_to_read, result_projection, input_order_info);
std::move(parts_with_ranges), num_streams, column_names_to_read, result_projection, query_info.input_order_info);
}
else
{

View File

@ -226,9 +226,8 @@ private:
int getSortDirection() const
{
const InputOrderInfoPtr & order_info = query_info.getInputOrderInfo();
if (order_info)
return order_info->direction;
if (query_info.input_order_info)
return query_info.input_order_info->direction;
return 1;
}

View File

@ -6,30 +6,37 @@
namespace DB
{
ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, ContextPtr context_, Context::QualifiedProjectionName qualified_projection_name_)
ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_)
: SourceStepWithFilter(DataStream{.header = pipe_.getHeader()})
, pipe(std::move(pipe_))
, context(std::move(context_))
, qualified_projection_name(std::move(qualified_projection_name_))
{
}
void ReadFromPreparedSource::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
if (context && context->hasQueryContext())
context->getQueryContext()->addQueryAccessInfo(qualified_projection_name);
for (const auto & processor : pipe.getProcessors())
processors.emplace_back(processor);
pipeline.init(std::move(pipe));
}
ReadFromStorageStep::ReadFromStorageStep(
Pipe pipe_,
String storage_name,
ContextPtr context_,
const SelectQueryInfo & query_info_)
: ReadFromPreparedSource(std::move(pipe_))
, context(std::move(context_))
, query_info(query_info_)
{
setStepDescription(storage_name);
for (const auto & processor : pipe.getProcessors())
processor->setStorageLimits(query_info.storage_limits);
}
void ReadFromStorageStep::applyFilters()
{
if (!context)
return;
for (const auto & processor : pipe.getProcessors())
if (auto * source = dynamic_cast<SourceWithKeyCondition *>(processor.get()))
source->setKeyCondition(filter_nodes.nodes, context);

View File

@ -13,36 +13,25 @@ namespace DB
class ReadFromPreparedSource : public SourceStepWithFilter
{
public:
explicit ReadFromPreparedSource(
Pipe pipe_, ContextPtr context_ = nullptr, Context::QualifiedProjectionName qualified_projection_name_ = {});
explicit ReadFromPreparedSource(Pipe pipe_);
String getName() const override { return "ReadFromPreparedSource"; }
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
protected:
Pipe pipe;
ContextPtr context;
Context::QualifiedProjectionName qualified_projection_name;
};
class ReadFromStorageStep : public ReadFromPreparedSource
{
public:
ReadFromStorageStep(Pipe pipe_, String storage_name, const SelectQueryInfo & query_info_, ContextPtr context_)
: ReadFromPreparedSource(std::move(pipe_), std::move(context_)), query_info(query_info_)
{
setStepDescription(storage_name);
for (const auto & processor : pipe.getProcessors())
processor->setStorageLimits(query_info.storage_limits);
}
ReadFromStorageStep(Pipe pipe_, String storage_name, ContextPtr context_, const SelectQueryInfo & query_info_);
String getName() const override { return "ReadFromStorage"; }
void applyFilters() override;
private:
ContextPtr context;
SelectQueryInfo query_info;
};

View File

@ -332,7 +332,7 @@ ReadFromSystemNumbersStep::ReadFromSystemNumbersStep(
, storage{std::move(storage_)}
, storage_snapshot{storage_snapshot_}
, context{std::move(context_)}
, key_expression{KeyDescription::parse(column_names[0], storage_snapshot->getMetadataForQuery()->columns, context).expression}
, key_expression{KeyDescription::parse(column_names[0], storage_snapshot->metadata->columns, context).expression}
, max_block_size{max_block_size_}
, num_streams{num_streams_}
, limit_length_and_offset(InterpreterSelectQuery::getLimitLengthAndOffset(query_info.query->as<ASTSelectQuery&>(), context))

View File

@ -420,7 +420,11 @@ Chain buildPushingToViewsChain(
if (!no_destination && context->hasQueryContext())
{
context->getQueryContext()->addQueryAccessInfo(
backQuoteIfNeed(view_id.getDatabaseName()), views_data->views.back().runtime_stats->target_name, {}, "", view_id.getFullTableName());
backQuoteIfNeed(view_id.getDatabaseName()),
views_data->views.back().runtime_stats->target_name,
/*column_names=*/ {});
context->getQueryContext()->addViewAccessInfo(view_id.getFullTableName());
}
}

View File

@ -112,6 +112,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
std::make_unique<WriteBufferFromOStream>(*response_body_ostr),
compress ? compression_method : CompressionMethod::None,
compression_level,
0,
working_buffer.size(),
working_buffer.begin());
else

View File

@ -724,13 +724,13 @@ public:
const CompressionMethod compression_method)
: SinkToStorage(sample_block)
{
const auto & settings = context->getSettingsRef();
write_buf = wrapWriteBufferWithCompressionMethod(
std::make_unique<WriteBufferFromHDFS>(
uri,
context->getGlobalContext()->getConfigRef(),
context->getSettingsRef().hdfs_replication,
context->getWriteSettings()),
compression_method, 3);
uri, context->getGlobalContext()->getConfigRef(), context->getSettingsRef().hdfs_replication, context->getWriteSettings()),
compression_method,
static_cast<int>(settings.output_format_compression_level),
static_cast<int>(settings.output_format_compression_zstd_window_log));
writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context);
}

View File

@ -165,11 +165,11 @@ void IStorage::readFromPipe(
if (pipe.empty())
{
auto header = storage_snapshot->getSampleBlockForColumns(column_names);
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, context);
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info);
}
else
{
auto read_step = std::make_unique<ReadFromStorageStep>(std::move(pipe), storage_name, query_info, context);
auto read_step = std::make_unique<ReadFromStorageStep>(std::move(pipe), storage_name, context, query_info);
query_plan.addStep(std::move(read_step));
}
}

View File

@ -6888,7 +6888,7 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage(
ContextPtr query_context,
QueryProcessingStage::Enum to_stage,
const StorageSnapshotPtr &,
SelectQueryInfo & query_info) const
SelectQueryInfo &) const
{
if (query_context->getClientInfo().collaborate_with_initiator)
return QueryProcessingStage::Enum::FetchColumns;
@ -6905,11 +6905,6 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage(
return QueryProcessingStage::Enum::WithMergeableState;
}
if (to_stage >= QueryProcessingStage::Enum::WithMergeableState)
{
query_info.projection = std::nullopt;
}
return QueryProcessingStage::Enum::FetchColumns;
}

View File

@ -125,22 +125,6 @@ static RelativeSize convertAbsoluteSampleSizeToRelative(const ASTSampleRatio::Ra
return std::min(RelativeSize(1), RelativeSize(absolute_sample_size) / RelativeSize(approx_total_rows));
}
static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & query)
{
SortDescription order_descr;
order_descr.reserve(query.groupBy()->children.size());
for (const auto & elem : query.groupBy()->children)
{
/// Note, here aliases should not be used, since there will be no such column in a block.
String name = elem->getColumnNameWithoutAlias();
order_descr.emplace_back(name, 1, 1);
}
return order_descr;
}
QueryPlanPtr MergeTreeDataSelectExecutor::read(
const Names & column_names_to_return,
const StorageSnapshotPtr & storage_snapshot,
@ -148,339 +132,32 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read(
ContextPtr context,
const UInt64 max_block_size,
const size_t num_streams,
QueryProcessingStage::Enum processed_stage,
std::shared_ptr<PartitionIdToMaxBlock> max_block_numbers_to_read,
bool enable_parallel_reading) const
{
if (query_info.merge_tree_empty_result)
return std::make_unique<QueryPlan>();
const auto & settings = context->getSettingsRef();
const auto & metadata_for_reading = storage_snapshot->getMetadataForQuery();
const auto & snapshot_data = assert_cast<const MergeTreeData::SnapshotData &>(*storage_snapshot->data);
const auto & parts = snapshot_data.parts;
const auto & alter_conversions = snapshot_data.alter_conversions;
if (!query_info.projection)
{
auto step = readFromParts(
query_info.merge_tree_select_result_ptr ? MergeTreeData::DataPartsVector{} : parts,
query_info.merge_tree_select_result_ptr ? std::vector<AlterConversionsPtr>{} : alter_conversions,
column_names_to_return,
storage_snapshot,
query_info,
context,
max_block_size,
num_streams,
max_block_numbers_to_read,
query_info.merge_tree_select_result_ptr,
enable_parallel_reading);
auto plan = std::make_unique<QueryPlan>();
if (step)
plan->addStep(std::move(step));
return plan;
}
LOG_DEBUG(
log,
"Choose {} {} projection {}",
query_info.projection->complete ? "complete" : "incomplete",
query_info.projection->desc->type,
query_info.projection->desc->name);
const ASTSelectQuery & select_query = query_info.query->as<ASTSelectQuery &>();
QueryPlanResourceHolder resources;
auto projection_plan = std::make_unique<QueryPlan>();
if (query_info.projection->desc->is_minmax_count_projection)
{
Pipe pipe(std::make_shared<SourceFromSingleChunk>(query_info.minmax_count_projection_block));
auto read_from_pipe = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
projection_plan->addStep(std::move(read_from_pipe));
}
else if (query_info.projection->merge_tree_projection_select_result_ptr)
{
LOG_DEBUG(log, "projection required columns: {}", fmt::join(query_info.projection->required_columns, ", "));
projection_plan->addStep(readFromParts(
/*parts=*/ {},
/*alter_conversions=*/ {},
query_info.projection->required_columns,
storage_snapshot,
query_info,
context,
max_block_size,
num_streams,
max_block_numbers_to_read,
query_info.projection->merge_tree_projection_select_result_ptr,
enable_parallel_reading));
}
if (projection_plan->isInitialized())
{
if (query_info.projection->before_where)
{
auto where_step = std::make_unique<FilterStep>(
projection_plan->getCurrentDataStream(),
query_info.projection->before_where,
query_info.projection->where_column_name,
query_info.projection->remove_where_filter);
where_step->setStepDescription("WHERE");
projection_plan->addStep(std::move(where_step));
}
if (query_info.projection->before_aggregation)
{
auto expression_before_aggregation
= std::make_unique<ExpressionStep>(projection_plan->getCurrentDataStream(), query_info.projection->before_aggregation);
expression_before_aggregation->setStepDescription("Before GROUP BY");
projection_plan->addStep(std::move(expression_before_aggregation));
}
/// NOTE: input_order_info (for projection and not) is set only if projection is complete
if (query_info.has_order_by && !query_info.need_aggregate && query_info.projection->input_order_info)
{
chassert(query_info.projection->complete);
SortDescription output_order_descr = InterpreterSelectQuery::getSortDescription(select_query, context);
UInt64 limit = InterpreterSelectQuery::getLimitForSorting(select_query, context);
auto sorting_step = std::make_unique<SortingStep>(
projection_plan->getCurrentDataStream(),
query_info.projection->input_order_info->sort_description_for_merging,
output_order_descr,
settings.max_block_size,
limit);
sorting_step->setStepDescription("ORDER BY for projections");
projection_plan->addStep(std::move(sorting_step));
}
}
auto ordinary_query_plan = std::make_unique<QueryPlan>();
if (query_info.projection->merge_tree_normal_select_result_ptr)
{
auto storage_from_base_parts_of_projection
= std::make_shared<StorageFromMergeTreeDataPart>(data, query_info.projection->merge_tree_normal_select_result_ptr);
auto interpreter = InterpreterSelectQuery(
query_info.query,
context,
storage_from_base_parts_of_projection,
nullptr,
SelectQueryOptions{processed_stage}.projectionQuery());
interpreter.buildQueryPlan(*ordinary_query_plan);
const auto & expressions = interpreter.getAnalysisResult();
if (processed_stage == QueryProcessingStage::Enum::FetchColumns && expressions.before_where)
{
auto where_step = std::make_unique<FilterStep>(
ordinary_query_plan->getCurrentDataStream(),
expressions.before_where,
expressions.where_column_name,
expressions.remove_where_filter);
where_step->setStepDescription("WHERE");
ordinary_query_plan->addStep(std::move(where_step));
}
}
Pipe projection_pipe;
Pipe ordinary_pipe;
if (query_info.projection->desc->type == ProjectionDescription::Type::Aggregate)
{
auto make_aggregator_params = [&](bool projection)
{
const auto & keys = query_info.projection->aggregation_keys.getNames();
AggregateDescriptions aggregates = query_info.projection->aggregate_descriptions;
/// This part is hacky.
/// We want AggregatingTransform to work with aggregate states instead of normal columns.
/// It is almost the same, just instead of adding new data to aggregation state we merge it with existing.
///
/// It is needed because data in projection:
/// * is not merged completely (we may have states with the same key in different parts)
/// * is not split into buckets (so if we just use MergingAggregated, it will use single thread)
const bool only_merge = projection;
Aggregator::Params params(
keys,
aggregates,
query_info.projection->aggregate_overflow_row,
settings.max_rows_to_group_by,
settings.group_by_overflow_mode,
settings.group_by_two_level_threshold,
settings.group_by_two_level_threshold_bytes,
settings.max_bytes_before_external_group_by,
settings.empty_result_for_aggregation_by_empty_set,
context->getTempDataOnDisk(),
settings.max_threads,
settings.min_free_disk_space_for_temporary_data,
settings.compile_aggregate_expressions,
settings.min_count_to_compile_aggregate_expression,
settings.max_block_size,
settings.enable_software_prefetch_in_aggregation,
only_merge,
settings.optimize_group_by_constant_keys,
settings.min_hit_rate_to_use_consecutive_keys_optimization,
/*stats_collecting_params=*/ {});
return std::make_pair(params, only_merge);
};
if (ordinary_query_plan->isInitialized() && projection_plan->isInitialized())
{
auto projection_builder = projection_plan->buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
projection_pipe = QueryPipelineBuilder::getPipe(std::move(*projection_builder), resources);
auto ordinary_builder = ordinary_query_plan->buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
ordinary_pipe = QueryPipelineBuilder::getPipe(std::move(*ordinary_builder), resources);
/// Here we create shared ManyAggregatedData for both projection and ordinary data.
/// For ordinary data, AggregatedData is filled in a usual way.
/// For projection data, AggregatedData is filled by merging aggregation states.
/// When all AggregatedData is filled, we merge aggregation states together in a usual way.
/// Pipeline will look like:
/// ReadFromProjection -> Aggregating (only merge states) ->
/// ReadFromProjection -> Aggregating (only merge states) ->
/// ... -> Resize -> ConvertingAggregatedToChunks
/// ReadFromOrdinaryPart -> Aggregating (usual) -> (added by last Aggregating)
/// ReadFromOrdinaryPart -> Aggregating (usual) ->
/// ...
auto many_data = std::make_shared<ManyAggregatedData>(projection_pipe.numOutputPorts() + ordinary_pipe.numOutputPorts());
size_t counter = 0;
AggregatorListPtr aggregator_list_ptr = std::make_shared<AggregatorList>();
/// TODO apply optimize_aggregation_in_order here too (like below)
auto build_aggregate_pipe = [&](Pipe & pipe, bool projection)
{
auto [params, only_merge] = make_aggregator_params(projection);
AggregatingTransformParamsPtr transform_params = std::make_shared<AggregatingTransformParams>(
pipe.getHeader(), std::move(params), aggregator_list_ptr, query_info.projection->aggregate_final);
pipe.resize(pipe.numOutputPorts(), true, true);
auto merge_threads = num_streams;
auto temporary_data_merge_threads = settings.aggregation_memory_efficient_merge_threads
? static_cast<size_t>(settings.aggregation_memory_efficient_merge_threads)
: static_cast<size_t>(settings.max_threads);
pipe.addSimpleTransform([&](const Block & header)
{
return std::make_shared<AggregatingTransform>(
header, transform_params, many_data, counter++, merge_threads, temporary_data_merge_threads);
});
};
if (!projection_pipe.empty())
build_aggregate_pipe(projection_pipe, true);
if (!ordinary_pipe.empty())
build_aggregate_pipe(ordinary_pipe, false);
}
else
{
auto add_aggregating_step = [&](QueryPlanPtr & query_plan, bool projection)
{
auto [params, only_merge] = make_aggregator_params(projection);
auto merge_threads = num_streams;
auto temporary_data_merge_threads = settings.aggregation_memory_efficient_merge_threads
? static_cast<size_t>(settings.aggregation_memory_efficient_merge_threads)
: static_cast<size_t>(settings.max_threads);
InputOrderInfoPtr group_by_info = query_info.projection->input_order_info;
SortDescription sort_description_for_merging;
SortDescription group_by_sort_description;
if (group_by_info && settings.optimize_aggregation_in_order)
{
group_by_sort_description = getSortDescriptionFromGroupBy(select_query);
sort_description_for_merging = group_by_info->sort_description_for_merging;
}
else
group_by_info = nullptr;
// We don't have information regarding the `to_stage` of the query processing, only about `from_stage` (which is passed through `processed_stage` argument).
// Thus we cannot assign false here since it may be a query over distributed table.
const bool should_produce_results_in_order_of_bucket_number = true;
auto aggregating_step = std::make_unique<AggregatingStep>(
query_plan->getCurrentDataStream(),
std::move(params),
/* grouping_sets_params_= */ GroupingSetsParamsList{},
query_info.projection->aggregate_final,
settings.max_block_size,
settings.aggregation_in_order_max_block_bytes,
merge_threads,
temporary_data_merge_threads,
/* storage_has_evenly_distributed_read_= */ false,
/* group_by_use_nulls */ false,
std::move(sort_description_for_merging),
std::move(group_by_sort_description),
should_produce_results_in_order_of_bucket_number,
settings.enable_memory_bound_merging_of_aggregation_results,
!group_by_info && settings.force_aggregation_in_order);
query_plan->addStep(std::move(aggregating_step));
};
if (projection_plan->isInitialized())
{
add_aggregating_step(projection_plan, true);
auto projection_builder = projection_plan->buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
projection_pipe = QueryPipelineBuilder::getPipe(std::move(*projection_builder), resources);
}
if (ordinary_query_plan->isInitialized())
{
add_aggregating_step(ordinary_query_plan, false);
auto ordinary_builder = ordinary_query_plan->buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
ordinary_pipe = QueryPipelineBuilder::getPipe(std::move(*ordinary_builder), resources);
}
}
}
else
{
if (projection_plan->isInitialized())
{
auto projection_builder = projection_plan->buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
projection_pipe = QueryPipelineBuilder::getPipe(std::move(*projection_builder), resources);
}
if (ordinary_query_plan->isInitialized())
{
auto ordinary_builder = ordinary_query_plan->buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
ordinary_pipe = QueryPipelineBuilder::getPipe(std::move(*ordinary_builder), resources);
}
}
Pipes pipes;
pipes.emplace_back(std::move(projection_pipe));
pipes.emplace_back(std::move(ordinary_pipe));
auto pipe = Pipe::unitePipes(std::move(pipes));
auto plan = std::make_unique<QueryPlan>();
if (pipe.empty())
return plan;
pipe.resize(1);
auto step = std::make_unique<ReadFromStorageStep>(
std::move(pipe),
fmt::format("MergeTree(with {} projection {})", query_info.projection->desc->type, query_info.projection->desc->name),
auto step = readFromParts(
parts,
alter_conversions,
column_names_to_return,
storage_snapshot,
query_info,
context);
plan->addStep(std::move(step));
plan->addInterpreterContext(query_info.projection->context);
context,
max_block_size,
num_streams,
max_block_numbers_to_read,
/*merge_tree_select_result_ptr=*/ nullptr,
enable_parallel_reading);
auto plan = std::make_unique<QueryPlan>();
if (step)
plan->addStep(std::move(step));
return plan;
}

View File

@ -34,7 +34,6 @@ public:
ContextPtr context,
UInt64 max_block_size,
size_t num_streams,
QueryProcessingStage::Enum processed_stage,
std::shared_ptr<PartitionIdToMaxBlock> max_block_numbers_to_read = nullptr,
bool enable_parallel_reading = false) const;

View File

@ -347,11 +347,11 @@ void StorageNATS::read(
if (pipe.empty())
{
auto header = storage_snapshot->getSampleBlockForColumns(column_names);
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, local_context);
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info);
}
else
{
auto read_step = std::make_unique<ReadFromStorageStep>(std::move(pipe), getName(), query_info, local_context);
auto read_step = std::make_unique<ReadFromStorageStep>(std::move(pipe), getName(), local_context, query_info);
query_plan.addStep(std::move(read_step));
query_plan.addInterpreterContext(modified_context);
}

View File

@ -700,7 +700,7 @@ void StorageRabbitMQ::read(
if (num_created_consumers == 0)
{
auto header = storage_snapshot->getSampleBlockForColumns(column_names);
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, local_context);
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info);
return;
}
@ -758,11 +758,11 @@ void StorageRabbitMQ::read(
if (pipe.empty())
{
auto header = storage_snapshot->getSampleBlockForColumns(column_names);
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, local_context);
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info);
}
else
{
auto read_step = std::make_unique<ReadFromStorageStep>(std::move(pipe), getName(), query_info, local_context);
auto read_step = std::make_unique<ReadFromStorageStep>(std::move(pipe), getName(), local_context, query_info);
query_plan.addStep(std::move(read_step));
query_plan.addInterpreterContext(modified_context);
}

View File

@ -64,7 +64,7 @@ void readFinalFromNestedStorage(
if (!query_plan.isInitialized())
{
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, nested_header, query_info, context);
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, nested_header, query_info);
return;
}

View File

@ -9,7 +9,6 @@
#include <Interpreters/PreparedSets.h>
#include <Planner/PlannerContext.h>
#include <QueryPipeline/StreamLocalLimits.h>
#include <Storages/ProjectionsDescription.h>
#include <memory>
@ -142,32 +141,6 @@ class IMergeTreeDataPart;
using ManyExpressionActions = std::vector<ExpressionActionsPtr>;
// The projection selected to execute current query
struct ProjectionCandidate
{
ProjectionDescriptionRawPtr desc{};
PrewhereInfoPtr prewhere_info;
ActionsDAGPtr before_where;
String where_column_name;
bool remove_where_filter = false;
ActionsDAGPtr before_aggregation;
Names required_columns;
NamesAndTypesList aggregation_keys;
AggregateDescriptions aggregate_descriptions;
bool aggregate_overflow_row = false;
bool aggregate_final = false;
bool complete = false;
ReadInOrderOptimizerPtr order_optimizer;
InputOrderInfoPtr input_order_info;
ManyExpressionActions group_by_elements_actions;
SortDescription group_by_elements_order_descr;
MergeTreeDataSelectAnalysisResultPtr merge_tree_projection_select_result_ptr;
MergeTreeDataSelectAnalysisResultPtr merge_tree_normal_select_result_ptr;
/// Because projection analysis uses a separate interpreter.
ContextPtr context;
};
/** Query along with some additional data,
* that can be used during query processing
* inside storage engines.
@ -180,7 +153,6 @@ struct SelectQueryInfo
ASTPtr query;
ASTPtr view_query; /// Optimized VIEW query
ASTPtr original_query; /// Unmodified query for projection analysis
/// Query tree
QueryTreeNodePtr query_tree;
@ -242,18 +214,11 @@ struct SelectQueryInfo
ClusterPtr getCluster() const { return !optimized_cluster ? cluster : optimized_cluster; }
/// If not null, it means we choose a projection to execute current query.
std::optional<ProjectionCandidate> projection;
bool ignore_projections = false;
bool is_projection_query = false;
bool merge_tree_empty_result = false;
bool settings_limit_offset_done = false;
bool is_internal = false;
Block minmax_count_projection_block;
MergeTreeDataSelectAnalysisResultPtr merge_tree_select_result_ptr;
bool parallel_replicas_disabled = false;
bool is_parameterized_view = false;
bool optimize_trivial_count = false;
// If limit is not 0, that means it's a trivial limit query.
@ -262,11 +227,6 @@ struct SelectQueryInfo
/// For IStorageSystemOneBlock
std::vector<UInt8> columns_mask;
InputOrderInfoPtr getInputOrderInfo() const
{
return input_order_info ? input_order_info : (projection ? projection->input_order_info : nullptr);
}
bool isFinal() const;
};
}

View File

@ -535,7 +535,12 @@ public:
, format_settings(format_settings_)
{
StoredObject object(blob_path);
write_buf = wrapWriteBufferWithCompressionMethod(object_storage->writeObject(object, WriteMode::Rewrite), compression_method, 3);
const auto & settings = context->getSettingsRef();
write_buf = wrapWriteBufferWithCompressionMethod(
object_storage->writeObject(object, WriteMode::Rewrite),
compression_method,
static_cast<int>(settings.output_format_compression_level),
static_cast<int>(settings.output_format_compression_zstd_window_log));
writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, format_settings);
}

View File

@ -212,8 +212,6 @@ QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(
{
if (auto destination = getDestinationTable())
{
/// TODO: Find a way to support projections for StorageBuffer
query_info.ignore_projections = true;
const auto & destination_metadata = destination->getInMemoryMetadataPtr();
return destination->getQueryProcessingStage(local_context, to_stage, destination->getStorageSnapshot(destination_metadata, local_context), query_info);
}
@ -337,12 +335,12 @@ void StorageBuffer::read(
pipes_from_buffers.emplace_back(std::make_shared<BufferSource>(column_names, buf, storage_snapshot));
pipe_from_buffers = Pipe::unitePipes(std::move(pipes_from_buffers));
if (query_info.getInputOrderInfo())
if (query_info.input_order_info)
{
/// Each buffer has one block, and it not guaranteed that rows in each block are sorted by order keys
pipe_from_buffers.addSimpleTransform([&](const Block & header)
{
return std::make_shared<PartialSortingTransform>(header, query_info.getInputOrderInfo()->sort_description_for_merging, 0);
return std::make_shared<PartialSortingTransform>(header, query_info.input_order_info->sort_description_for_merging, 0);
});
}
}
@ -360,7 +358,7 @@ void StorageBuffer::read(
/// TODO: Find a way to support projections for StorageBuffer
auto interpreter = InterpreterSelectQuery(
query_info.query, local_context, std::move(pipe_from_buffers),
SelectQueryOptions(processed_stage).ignoreProjections());
SelectQueryOptions(processed_stage));
interpreter.addStorageLimits(*query_info.storage_limits);
interpreter.buildQueryPlan(buffers_plan);
}

View File

@ -73,7 +73,7 @@ void StorageExternalDistributed::read(
if (plans.empty())
{
auto header = storage_snapshot->getSampleBlockForColumns(column_names);
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, context);
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info);
}
if (plans.size() == 1)

View File

@ -1397,7 +1397,7 @@ void StorageFile::read(
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File {} doesn't exist", p->at(0));
auto header = storage_snapshot->getSampleBlockForColumns(column_names);
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, context);
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info);
return;
}
}
@ -1575,8 +1575,12 @@ public:
/// In case of formats with prefixes if file is not empty we have already written prefix.
bool do_not_write_prefix = naked_buffer->size();
write_buf = wrapWriteBufferWithCompressionMethod(std::move(naked_buffer), compression_method, 3);
const auto & settings = context->getSettingsRef();
write_buf = wrapWriteBufferWithCompressionMethod(
std::move(naked_buffer),
compression_method,
static_cast<int>(settings.output_format_compression_level),
static_cast<int>(settings.output_format_compression_zstd_window_log));
writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format_name,
*write_buf, metadata_snapshot->getSampleBlock(), context, format_settings);

View File

@ -149,10 +149,6 @@ QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(
const StorageSnapshotPtr &,
SelectQueryInfo & query_info) const
{
/// TODO: Find a way to support projections for StorageMaterializedView. Why do we use different
/// metadata for materialized view and target table? If they are the same, we can get rid of all
/// converting and use it just like a normal view.
query_info.ignore_projections = true;
const auto & target_metadata = getTargetTable()->getInMemoryMetadataPtr();
return getTargetTable()->getQueryProcessingStage(local_context, to_stage, getTargetTable()->getStorageSnapshot(target_metadata, local_context), query_info);
}

View File

@ -278,8 +278,6 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(
size_t selected_table_size = 0;
/// TODO: Find a way to support projections for StorageMerge
query_info.ignore_projections = true;
for (const auto & iterator : database_table_iterators)
{
while (iterator->isValid())
@ -854,7 +852,8 @@ QueryPlan ReadFromMerge::createPlanForTable(
{
InterpreterSelectQueryAnalyzer interpreter(modified_query_info.query_tree,
modified_context,
SelectQueryOptions(processed_stage).ignoreProjections());
SelectQueryOptions(processed_stage));
auto & planner = interpreter.getPlanner();
planner.buildQueryPlanIfNeeded();
plan = std::move(planner).extractQueryPlan();
@ -865,7 +864,8 @@ QueryPlan ReadFromMerge::createPlanForTable(
/// TODO: Find a way to support projections for StorageMerge
InterpreterSelectQuery interpreter{modified_query_info.query,
modified_context,
SelectQueryOptions(processed_stage).ignoreProjections()};
SelectQueryOptions(processed_stage)};
interpreter.buildQueryPlan(plan);
}
}

View File

@ -250,7 +250,6 @@ void StorageMergeTree::read(
local_context,
max_block_size,
num_streams,
processed_stage,
nullptr,
enable_parallel_reading))
query_plan = std::move(*plan);

View File

@ -38,8 +38,6 @@ public:
const StorageSnapshotPtr &,
SelectQueryInfo & info) const override
{
/// TODO: Find a way to support projections for StorageProxy
info.ignore_projections = true;
const auto & nested_metadata = getNested()->getInMemoryMetadataPtr();
return getNested()->getQueryProcessingStage(context, to_stage, getNested()->getStorageSnapshot(nested_metadata, context), info);
}

View File

@ -5345,12 +5345,12 @@ void StorageReplicatedMergeTree::read(
/// 2. Do not read parts that have not yet been written to the quorum of the replicas.
/// For this you have to synchronously go to ZooKeeper.
if (settings.select_sequential_consistency)
return readLocalSequentialConsistencyImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams);
return readLocalSequentialConsistencyImpl(query_plan, column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams);
if (local_context->canUseParallelReplicasOnInitiator())
return readParallelReplicasImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage);
readLocalImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams);
readLocalImpl(query_plan, column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams);
}
void StorageReplicatedMergeTree::readLocalSequentialConsistencyImpl(
@ -5359,14 +5359,15 @@ void StorageReplicatedMergeTree::readLocalSequentialConsistencyImpl(
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr local_context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams)
{
auto max_added_blocks = std::make_shared<ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock>(getMaxAddedBlocks());
auto plan = reader.read(column_names, storage_snapshot, query_info, local_context,
max_block_size, num_streams, processed_stage, std::move(max_added_blocks),
/* enable_parallel_reading= */false);
auto plan = reader.read(
column_names, storage_snapshot, query_info, local_context,
max_block_size, num_streams, std::move(max_added_blocks),
/* enable_parallel_reading=*/ false);
if (plan)
query_plan = std::move(*plan);
}
@ -5420,16 +5421,15 @@ void StorageReplicatedMergeTree::readLocalImpl(
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr local_context,
QueryProcessingStage::Enum processed_stage,
const size_t max_block_size,
const size_t num_streams)
{
auto plan = reader.read(
column_names, storage_snapshot, query_info,
local_context, max_block_size, num_streams,
processed_stage,
/* max_block_numbers_to_read= */ nullptr,
/* enable_parallel_reading= */ local_context->canUseParallelReplicasOnFollower());
if (plan)
query_plan = std::move(*plan);
}

View File

@ -561,7 +561,6 @@ private:
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr local_context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams);
@ -571,7 +570,6 @@ private:
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr local_context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams);

View File

@ -8,6 +8,8 @@
#include <IO/S3/Requests.h>
#include <IO/ParallelReadBuffer.h>
#include <IO/SharedThreadPools.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/evaluateConstantExpression.h>
@ -856,6 +858,7 @@ public:
blob_log->query_id = context->getCurrentQueryId();
}
const auto & settings = context->getSettingsRef();
write_buf = wrapWriteBufferWithCompressionMethod(
std::make_unique<WriteBufferFromS3>(
configuration_.client,
@ -868,7 +871,8 @@ public:
threadPoolCallbackRunner<void>(getIOThreadPool().get(), "S3ParallelWrite"),
context->getWriteSettings()),
compression_method,
3);
static_cast<int>(settings.output_format_compression_level),
static_cast<int>(settings.output_format_compression_zstd_window_log));
writer
= FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, format_settings);
}
@ -1377,7 +1381,7 @@ bool StorageS3::Configuration::update(ContextPtr context)
request_settings = s3_settings.request_settings;
request_settings.updateFromSettings(context->getSettings());
if (client && (static_configuration || s3_settings.auth_settings == auth_settings))
if (client && (static_configuration || !auth_settings.hasUpdates(s3_settings.auth_settings)))
return false;
auth_settings.updateFrom(s3_settings.auth_settings);
@ -1600,11 +1604,11 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context
if (engine_args_to_idx.contains("session_token"))
configuration.auth_settings.session_token = checkAndGetLiteralArgument<String>(engine_args[engine_args_to_idx["session_token"]], "session_token");
configuration.auth_settings.no_sign_request = no_sign_request;
if (no_sign_request)
configuration.auth_settings.no_sign_request = no_sign_request;
}
configuration.static_configuration = !configuration.auth_settings.access_key_id.empty();
configuration.static_configuration = !configuration.auth_settings.access_key_id.empty() || configuration.auth_settings.no_sign_request.has_value();
configuration.keys = {configuration.url.key};

View File

@ -541,11 +541,12 @@ StorageURLSink::StorageURLSink(
Poco::URI(uri), http_method, content_type, content_encoding, headers, timeouts, DBMS_DEFAULT_BUFFER_SIZE, proxy_config
);
const auto & settings = context->getSettingsRef();
write_buf = wrapWriteBufferWithCompressionMethod(
std::move(write_buffer),
compression_method,
3
);
static_cast<int>(settings.output_format_compression_level),
static_cast<int>(settings.output_format_compression_zstd_window_log));
writer = FormatFactory::instance().getOutputFormat(format, *write_buf, sample_block, context, format_settings);
}

View File

@ -1,9 +1,9 @@
#!/usr/bin/env python3
from enum import Enum
import logging
from argparse import ArgumentDefaultsHelpFormatter, ArgumentParser
from dataclasses import dataclass, field
from enum import Enum
from pathlib import Path
from typing import Callable, Dict, Iterable, List, Literal, Optional, Union
@ -47,7 +47,7 @@ class JobConfig:
@num_batches - sets number of batches for multi-batch job
"""
digest: DigestConfig = DigestConfig()
digest: DigestConfig = field(default_factory=DigestConfig)
run_command: str = ""
timeout: Optional[int] = None
num_batches: int = 1
@ -67,30 +67,32 @@ class BuildConfig:
sparse_checkout: bool = False
comment: str = ""
static_binary_name: str = ""
job_config: JobConfig = JobConfig(
digest=DigestConfig(
include_paths=[
"./src",
"./contrib/*-cmake",
"./contrib/consistent-hashing",
"./contrib/murmurhash",
"./contrib/libfarmhash",
"./contrib/pdqsort",
"./contrib/cityhash102",
"./contrib/sparse-checkout",
"./contrib/libmetrohash",
"./contrib/update-submodules.sh",
"./contrib/CMakeLists.txt",
"./cmake",
"./base",
"./programs",
"./packages",
"./docker/packager/packager",
],
exclude_files=[".md"],
docker=["clickhouse/binary-builder"],
git_submodules=True,
),
job_config: JobConfig = field(
default_factory=lambda: JobConfig(
digest=DigestConfig(
include_paths=[
"./src",
"./contrib/*-cmake",
"./contrib/consistent-hashing",
"./contrib/murmurhash",
"./contrib/libfarmhash",
"./contrib/pdqsort",
"./contrib/cityhash102",
"./contrib/sparse-checkout",
"./contrib/libmetrohash",
"./contrib/update-submodules.sh",
"./contrib/CMakeLists.txt",
"./cmake",
"./base",
"./programs",
"./packages",
"./docker/packager/packager",
],
exclude_files=[".md"],
docker=["clickhouse/binary-builder"],
git_submodules=True,
),
)
)
def export_env(self, export: bool = False) -> str:
@ -107,14 +109,14 @@ class BuildConfig:
@dataclass
class BuildReportConfig:
builds: List[str]
job_config: JobConfig = JobConfig()
job_config: JobConfig = field(default_factory=JobConfig)
@dataclass
class TestConfig:
required_build: str
force_tests: bool = False
job_config: JobConfig = JobConfig()
job_config: JobConfig = field(default_factory=JobConfig)
BuildConfigs = Dict[str, BuildConfig]

View File

@ -55,6 +55,7 @@ def get_additional_envs(
result.append("USE_PARALLEL_REPLICAS=1")
if "s3 storage" in check_name:
result.append("USE_S3_STORAGE_FOR_MERGE_TREE=1")
result.append("RANDOMIZE_OBJECT_KEY_TYPE=1")
if "analyzer" in check_name:
result.append("USE_NEW_ANALYZER=1")

View File

@ -47,6 +47,7 @@ def get_additional_envs(check_name, run_by_hash_num, run_by_hash_total):
result.append("USE_PARALLEL_REPLICAS=1")
if "s3 storage" in check_name:
result.append("USE_S3_STORAGE_FOR_MERGE_TREE=1")
result.append("RANDOMIZE_OBJECT_KEY_TYPE=1")
if "analyzer" in check_name:
result.append("USE_NEW_ANALYZER=1")

View File

@ -5,20 +5,19 @@ import shutil
import time
from multiprocessing.dummy import Pool
from pathlib import Path
from typing import List, Union
from typing import Any, List, Union
import boto3 # type: ignore
import botocore # type: ignore
from env_helper import (
S3_TEST_REPORTS_BUCKET,
S3_BUILDS_BUCKET,
RUNNER_TEMP,
CI,
S3_URL,
S3_DOWNLOAD,
)
from compress_files import compress_file_fast
from env_helper import (
CI,
RUNNER_TEMP,
S3_BUILDS_BUCKET,
S3_DOWNLOAD,
S3_TEST_REPORTS_BUCKET,
S3_URL,
)
def _flatten_list(lst):
@ -34,11 +33,14 @@ def _flatten_list(lst):
class S3Helper:
max_pool_size = 100
def __init__(self):
def __init__(self, client: Any = None, endpoint: str = S3_URL):
self.host = endpoint
if client is not None:
self.client = client
return
config = botocore.config.Config(max_pool_connections=self.max_pool_size)
self.session = boto3.session.Session(region_name="us-east-1")
self.client = self.session.client("s3", endpoint_url=S3_URL, config=config)
self.host = S3_URL
session = boto3.session.Session(region_name="us-east-1")
self.client = session.client("s3", endpoint_url=endpoint, config=config)
def _upload_file_to_s3(
self, bucket_name: str, file_path: Path, s3_path: str
@ -199,6 +201,7 @@ class S3Helper:
t = time.time()
except Exception as ex:
logging.critical("Failed to upload file, expcetion %s", ex)
return ""
return self.s3_url(bucket_name, s3_path)
p = Pool(self.max_pool_size)

View File

@ -0,0 +1,32 @@
<clickhouse>
<storage_configuration>
<disks>
<s3>
<type>s3</type>
<endpoint>http://localhost:11111/test/</endpoint>
<access_key_id>clickhouse</access_key_id>
<secret_access_key>clickhouse</secret_access_key>
<key_compatibility_prefix>test</key_compatibility_prefix>
<!--key_template should not be used without enabled storage_metadata_write_full_object_key-->
<key_template>[a-z]{3}-first-random-part/new-style-prefix/[a-z]{3}/[a-z]{29}</key_template>
</s3>
<cached_s3>
<type>cache</type>
<max_size>1Gi</max_size>
<path>cached_s3/</path>
<disk>s3</disk>
</cached_s3>
</disks>
<policies>
<s3>
<volumes>
<main><disk>cached_s3</disk></main>
</volumes>
</s3>
</policies>
</storage_configuration>
<merge_tree>
<storage_policy>s3</storage_policy>
</merge_tree>
<temporary_data_in_cache>cached_s3</temporary_data_in_cache>
</clickhouse>

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