Merge branch 'master' into revert-34211-revert-34153-add_func_tests_over_s3

This commit is contained in:
alesapin 2022-02-04 11:23:46 +03:00
commit 2ed45b2a98
95 changed files with 2280 additions and 526 deletions

View File

@ -86,6 +86,7 @@ jobs:
StyleCheck:
needs: DockerHubPush
runs-on: [self-hosted, style-checker]
if: ${{ success() || failure() }}
steps:
- name: Set envs
run: |
@ -93,6 +94,8 @@ jobs:
TEMP_PATH=${{ runner.temp }}/style_check
EOF
- name: Download changed images
# even if artifact does not exist, e.g. on `do not test` label or failed Docker job
continue-on-error: true
uses: actions/download-artifact@v2
with:
name: changed_images

View File

@ -111,6 +111,7 @@ jobs:
StyleCheck:
needs: DockerHubPush
runs-on: [self-hosted, style-checker]
if: ${{ success() || failure() }}
steps:
- name: Set envs
run: |
@ -118,6 +119,8 @@ jobs:
TEMP_PATH=${{ runner.temp }}/style_check
EOF
- name: Download changed images
# even if artifact does not exist, e.g. on `do not test` label or failed Docker job
continue-on-error: true
uses: actions/download-artifact@v2
with:
name: changed_images

View File

@ -22,7 +22,6 @@ jobs:
- name: Check out repository code
uses: actions/checkout@v2
- name: Download packages and push to Artifactory
env:
run: |
rm -rf "$TEMP_PATH" && mkdir -p "$REPO_COPY"
cp -r "$GITHUB_WORKSPACE" "$REPO_COPY"

View File

@ -14,25 +14,25 @@ jobs:
UpdateVersions:
runs-on: [self-hosted, style-checker]
steps:
- name: Get tag name
run: echo "GITHUB_TAG=${GITHUB_REF#refs/tags/}" >> "$GITHUB_ENV"
- name: Check out repository code
uses: actions/checkout@v2
with:
ref: master
- name: Generate versions
run: |
git fetch --tags
./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv
- name: Create Pull Request
uses: peter-evans/create-pull-request@v3
with:
commit-message: Update version_date.tsv after ${{ env.GITHUB_TAG }}
branch: auto/${{ env.GITHUB_TAG }}
delete-branch: true
title: Update version_date.tsv after ${{ env.GITHUB_TAG }}
body: |
Update version_date.tsv after ${{ env.GITHUB_TAG }}
- name: Get tag name
run: echo "GITHUB_TAG=${GITHUB_REF#refs/tags/}" >> "$GITHUB_ENV"
- name: Check out repository code
uses: actions/checkout@v2
with:
ref: master
- name: Generate versions
run: |
git fetch --tags
./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv
- name: Create Pull Request
uses: peter-evans/create-pull-request@v3
with:
commit-message: Update version_date.tsv after ${{ env.GITHUB_TAG }}
branch: auto/${{ env.GITHUB_TAG }}
delete-branch: true
title: Update version_date.tsv after ${{ env.GITHUB_TAG }}
body: |
Update version_date.tsv after ${{ env.GITHUB_TAG }}
Changelog category (leave one):
- Not for changelog (changelog entry is not required)
Changelog category (leave one):
- Not for changelog (changelog entry is not required)

View File

@ -11,6 +11,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
curl \
git \
libxml2-utils \
moreutils \
pylint \
python3-pip \
shellcheck \

View File

@ -10,72 +10,26 @@ def process_result(result_folder):
status = "success"
description = ""
test_results = []
checks = (
("header duplicates", "duplicate_output.txt"),
("shellcheck", "shellcheck_output.txt"),
("style", "style_output.txt"),
("typos", "typos_output.txt"),
("whitespaces", "whitespaces_output.txt"),
("workflows", "workflows_output.txt"),
)
duplicate_log_path = "{}/duplicate_output.txt".format(result_folder)
if not os.path.exists(duplicate_log_path):
logging.info("No header duplicates check log on path %s", duplicate_log_path)
return "exception", "No header duplicates check log", []
elif os.stat(duplicate_log_path).st_size != 0:
description += " Header duplicates check failed. "
test_results.append(("Header duplicates check", "FAIL"))
status = "failure"
else:
test_results.append(("Header duplicates check", "OK"))
shellcheck_log_path = "{}/shellcheck_output.txt".format(result_folder)
if not os.path.exists(shellcheck_log_path):
logging.info("No shellcheck log on path %s", shellcheck_log_path)
return "exception", "No shellcheck log", []
elif os.stat(shellcheck_log_path).st_size != 0:
description += " Shellcheck check failed. "
test_results.append(("Shellcheck ", "FAIL"))
status = "failure"
else:
test_results.append(("Shellcheck", "OK"))
style_log_path = "{}/style_output.txt".format(result_folder)
if not os.path.exists(style_log_path):
logging.info("No style check log on path %s", style_log_path)
return "exception", "No style check log", []
elif os.stat(style_log_path).st_size != 0:
description += "Style check failed. "
test_results.append(("Style check", "FAIL"))
status = "failure"
else:
test_results.append(("Style check", "OK"))
typos_log_path = "{}/typos_output.txt".format(result_folder)
if not os.path.exists(typos_log_path):
logging.info("No typos check log on path %s", typos_log_path)
return "exception", "No typos check log", []
elif os.stat(typos_log_path).st_size != 0:
description += "Typos check failed. "
test_results.append(("Typos check", "FAIL"))
status = "failure"
else:
test_results.append(("Typos check", "OK"))
whitespaces_log_path = "{}/whitespaces_output.txt".format(result_folder)
if not os.path.exists(whitespaces_log_path):
logging.info("No whitespaces check log on path %s", whitespaces_log_path)
return "exception", "No whitespaces check log", []
elif os.stat(whitespaces_log_path).st_size != 0:
description += "Whitespaces check failed. "
test_results.append(("Whitespaces check", "FAIL"))
status = "failure"
else:
test_results.append(("Whitespaces check", "OK"))
workflows_log_path = "{}/workflows_output.txt".format(result_folder)
if not os.path.exists(workflows_log_path):
logging.info("No workflows check log on path %s", style_log_path)
return "exception", "No workflows check log", []
elif os.stat(whitespaces_log_path).st_size != 0:
description += "Workflows check failed. "
test_results.append(("Workflows check", "FAIL"))
status = "failure"
else:
test_results.append(("Workflows check", "OK"))
for name, out_file in checks:
full_path = os.path.join(result_folder, out_file)
if not os.path.exists(full_path):
logging.info("No %s check log on path %s", name, full_path)
return "exception", f"No {name} check log", []
elif os.stat(full_path).st_size != 0:
description += f"Check {name} failed. "
test_results.append((f"Check {name}", "FAIL"))
status = "failure"
else:
test_results.append((f"Check {name}", "OK"))
if not description:
description += "Style check success"

View File

@ -3,10 +3,16 @@
# yaml check is not the best one
cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv
echo "Check duplicates" | ts
./check-duplicate-includes.sh |& tee /test_output/duplicate_output.txt
echo "Check style" | ts
./check-style -n |& tee /test_output/style_output.txt
echo "Check typos" | ts
./check-typos |& tee /test_output/typos_output.txt
echo "Check whitespaces" | ts
./check-whitespaces -n |& tee /test_output/whitespaces_output.txt
echo "Check sorkflows" | ts
./check-workflows |& tee /test_output/workflows_output.txt
echo "Check shell scripts with shellcheck" | ts
./shellcheck-run.sh |& tee /test_output/shellcheck_output.txt
/process_style_check_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv

View File

@ -10,7 +10,7 @@ Applies Student's t-test to samples from two populations.
**Syntax**
``` sql
studentTTest(sample_data, sample_index)
studentTTest([confidence_level])(sample_data, sample_index)
```
Values of both samples are in the `sample_data` column. If `sample_index` equals to 0 then the value in that row belongs to the sample from the first population. Otherwise it belongs to the sample from the second population.
@ -21,12 +21,19 @@ The null hypothesis is that means of populations are equal. Normal distribution
- `sample_data` — Sample data. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md).
- `sample_index` — Sample index. [Integer](../../../sql-reference/data-types/int-uint.md).
**Parameters**
- `confidence_level` — Confidence level in order to calculate confidence intervals. [Float](../../../sql-reference/data-types/float.md).
**Returned values**
[Tuple](../../../sql-reference/data-types/tuple.md) with two elements:
[Tuple](../../../sql-reference/data-types/tuple.md) with two or four elements (if the optional `confidence_level` is specified):
- calculated t-statistic. [Float64](../../../sql-reference/data-types/float.md).
- calculated p-value. [Float64](../../../sql-reference/data-types/float.md).
- [calculated confidence-interval-low.] [Float64](../../../sql-reference/data-types/float.md).
- [calculated confidence-interval-high.] [Float64](../../../sql-reference/data-types/float.md).
**Example**

View File

@ -10,7 +10,7 @@ Applies Welch's t-test to samples from two populations.
**Syntax**
``` sql
welchTTest(sample_data, sample_index)
welchTTest([confidence_level])(sample_data, sample_index)
```
Values of both samples are in the `sample_data` column. If `sample_index` equals to 0 then the value in that row belongs to the sample from the first population. Otherwise it belongs to the sample from the second population.
@ -21,12 +21,18 @@ The null hypothesis is that means of populations are equal. Normal distribution
- `sample_data` — Sample data. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md).
- `sample_index` — Sample index. [Integer](../../../sql-reference/data-types/int-uint.md).
**Parameters**
- `confidence_level` — Confidence level in order to calculate confidence intervals. [Float](../../../sql-reference/data-types/float.md).
**Returned values**
[Tuple](../../../sql-reference/data-types/tuple.md) with two elements:
[Tuple](../../../sql-reference/data-types/tuple.md) with two two or four elements (if the optional `confidence_level` is specified)
- calculated t-statistic. [Float64](../../../sql-reference/data-types/float.md).
- calculated p-value. [Float64](../../../sql-reference/data-types/float.md).
- [calculated confidence-interval-low.] [Float64](../../../sql-reference/data-types/float.md).
- [calculated confidence-interval-high.] [Float64](../../../sql-reference/data-types/float.md).
**Example**

View File

@ -1 +0,0 @@
../../../en/faq/general/how-do-i-contribute-code-to-clickhouse.md

View File

@ -0,0 +1,17 @@
---
title: 我如何为ClickHouse贡献代码?
toc_hidden: true
toc_priority: 120
---
# 我如何为ClickHouse贡献代码? {#how-do-i-contribute-code-to-clickhouse}
ClickHouse是一个开源项目[在GitHub上开发](https://github.com/ClickHouse/ClickHouse)。
按照惯例,贡献指南发布在源代码库根目录的 [CONTRIBUTING.md](https://github.com/ClickHouse/ClickHouse/blob/master/CONTRIBUTING.md)文件中。
如果你想对ClickHouse提出实质性的改变建议可以考虑[在GitHub上发布一个问题](https://github.com/ClickHouse/ClickHouse/issues/new/choose),解释一下你想做什么,先与维护人员和社区讨论一下。[此类RFC问题的例子](https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aissue+is%3Aopen+rfc)。
如果您的贡献与安全相关,也请查看[我们的安全政策](https://github.com/ClickHouse/ClickHouse/security/policy/)。

View File

@ -80,6 +80,7 @@
#include <Common/Elf.h>
#include <Server/MySQLHandlerFactory.h>
#include <Server/PostgreSQLHandlerFactory.h>
#include <Server/CertificateReloader.h>
#include <Server/ProtocolServerAdapter.h>
#include <Server/HTTP/HTTPServer.h>
#include <Interpreters/AsynchronousInsertQueue.h>
@ -972,7 +973,9 @@ if (ThreadFuzzer::instance().isEffective())
global_context->updateInterserverCredentials(*config);
CompressionCodecEncrypted::Configuration::instance().tryLoad(*config, "encryption_codecs");
#if USE_SSL
CertificateReloader::instance().tryLoad(*config);
#endif
ProfileEvents::increment(ProfileEvents::MainConfigLoads);
},
/* already_loaded = */ false); /// Reload it right now (initial loading)
@ -1362,6 +1365,16 @@ if (ThreadFuzzer::instance().isEffective())
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
}
if (servers.empty())
throw Exception("No servers started (add valid listen_host and 'tcp_port' or 'http_port' to configuration file.)",
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
#if USE_SSL
CertificateReloader::instance().tryLoad(config());
#endif
/// Must be done after initialization of `servers`, because async_metrics will access `servers` variable from its thread.
async_metrics.start();
{

View File

@ -244,7 +244,7 @@
openssl dhparam -out /etc/clickhouse-server/dhparam.pem 4096
Only file format with BEGIN DH PARAMETERS is supported.
-->
<dhParamsFile>/etc/clickhouse-server/dhparam.pem</dhParamsFile>
<!-- <dhParamsFile>/etc/clickhouse-server/dhparam.pem</dhParamsFile> -->
<verificationMode>none</verificationMode>
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>

View File

@ -7,6 +7,7 @@
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
@ -27,14 +28,24 @@ struct StudentTTestData : public TTestMoments<Float64>
{
static constexpr auto name = "studentTTest";
std::pair<Float64, Float64> getResult() const
bool hasEnoughObservations() const
{
Float64 mean_x = x1 / nx;
Float64 mean_y = y1 / ny;
return nx > 0 && ny > 0 && nx + ny > 2;
}
Float64 getDegreesOfFreedom() const
{
return nx + ny - 2;
}
std::tuple<Float64, Float64> getResult() const
{
Float64 mean_x = getMeanX();
Float64 mean_y = getMeanY();
/// To estimate the variance we first estimate two means.
/// That's why the number of degrees of freedom is the total number of values of both samples minus 2.
Float64 degrees_of_freedom = nx + ny - 2;
Float64 degrees_of_freedom = getDegreesOfFreedom();
/// Calculate s^2
/// The original formulae looks like
@ -59,12 +70,14 @@ AggregateFunctionPtr createAggregateFunctionStudentTTest(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertBinary(name, argument_types);
assertNoParameters(name, parameters);
if (parameters.size() > 1)
throw Exception("Aggregate function " + name + " requires zero or one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!isNumber(argument_types[0]) || !isNumber(argument_types[1]))
throw Exception("Aggregate function " + name + " only supports numerical types", ErrorCodes::BAD_ARGUMENTS);
return std::make_shared<AggregateFunctionTTest<StudentTTestData>>(argument_types);
return std::make_shared<AggregateFunctionTTest<StudentTTestData>>(argument_types, parameters);
}
}

View File

@ -9,6 +9,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <cmath>
#include <cfloat>
/// This function is used in implementations of different T-Tests.
@ -28,6 +29,11 @@ struct Settings;
class ReadBuffer;
class WriteBuffer;
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
/**
* If you have a cumulative distribution function F, then calculating the p-value for given statistic T is simply 1F(T)
* In our case p-value is two-sided, so we multiply it by 2.
@ -79,10 +85,29 @@ template <typename Data>
class AggregateFunctionTTest :
public IAggregateFunctionDataHelper<Data, AggregateFunctionTTest<Data>>
{
private:
bool need_confidence_interval = false;
Float64 confidence_level;
public:
AggregateFunctionTTest(const DataTypes & arguments)
: IAggregateFunctionDataHelper<Data, AggregateFunctionTTest<Data>>({arguments}, {})
AggregateFunctionTTest(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<Data, AggregateFunctionTTest<Data>>({arguments}, params)
{
if (params.size() > 0)
{
need_confidence_interval = true;
confidence_level = params.at(0).safeGet<Float64>();
if (!std::isfinite(confidence_level))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} requires finite parameter values.", Data::name);
}
if (confidence_level <= 0.0 || confidence_level >= 1.0 || fabs(confidence_level - 0.0) < DBL_EPSILON || fabs(confidence_level - 1.0) < DBL_EPSILON)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Confidence level parameter must be between 0 and 1 in aggregate function {}.", Data::name);
}
}
}
String getName() const override
@ -92,22 +117,48 @@ public:
DataTypePtr getReturnType() const override
{
DataTypes types
if (need_confidence_interval)
{
std::make_shared<DataTypeNumber<Float64>>(),
std::make_shared<DataTypeNumber<Float64>>(),
};
DataTypes types
{
std::make_shared<DataTypeNumber<Float64>>(),
std::make_shared<DataTypeNumber<Float64>>(),
std::make_shared<DataTypeNumber<Float64>>(),
std::make_shared<DataTypeNumber<Float64>>(),
};
Strings names
Strings names
{
"t_statistic",
"p_value",
"confidence_interval_low",
"confidence_interval_high",
};
return std::make_shared<DataTypeTuple>(
std::move(types),
std::move(names)
);
}
else
{
"t_statistic",
"p_value"
};
DataTypes types
{
std::make_shared<DataTypeNumber<Float64>>(),
std::make_shared<DataTypeNumber<Float64>>(),
};
return std::make_shared<DataTypeTuple>(
std::move(types),
std::move(names)
);
Strings names
{
"t_statistic",
"p_value",
};
return std::make_shared<DataTypeTuple>(
std::move(types),
std::move(names)
);
}
}
bool allocatesMemoryInArena() const override { return false; }
@ -140,17 +191,46 @@ public:
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto [t_statistic, p_value] = this->data(place).getResult();
auto & data = this->data(place);
auto & column_tuple = assert_cast<ColumnTuple &>(to);
if (!data.hasEnoughObservations() || data.isEssentiallyConstant())
{
auto & column_stat = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(0));
auto & column_value = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(1));
column_stat.getData().push_back(std::numeric_limits<Float64>::quiet_NaN());
column_value.getData().push_back(std::numeric_limits<Float64>::quiet_NaN());
if (need_confidence_interval)
{
auto & column_ci_low = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(2));
auto & column_ci_high = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(3));
column_ci_low.getData().push_back(std::numeric_limits<Float64>::quiet_NaN());
column_ci_high.getData().push_back(std::numeric_limits<Float64>::quiet_NaN());
}
return;
}
auto [t_statistic, p_value] = data.getResult();
/// Because p-value is a probability.
p_value = std::min(1.0, std::max(0.0, p_value));
auto & column_tuple = assert_cast<ColumnTuple &>(to);
auto & column_stat = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(0));
auto & column_value = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(1));
column_stat.getData().push_back(t_statistic);
column_value.getData().push_back(p_value);
if (need_confidence_interval)
{
auto [ci_low, ci_high] = data.getConfidenceIntervals(confidence_level, data.getDegreesOfFreedom());
auto & column_ci_low = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(2));
auto & column_ci_high = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(3));
column_ci_low.getData().push_back(ci_low);
column_ci_high.getData().push_back(ci_high);
}
}
};

View File

@ -7,6 +7,7 @@
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
@ -21,34 +22,38 @@ struct WelchTTestData : public TTestMoments<Float64>
{
static constexpr auto name = "welchTTest";
std::pair<Float64, Float64> getResult() const
bool hasEnoughObservations() const
{
Float64 mean_x = x1 / nx;
Float64 mean_y = y1 / ny;
return nx > 1 && ny > 1;
}
/// s_x^2, s_y^2
/// The original formulae looks like \frac{1}{size_x - 1} \sum_{i = 1}^{size_x}{(x_i - \bar{x}) ^ 2}
/// But we made some mathematical transformations not to store original sequences.
/// Also we dropped sqrt, because later it will be squared later.
Float64 getDegreesOfFreedom() const
{
Float64 mean_x = getMeanX();
Float64 mean_y = getMeanY();
Float64 sx2 = (x2 + nx * mean_x * mean_x - 2 * mean_x * x1) / (nx - 1);
Float64 sy2 = (y2 + ny * mean_y * mean_y - 2 * mean_y * y1) / (ny - 1);
/// t-statistic
Float64 t_stat = (mean_x - mean_y) / sqrt(sx2 / nx + sy2 / ny);
/// degrees of freedom
Float64 numerator_sqrt = sx2 / nx + sy2 / ny;
Float64 numerator = numerator_sqrt * numerator_sqrt;
Float64 denominator_x = sx2 * sx2 / (nx * nx * (nx - 1));
Float64 denominator_y = sy2 * sy2 / (ny * ny * (ny - 1));
Float64 degrees_of_freedom = numerator / (denominator_x + denominator_y);
return numerator / (denominator_x + denominator_y);
}
return {t_stat, getPValue(degrees_of_freedom, t_stat * t_stat)};
std::tuple<Float64, Float64> getResult() const
{
Float64 mean_x = getMeanX();
Float64 mean_y = getMeanY();
/// t-statistic
Float64 se = getStandardError();
Float64 t_stat = (mean_x - mean_y) / se;
return {t_stat, getPValue(getDegreesOfFreedom(), t_stat * t_stat)};
}
};
@ -56,12 +61,14 @@ AggregateFunctionPtr createAggregateFunctionWelchTTest(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertBinary(name, argument_types);
assertNoParameters(name, parameters);
if (parameters.size() > 1)
throw Exception("Aggregate function " + name + " requires zero or one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!isNumber(argument_types[0]) || !isNumber(argument_types[1]))
throw Exception("Aggregate function " + name + " only supports numerical types", ErrorCodes::BAD_ARGUMENTS);
return std::make_shared<AggregateFunctionTTest<WelchTTestData>>(argument_types);
return std::make_shared<AggregateFunctionTTest<WelchTTestData>>(argument_types, parameters);
}
}

View File

@ -2,7 +2,9 @@
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <boost/math/distributions/students_t.hpp>
#include <boost/math/distributions/normal.hpp>
#include <cfloat>
namespace DB
@ -358,6 +360,50 @@ struct TTestMoments
{
readPODBinary(*this, buf);
}
Float64 getMeanX() const
{
return x1 / nx;
}
Float64 getMeanY() const
{
return y1 / ny;
}
Float64 getStandardError() const
{
/// The original formulae looks like \frac{1}{size_x - 1} \sum_{i = 1}^{size_x}{(x_i - \bar{x}) ^ 2}
/// But we made some mathematical transformations not to store original sequences.
/// Also we dropped sqrt, because later it will be squared later.
Float64 mean_x = getMeanX();
Float64 mean_y = getMeanY();
Float64 sx2 = (x2 + nx * mean_x * mean_x - 2 * mean_x * x1) / (nx - 1);
Float64 sy2 = (y2 + ny * mean_y * mean_y - 2 * mean_y * y1) / (ny - 1);
return sqrt(sx2 / nx + sy2 / ny);
}
std::pair<Float64, Float64> getConfidenceIntervals(Float64 confidence_level, Float64 degrees_of_freedom) const
{
Float64 mean_x = getMeanX();
Float64 mean_y = getMeanY();
Float64 se = getStandardError();
boost::math::students_t dist(degrees_of_freedom);
Float64 t = boost::math::quantile(boost::math::complement(dist, (1.0 - confidence_level) / 2.0));
Float64 mean_diff = mean_x - mean_y;
Float64 ci_low = mean_diff - t * se;
Float64 ci_high = mean_diff + t * se;
return {ci_low, ci_high};
}
bool isEssentiallyConstant() const
{
return getStandardError() < 10 * DBL_EPSILON * std::max(std::abs(getMeanX()), std::abs(getMeanY()));
}
};
template <typename T>

View File

@ -262,9 +262,7 @@ struct QuantileExactLow : public QuantileExactBase<Value, QuantileExactLow<Value
{
if (!array.empty())
{
// sort inputs in ascending order
::sort(array.begin(), array.end());
size_t n = 0;
// if level is 0.5 then compute the "low" median of the sorted array
// by the method of rounding.
if (level == 0.5)
@ -272,11 +270,11 @@ struct QuantileExactLow : public QuantileExactBase<Value, QuantileExactLow<Value
auto s = array.size();
if (s % 2 == 1)
{
return array[static_cast<size_t>(floor(s / 2))];
n = static_cast<size_t>(floor(s / 2));
}
else
{
return array[static_cast<size_t>((floor(s / 2)) - 1)];
n = static_cast<size_t>((floor(s / 2)) - 1);
}
}
else
@ -284,9 +282,10 @@ struct QuantileExactLow : public QuantileExactBase<Value, QuantileExactLow<Value
// else quantile is the nth index of the sorted array obtained by multiplying
// level and size of array. Example if level = 0.1 and size of array is 10,
// then return array[1].
size_t n = level < 1 ? level * array.size() : (array.size() - 1);
return array[n];
n = level < 1 ? level * array.size() : (array.size() - 1);
}
::nth_element(array.begin(), array.begin() + n, array.end());
return array[n];
}
return std::numeric_limits<Value>::quiet_NaN();
}
@ -295,12 +294,11 @@ struct QuantileExactLow : public QuantileExactBase<Value, QuantileExactLow<Value
{
if (!array.empty())
{
// sort inputs in ascending order
::sort(array.begin(), array.end());
size_t prev_n = 0;
for (size_t i = 0; i < size; ++i)
{
auto level = levels[indices[i]];
size_t n = 0;
// if level is 0.5 then compute the "low" median of the sorted array
// by the method of rounding.
if (level == 0.5)
@ -308,20 +306,22 @@ struct QuantileExactLow : public QuantileExactBase<Value, QuantileExactLow<Value
auto s = array.size();
if (s % 2 == 1)
{
result[indices[i]] = array[static_cast<size_t>(floor(s / 2))];
n = static_cast<size_t>(floor(s / 2));
}
else
{
result[indices[i]] = array[static_cast<size_t>(floor((s / 2) - 1))];
n = static_cast<size_t>(floor((s / 2) - 1));
}
}
else
{
// else quantile is the nth index of the sorted array obtained by multiplying
// level and size of array. Example if level = 0.1 and size of array is 10.
size_t n = level < 1 ? level * array.size() : (array.size() - 1);
result[indices[i]] = array[n];
n = level < 1 ? level * array.size() : (array.size() - 1);
}
::nth_element(array.begin() + prev_n, array.begin() + n, array.end());
result[indices[i]] = array[n];
prev_n = n;
}
}
else
@ -344,23 +344,22 @@ struct QuantileExactHigh : public QuantileExactBase<Value, QuantileExactHigh<Val
{
if (!array.empty())
{
// sort inputs in ascending order
::sort(array.begin(), array.end());
size_t n = 0;
// if level is 0.5 then compute the "high" median of the sorted array
// by the method of rounding.
if (level == 0.5)
{
auto s = array.size();
return array[static_cast<size_t>(floor(s / 2))];
n = static_cast<size_t>(floor(s / 2));
}
else
{
// else quantile is the nth index of the sorted array obtained by multiplying
// level and size of array. Example if level = 0.1 and size of array is 10.
size_t n = level < 1 ? level * array.size() : (array.size() - 1);
return array[n];
n = level < 1 ? level * array.size() : (array.size() - 1);
}
::nth_element(array.begin(), array.begin() + n, array.end());
return array[n];
}
return std::numeric_limits<Value>::quiet_NaN();
}
@ -369,26 +368,27 @@ struct QuantileExactHigh : public QuantileExactBase<Value, QuantileExactHigh<Val
{
if (!array.empty())
{
// sort inputs in ascending order
::sort(array.begin(), array.end());
size_t prev_n = 0;
for (size_t i = 0; i < size; ++i)
{
auto level = levels[indices[i]];
size_t n = 0;
// if level is 0.5 then compute the "high" median of the sorted array
// by the method of rounding.
if (level == 0.5)
{
auto s = array.size();
result[indices[i]] = array[static_cast<size_t>(floor(s / 2))];
n = static_cast<size_t>(floor(s / 2));
}
else
{
// else quantile is the nth index of the sorted array obtained by multiplying
// level and size of array. Example if level = 0.1 and size of array is 10.
size_t n = level < 1 ? level * array.size() : (array.size() - 1);
result[indices[i]] = array[n];
n = level < 1 ? level * array.size() : (array.size() - 1);
}
::nth_element(array.begin() + prev_n, array.begin() + n, array.end());
result[indices[i]] = array[n];
prev_n = n;
}
}
else

View File

@ -8,15 +8,22 @@
namespace DB
{
/**
* Write buffer with possibility to set and invoke callback after 'finalize' call.
* This buffer writes to cache, but after finalize() copy written file from cache to disk.
*/
class CompletionAwareWriteBuffer : public WriteBufferFromFileDecorator
class WritingToCacheWriteBuffer final : public WriteBufferFromFileDecorator
{
public:
CompletionAwareWriteBuffer(std::unique_ptr<WriteBufferFromFileBase> impl_, std::function<void()> completion_callback_)
: WriteBufferFromFileDecorator(std::move(impl_)), completion_callback(completion_callback_) { }
WritingToCacheWriteBuffer(
std::unique_ptr<WriteBufferFromFileBase> impl_,
std::function<std::unique_ptr<ReadBuffer>()> create_read_buffer_,
std::function<std::unique_ptr<WriteBuffer>()> create_write_buffer_)
: WriteBufferFromFileDecorator(std::move(impl_))
, create_read_buffer(std::move(create_read_buffer_))
, create_write_buffer(std::move(create_write_buffer_))
{
}
~CompletionAwareWriteBuffer() override
~WritingToCacheWriteBuffer() override
{
try
{
@ -28,15 +35,36 @@ public:
}
}
void preFinalize() override
{
impl->next();
impl->preFinalize();
impl->finalize();
read_buffer = create_read_buffer();
write_buffer = create_write_buffer();
copyData(*read_buffer, *write_buffer);
write_buffer->next();
write_buffer->preFinalize();
is_prefinalized = true;
}
void finalizeImpl() override
{
WriteBufferFromFileDecorator::finalizeImpl();
if (!is_prefinalized)
preFinalize();
completion_callback();
write_buffer->finalize();
}
private:
const std::function<void()> completion_callback;
std::function<std::unique_ptr<ReadBuffer>()> create_read_buffer;
std::function<std::unique_ptr<WriteBuffer>()> create_write_buffer;
std::unique_ptr<ReadBuffer> read_buffer;
std::unique_ptr<WriteBuffer> write_buffer;
bool is_prefinalized = false;
};
enum FileDownloadStatus
@ -165,21 +193,22 @@ DiskCacheWrapper::writeFile(const String & path, size_t buf_size, WriteMode mode
if (!cache_file_predicate(path))
return DiskDecorator::writeFile(path, buf_size, mode);
LOG_TRACE(log, "Write file {} to cache", backQuote(path));
LOG_TEST(log, "Write file {} to cache", backQuote(path));
auto dir_path = directoryPath(path);
if (!cache_disk->exists(dir_path))
cache_disk->createDirectories(dir_path);
return std::make_unique<CompletionAwareWriteBuffer>(
return std::make_unique<WritingToCacheWriteBuffer>(
cache_disk->writeFile(path, buf_size, mode),
[this, path, buf_size, mode]()
[this, path]()
{
/// Copy file from cache to actual disk when cached buffer is finalized.
auto src_buffer = cache_disk->readFile(path, ReadSettings(), /* read_hint= */ {}, /* file_size= */ {});
auto dst_buffer = DiskDecorator::writeFile(path, buf_size, mode);
copyData(*src_buffer, *dst_buffer);
dst_buffer->finalize();
return cache_disk->readFile(path, ReadSettings(), /* read_hint= */ {}, /* file_size= */ {});
},
[this, path, buf_size, mode]()
{
return DiskDecorator::writeFile(path, buf_size, mode);
});
}

View File

@ -151,6 +151,11 @@ void DiskDecorator::removeSharedFile(const String & path, bool keep_s3)
delegate->removeSharedFile(path, keep_s3);
}
void DiskDecorator::removeSharedFiles(const RemoveBatchRequest & files, bool keep_in_remote_fs)
{
delegate->removeSharedFiles(files, keep_in_remote_fs);
}
void DiskDecorator::removeSharedRecursive(const String & path, bool keep_s3)
{
delegate->removeSharedRecursive(path, keep_s3);

View File

@ -52,6 +52,7 @@ public:
void removeRecursive(const String & path) override;
void removeSharedFile(const String & path, bool keep_s3) override;
void removeSharedRecursive(const String & path, bool keep_s3) override;
void removeSharedFiles(const RemoveBatchRequest & files, bool keep_in_remote_fs) override;
void setLastModified(const String & path, const Poco::Timestamp & timestamp) override;
Poco::Timestamp getLastModified(const String & path) override;
void setReadOnly(const String & path) override;

View File

@ -234,6 +234,12 @@ void DiskRestartProxy::removeSharedFile(const String & path, bool keep_s3)
DiskDecorator::removeSharedFile(path, keep_s3);
}
void DiskRestartProxy::removeSharedFiles(const RemoveBatchRequest & files, bool keep_in_remote_fs)
{
ReadLock lock (mutex);
DiskDecorator::removeSharedFiles(files, keep_in_remote_fs);
}
void DiskRestartProxy::removeSharedRecursive(const String & path, bool keep_s3)
{
ReadLock lock (mutex);

View File

@ -54,6 +54,7 @@ public:
void removeDirectory(const String & path) override;
void removeRecursive(const String & path) override;
void removeSharedFile(const String & path, bool keep_s3) override;
void removeSharedFiles(const RemoveBatchRequest & files, bool keep_in_remote_fs) override;
void removeSharedRecursive(const String & path, bool keep_s3) override;
void setLastModified(const String & path, const Poco::Timestamp & timestamp) override;
Poco::Timestamp getLastModified(const String & path) override;

View File

@ -197,6 +197,32 @@ public:
/// Second bool param is a flag to remove (true) or keep (false) shared data on S3
virtual void removeSharedFileIfExists(const String & path, bool) { removeFileIfExists(path); }
struct RemoveRequest
{
String path;
bool if_exists = false;
explicit RemoveRequest(String path_, bool if_exists_ = false)
: path(std::move(path_)), if_exists(std::move(if_exists_))
{
}
};
using RemoveBatchRequest = std::vector<RemoveRequest>;
/// Batch request to remove multiple files.
/// May be much faster for blob storage.
virtual void removeSharedFiles(const RemoveBatchRequest & files, bool keep_in_remote_fs)
{
for (const auto & file : files)
{
if (file.if_exists)
removeSharedFileIfExists(file.path, keep_in_remote_fs);
else
removeSharedFile(file.path, keep_in_remote_fs);
}
}
/// Set last modified time to file or directory at `path`.
virtual void setLastModified(const String & path, const Poco::Timestamp & timestamp) = 0;

View File

@ -415,6 +415,19 @@ void IDiskRemote::removeSharedFileIfExists(const String & path, bool delete_meta
}
}
void IDiskRemote::removeSharedFiles(const RemoveBatchRequest & files, bool keep_in_remote_fs)
{
RemoteFSPathKeeperPtr fs_paths_keeper = createFSPathKeeper();
for (const auto & file : files)
{
bool skip = file.if_exists && !metadata_disk->exists(file.path);
if (!skip)
removeMeta(file.path, fs_paths_keeper);
}
if (!keep_in_remote_fs)
removeFromRemoteFS(fs_paths_keeper);
}
void IDiskRemote::removeSharedRecursive(const String & path, bool delete_metadata_only)
{
@ -555,4 +568,12 @@ UInt32 IDiskRemote::getRefCount(const String & path) const
return readMetadata(path).ref_count;
}
ThreadPool & IDiskRemote::getThreadPoolWriter()
{
constexpr size_t pool_size = 100;
constexpr size_t queue_size = 1000000;
static ThreadPool writer(pool_size, pool_size, queue_size);
return writer;
}
}

View File

@ -100,6 +100,8 @@ public:
void removeSharedFileIfExists(const String & path, bool delete_metadata_only) override;
void removeSharedFiles(const RemoveBatchRequest & files, bool keep_in_remote_fs) override;
void removeSharedRecursive(const String & path, bool delete_metadata_only) override;
void listFiles(const String & path, std::vector<String> & file_names) override;
@ -137,6 +139,7 @@ public:
virtual RemoteFSPathKeeperPtr createFSPathKeeper() const = 0;
static AsynchronousReaderPtr getThreadPoolReader();
static ThreadPool & getThreadPoolWriter();
DiskPtr getMetadataDiskIfExistsOrSelf() override { return metadata_disk; }

View File

@ -9,6 +9,7 @@
#include <boost/algorithm/string.hpp>
#include <base/scope_guard_safe.h>
#include <base/unit.h>
#include <base/FnTraits.h>
@ -261,6 +262,21 @@ std::unique_ptr<WriteBufferFromFileBase> DiskS3::writeFile(const String & path,
LOG_TRACE(log, "{} to file by path: {}. S3 path: {}",
mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_disk->getPath() + path), remote_fs_root_path + s3_path);
ScheduleFunc schedule = [pool = &getThreadPoolWriter()](auto callback)
{
pool->scheduleOrThrow([callback = std::move(callback), thread_group = CurrentThread::getGroup()]()
{
if (thread_group)
CurrentThread::attachTo(thread_group);
SCOPE_EXIT_SAFE(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
);
callback();
});
};
auto s3_buffer = std::make_unique<WriteBufferFromS3>(
settings->client,
bucket,
@ -268,7 +284,8 @@ std::unique_ptr<WriteBufferFromFileBase> DiskS3::writeFile(const String & path,
settings->s3_min_upload_part_size,
settings->s3_max_single_part_upload_size,
std::move(object_metadata),
buf_size);
buf_size,
std::move(schedule));
auto create_metadata_callback = [this, path, s3_path, mode] (size_t count)
{

View File

@ -104,10 +104,14 @@ public:
++pos;
}
virtual void sync()
{
next();
}
/// This method may be called before finalize() to tell there would not be any more data written.
/// Used does not have to call it, implementation should check it itself if needed.
///
/// The idea is similar to prefetch. In case if all data is written, we can flush the buffer
/// and start sending data asynchronously. It may improve writing performance in case you have
/// multiple files to finalize. Mainly, for blob storage, finalization has high latency,
/// and calling preFinalize in a loop may parallelize it.
virtual void preFinalize() { next(); }
/// Write the last data.
void finalize()
@ -130,6 +134,13 @@ public:
}
}
/// Wait for data to be reliably written. Mainly, call fsync for fd.
/// May be called after finalize() if needed.
virtual void sync()
{
next();
}
protected:
virtual void finalizeImpl()
{

View File

@ -14,6 +14,10 @@ WriteBufferFromFileDecorator::WriteBufferFromFileDecorator(std::unique_ptr<Write
void WriteBufferFromFileDecorator::finalizeImpl()
{
next();
if (!is_prefinalized)
WriteBufferFromFileDecorator::preFinalize();
impl->finalize();
}

View File

@ -17,6 +17,15 @@ public:
std::string getFileName() const override;
void preFinalize() override
{
next();
impl->preFinalize();
is_prefinalized = true;
}
const WriteBuffer & getImpl() const { return *impl; }
protected:
void finalizeImpl() override;
@ -24,6 +33,8 @@ protected:
private:
void nextImpl() override;
bool is_prefinalized = false;
};
}

View File

@ -34,6 +34,20 @@ namespace ErrorCodes
extern const int S3_ERROR;
}
struct WriteBufferFromS3::UploadPartTask
{
Aws::S3::Model::UploadPartRequest req;
bool is_finised = false;
std::string tag;
std::exception_ptr exception;
};
struct WriteBufferFromS3::PutObjectTask
{
Aws::S3::Model::PutObjectRequest req;
bool is_finised = false;
std::exception_ptr exception;
};
WriteBufferFromS3::WriteBufferFromS3(
std::shared_ptr<Aws::S3::S3Client> client_ptr_,
@ -42,7 +56,8 @@ WriteBufferFromS3::WriteBufferFromS3(
size_t minimum_upload_part_size_,
size_t max_single_part_upload_size_,
std::optional<std::map<String, String>> object_metadata_,
size_t buffer_size_)
size_t buffer_size_,
ScheduleFunc schedule_)
: BufferWithOwnMemory<WriteBuffer>(buffer_size_, nullptr, 0)
, bucket(bucket_)
, key(key_)
@ -50,6 +65,7 @@ WriteBufferFromS3::WriteBufferFromS3(
, client_ptr(std::move(client_ptr_))
, minimum_upload_part_size(minimum_upload_part_size_)
, max_single_part_upload_size(max_single_part_upload_size_)
, schedule(std::move(schedule_))
{
allocateBuffer();
}
@ -74,6 +90,8 @@ void WriteBufferFromS3::nextImpl()
writePart();
allocateBuffer();
}
waitForReadyBackGroundTasks();
}
void WriteBufferFromS3::allocateBuffer()
@ -88,7 +106,7 @@ WriteBufferFromS3::~WriteBufferFromS3()
finalize();
}
void WriteBufferFromS3::finalizeImpl()
void WriteBufferFromS3::preFinalize()
{
next();
@ -100,8 +118,20 @@ void WriteBufferFromS3::finalizeImpl()
{
/// Write rest of the data as last part.
writePart();
completeMultipartUpload();
}
is_prefinalized = true;
}
void WriteBufferFromS3::finalizeImpl()
{
if (!is_prefinalized)
preFinalize();
waitForAllBackGroundTasks();
if (!multipart_upload_id.empty())
completeMultipartUpload();
}
void WriteBufferFromS3::createMultipartUpload()
@ -144,22 +174,65 @@ void WriteBufferFromS3::writePart()
LOG_WARNING(log, "Maximum part number in S3 protocol has reached (too many parts). Server may not accept this whole upload.");
}
Aws::S3::Model::UploadPartRequest req;
if (schedule)
{
UploadPartTask * task = nullptr;
int part_number;
{
std::lock_guard lock(bg_tasks_mutex);
task = &upload_object_tasks.emplace_back();
++num_added_bg_tasks;
part_number = num_added_bg_tasks;
}
fillUploadRequest(task->req, part_number);
schedule([this, task]()
{
try
{
processUploadRequest(*task);
}
catch (...)
{
task->exception = std::current_exception();
}
{
std::lock_guard lock(bg_tasks_mutex);
task->is_finised = true;
++num_finished_bg_tasks;
}
bg_tasks_condvar.notify_one();
});
}
else
{
UploadPartTask task;
fillUploadRequest(task.req, part_tags.size() + 1);
processUploadRequest(task);
part_tags.push_back(task.tag);
}
}
void WriteBufferFromS3::fillUploadRequest(Aws::S3::Model::UploadPartRequest & req, int part_number)
{
req.SetBucket(bucket);
req.SetKey(key);
req.SetPartNumber(part_tags.size() + 1);
req.SetPartNumber(part_number);
req.SetUploadId(multipart_upload_id);
req.SetContentLength(size);
req.SetContentLength(temporary_buffer->tellp());
req.SetBody(temporary_buffer);
}
auto outcome = client_ptr->UploadPart(req);
void WriteBufferFromS3::processUploadRequest(UploadPartTask & task)
{
auto outcome = client_ptr->UploadPart(task.req);
if (outcome.IsSuccess())
{
auto etag = outcome.GetResult().GetETag();
part_tags.push_back(etag);
LOG_DEBUG(log, "Writing part finished. Bucket: {}, Key: {}, Upload_id: {}, Etag: {}, Parts: {}", bucket, key, multipart_upload_id, etag, part_tags.size());
task.tag = outcome.GetResult().GetETag();
LOG_DEBUG(log, "Writing part finished. Bucket: {}, Key: {}, Upload_id: {}, Etag: {}, Parts: {}", bucket, key, multipart_upload_id, task.tag, part_tags.size());
}
else
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
@ -191,14 +264,19 @@ void WriteBufferFromS3::completeMultipartUpload()
if (outcome.IsSuccess())
LOG_DEBUG(log, "Multipart upload has completed. Bucket: {}, Key: {}, Upload_id: {}, Parts: {}", bucket, key, multipart_upload_id, part_tags.size());
else
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
{
throw Exception(ErrorCodes::S3_ERROR, "{} Tags:{}",
outcome.GetError().GetMessage(),
fmt::join(part_tags.begin(), part_tags.end(), " "));
}
}
void WriteBufferFromS3::makeSinglepartUpload()
{
auto size = temporary_buffer->tellp();
bool with_pool = bool(schedule);
LOG_DEBUG(log, "Making single part upload. Bucket: {}, Key: {}, Size: {}", bucket, key, size);
LOG_DEBUG(log, "Making single part upload. Bucket: {}, Key: {}, Size: {}, WithPool: {}", bucket, key, size, with_pool);
if (size < 0)
throw Exception("Failed to make single part upload. Buffer in invalid state", ErrorCodes::S3_ERROR);
@ -209,22 +287,110 @@ void WriteBufferFromS3::makeSinglepartUpload()
return;
}
Aws::S3::Model::PutObjectRequest req;
if (schedule)
{
put_object_task = std::make_unique<PutObjectTask>();
fillPutRequest(put_object_task->req);
schedule([this]()
{
try
{
processPutRequest(*put_object_task);
}
catch (...)
{
put_object_task->exception = std::current_exception();
}
{
std::lock_guard lock(bg_tasks_mutex);
put_object_task->is_finised = true;
}
bg_tasks_condvar.notify_one();
});
}
else
{
PutObjectTask task;
fillPutRequest(task.req);
processPutRequest(task);
}
}
void WriteBufferFromS3::fillPutRequest(Aws::S3::Model::PutObjectRequest & req)
{
req.SetBucket(bucket);
req.SetKey(key);
req.SetContentLength(size);
req.SetContentLength(temporary_buffer->tellp());
req.SetBody(temporary_buffer);
if (object_metadata.has_value())
req.SetMetadata(object_metadata.value());
}
auto outcome = client_ptr->PutObject(req);
void WriteBufferFromS3::processPutRequest(PutObjectTask & task)
{
auto outcome = client_ptr->PutObject(task.req);
bool with_pool = bool(schedule);
if (outcome.IsSuccess())
LOG_DEBUG(log, "Single part upload has completed. Bucket: {}, Key: {}, Object size: {}", bucket, key, req.GetContentLength());
LOG_DEBUG(log, "Single part upload has completed. Bucket: {}, Key: {}, Object size: {}, WithPool: {}", bucket, key, task.req.GetContentLength(), with_pool);
else
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
}
void WriteBufferFromS3::waitForReadyBackGroundTasks()
{
if (schedule)
{
std::lock_guard lock(bg_tasks_mutex);
{
while (!upload_object_tasks.empty() && upload_object_tasks.front().is_finised)
{
auto & task = upload_object_tasks.front();
auto exception = std::move(task.exception);
auto tag = std::move(task.tag);
upload_object_tasks.pop_front();
if (exception)
{
waitForAllBackGroundTasks();
std::rethrow_exception(exception);
}
part_tags.push_back(tag);
}
}
}
}
void WriteBufferFromS3::waitForAllBackGroundTasks()
{
if (schedule)
{
std::unique_lock lock(bg_tasks_mutex);
bg_tasks_condvar.wait(lock, [this]() { return num_added_bg_tasks == num_finished_bg_tasks; });
while (!upload_object_tasks.empty())
{
auto & task = upload_object_tasks.front();
if (task.exception)
std::rethrow_exception(std::move(task.exception));
part_tags.push_back(task.tag);
upload_object_tasks.pop_front();
}
if (put_object_task)
{
bg_tasks_condvar.wait(lock, [this]() { return put_object_task->is_finised; });
if (put_object_task->exception)
std::rethrow_exception(std::move(put_object_task->exception));
}
}
}
}
#endif

View File

@ -14,14 +14,24 @@
# include <aws/core/utils/memory/stl/AWSStringStream.h>
# include <Common/ThreadPool.h>
namespace Aws::S3
{
class S3Client;
}
namespace Aws::S3::Model
{
class UploadPartRequest;
class PutObjectRequest;
}
namespace DB
{
using ScheduleFunc = std::function<void(std::function<void()>)>;
/**
* Buffer to write a data to a S3 object with specified bucket and key.
* If data size written to the buffer is less than 'max_single_part_upload_size' write is performed using singlepart upload.
@ -29,7 +39,7 @@ namespace DB
* Data is divided on chunks with size greater than 'minimum_upload_part_size'. Last chunk can be less than this threshold.
* Each chunk is written as a part to S3.
*/
class WriteBufferFromS3 : public BufferWithOwnMemory<WriteBuffer>
class WriteBufferFromS3 final : public BufferWithOwnMemory<WriteBuffer>
{
public:
explicit WriteBufferFromS3(
@ -39,12 +49,15 @@ public:
size_t minimum_upload_part_size_,
size_t max_single_part_upload_size_,
std::optional<std::map<String, String>> object_metadata_ = std::nullopt,
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE);
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
ScheduleFunc schedule_ = {});
~WriteBufferFromS3() override;
void nextImpl() override;
void preFinalize() override;
private:
void allocateBuffer();
@ -57,6 +70,17 @@ private:
/// Receives response from the server after sending all data.
void finalizeImpl() override;
struct UploadPartTask;
void fillUploadRequest(Aws::S3::Model::UploadPartRequest & req, int part_number);
void processUploadRequest(UploadPartTask & task);
struct PutObjectTask;
void fillPutRequest(Aws::S3::Model::PutObjectRequest & req);
void processPutRequest(PutObjectTask & task);
void waitForReadyBackGroundTasks();
void waitForAllBackGroundTasks();
String bucket;
String key;
std::optional<std::map<String, String>> object_metadata;
@ -72,6 +96,18 @@ private:
String multipart_upload_id;
std::vector<String> part_tags;
bool is_prefinalized = false;
/// Following fields are for background uploads in thread pool (if specified).
/// We use std::function to avoid dependency of Interpreters
ScheduleFunc schedule;
std::unique_ptr<PutObjectTask> put_object_task;
std::list<UploadPartTask> upload_object_tasks;
size_t num_added_bg_tasks = 0;
size_t num_finished_bg_tasks = 0;
std::mutex bg_tasks_mutex;
std::condition_variable bg_tasks_condvar;
Poco::Logger * log = &Poco::Logger::get("WriteBufferFromS3");
};

View File

@ -1929,6 +1929,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
{
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);
@ -1936,7 +1937,10 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
else
{
query_info.order_optimizer = std::make_shared<ReadInOrderOptimizer>(
analysis_result.order_by_elements_actions, getSortDescription(query, context), query_info.syntax_analyzer_result);
query,
analysis_result.order_by_elements_actions,
getSortDescription(query, context),
query_info.syntax_analyzer_result);
}
}
else
@ -1944,6 +1948,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
if (query_info.projection)
{
query_info.projection->order_optimizer = std::make_shared<ReadInOrderOptimizer>(
query,
query_info.projection->group_by_elements_actions,
getSortDescriptionFromGroupBy(query),
query_info.syntax_analyzer_result);
@ -1951,7 +1956,10 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
else
{
query_info.order_optimizer = std::make_shared<ReadInOrderOptimizer>(
analysis_result.group_by_elements_actions, getSortDescriptionFromGroupBy(query), query_info.syntax_analyzer_result);
query,
analysis_result.group_by_elements_actions,
getSortDescriptionFromGroupBy(query),
query_info.syntax_analyzer_result);
}
}

View File

@ -27,7 +27,7 @@ bool isLogicalFunction(const ASTFunction & func)
size_t countAtoms(const ASTPtr & node)
{
checkStackSize();
if (node->as<ASTIdentifier>())
if (node->as<ASTIdentifier>() || node->as<ASTLiteral>())
return 1;
const auto * func = node->as<ASTFunction>();

View File

@ -367,7 +367,6 @@ static ActionsDAGPtr createProjection(const Block & header)
Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
RangesInDataParts && parts_with_ranges,
const Names & column_names,
const ActionsDAGPtr & sorting_key_prefix_expr,
ActionsDAGPtr & out_projection,
const InputOrderInfoPtr & input_order_info)
{
@ -509,10 +508,19 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
if (need_preliminary_merge)
{
size_t fixed_prefix_size = input_order_info->order_key_fixed_prefix_descr.size();
size_t prefix_size = fixed_prefix_size + input_order_info->order_key_prefix_descr.size();
auto order_key_prefix_ast = metadata_snapshot->getSortingKey().expression_list_ast->clone();
order_key_prefix_ast->children.resize(prefix_size);
auto syntax_result = TreeRewriter(context).analyze(order_key_prefix_ast, metadata_snapshot->getColumns().getAllPhysical());
auto sorting_key_prefix_expr = ExpressionAnalyzer(order_key_prefix_ast, syntax_result, context).getActionsDAG(false);
const auto & sorting_columns = metadata_snapshot->getSortingKey().column_names;
SortDescription sort_description;
for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j)
sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j],
input_order_info->direction, 1);
for (size_t j = 0; j < prefix_size; ++j)
sort_description.emplace_back(sorting_columns[j], input_order_info->direction);
auto sorting_key_expr = std::make_shared<ExpressionActions>(sorting_key_prefix_expr);
@ -1055,17 +1063,9 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
}
else if ((settings.optimize_read_in_order || settings.optimize_aggregation_in_order) && input_order_info)
{
size_t prefix_size = input_order_info->order_key_prefix_descr.size();
auto order_key_prefix_ast = metadata_snapshot->getSortingKey().expression_list_ast->clone();
order_key_prefix_ast->children.resize(prefix_size);
auto syntax_result = TreeRewriter(context).analyze(order_key_prefix_ast, metadata_snapshot->getColumns().getAllPhysical());
auto sorting_key_prefix_expr = ExpressionAnalyzer(order_key_prefix_ast, syntax_result, context).getActionsDAG(false);
pipe = spreadMarkRangesAmongStreamsWithOrder(
std::move(result.parts_with_ranges),
column_names_to_read,
sorting_key_prefix_expr,
result_projection,
input_order_info);
}

View File

@ -173,7 +173,6 @@ private:
Pipe spreadMarkRangesAmongStreamsWithOrder(
RangesInDataParts && parts_with_ranges,
const Names & column_names,
const ActionsDAGPtr & sorting_key_prefix_expr,
ActionsDAGPtr & out_projection,
const InputOrderInfoPtr & input_order_info);

View File

@ -40,10 +40,12 @@ struct BlockIO
pipeline.reset();
}
void onException() const
void onException()
{
if (exception_callback)
exception_callback();
pipeline.reset();
}
private:

View File

@ -0,0 +1,136 @@
#include "CertificateReloader.h"
#if USE_SSL
#include <base/logger_useful.h>
#include <base/errnoToString.h>
#include <Poco/Net/Context.h>
#include <Poco/Net/SSLManager.h>
#include <Poco/Net/Utility.h>
namespace DB
{
namespace
{
/// Call set process for certificate.
int callSetCertificate(SSL * ssl, [[maybe_unused]] void * arg)
{
return CertificateReloader::instance().setCertificate(ssl);
}
}
namespace ErrorCodes
{
extern const int CANNOT_STAT;
}
/// This is callback for OpenSSL. It will be called on every connection to obtain a certificate and private key.
int CertificateReloader::setCertificate(SSL * ssl)
{
auto current = data.get();
if (!current)
return -1;
SSL_use_certificate(ssl, const_cast<X509 *>(current->cert.certificate()));
SSL_use_RSAPrivateKey(ssl, current->key.impl()->getRSA());
int err = SSL_check_private_key(ssl);
if (err != 1)
{
std::string msg = Poco::Net::Utility::getLastError();
LOG_ERROR(log, "Unusable key-pair {}", msg);
return -1;
}
return 1;
}
void CertificateReloader::init()
{
LOG_DEBUG(log, "Initializing certificate reloader.");
/// Set a callback for OpenSSL to allow get the updated cert and key.
auto* ctx = Poco::Net::SSLManager::instance().defaultServerContext()->sslContext();
SSL_CTX_set_cert_cb(ctx, callSetCertificate, nullptr);
init_was_not_made = false;
}
void CertificateReloader::tryLoad(const Poco::Util::AbstractConfiguration & config)
{
/// If at least one of the files is modified - recreate
std::string new_cert_path = config.getString("openSSL.server.certificateFile", "");
std::string new_key_path = config.getString("openSSL.server.privateKeyFile", "");
/// For empty paths (that means, that user doesn't want to use certificates)
/// no processing required
if (new_cert_path.empty() || new_key_path.empty())
{
LOG_INFO(log, "One of paths is empty. Cannot apply new configuration for certificates. Fill all paths and try again.");
}
else
{
bool cert_file_changed = cert_file.changeIfModified(std::move(new_cert_path), log);
bool key_file_changed = key_file.changeIfModified(std::move(new_key_path), log);
if (cert_file_changed || key_file_changed)
{
LOG_DEBUG(log, "Reloading certificate ({}) and key ({}).", cert_file.path, key_file.path);
data.set(std::make_unique<const Data>(cert_file.path, key_file.path));
LOG_INFO(log, "Reloaded certificate ({}) and key ({}).", cert_file.path, key_file.path);
}
/// If callback is not set yet
try
{
if (init_was_not_made)
init();
}
catch (...)
{
init_was_not_made = true;
LOG_ERROR(log, fmt::runtime(getCurrentExceptionMessage(false)));
}
}
}
CertificateReloader::Data::Data(std::string cert_path, std::string key_path)
: cert(cert_path), key(/* public key */ "", /* private key */ key_path)
{
}
bool CertificateReloader::File::changeIfModified(std::string new_path, Poco::Logger * logger)
{
std::error_code ec;
std::filesystem::file_time_type new_modification_time = std::filesystem::last_write_time(new_path, ec);
if (ec)
{
LOG_ERROR(logger, "Cannot obtain modification time for {} file {}, skipping update. {}",
description, new_path, errnoToString(ErrorCodes::CANNOT_STAT, ec.value()));
return false;
}
if (new_path != path || new_modification_time != modification_time)
{
path = new_path;
modification_time = new_modification_time;
return true;
}
return false;
}
}
#endif

View File

@ -0,0 +1,88 @@
#pragma once
#if !defined(ARCADIA_BUILD)
# include <Common/config.h>
#endif
#if USE_SSL
#include <string>
#include <filesystem>
#include <Poco/Logger.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <openssl/ssl.h>
#include <openssl/x509v3.h>
#include <Poco/Crypto/RSAKey.h>
#include <Poco/Crypto/X509Certificate.h>
#include <Common/MultiVersion.h>
namespace DB
{
/// The CertificateReloader singleton performs 2 functions:
/// 1. Dynamic reloading of TLS key-pair when requested by server:
/// Server config reloader notifies CertificateReloader when the config changes.
/// On changed config, CertificateReloader reloads certs from disk.
/// 2. Implement `SSL_CTX_set_cert_cb` to set certificate for a new connection:
/// OpenSSL invokes a callback to setup a connection.
class CertificateReloader
{
public:
using stat_t = struct stat;
/// Singleton
CertificateReloader(CertificateReloader const &) = delete;
void operator=(CertificateReloader const &) = delete;
static CertificateReloader & instance()
{
static CertificateReloader instance;
return instance;
}
/// Initialize the callback and perform the initial cert loading
void init();
/// Handle configuration reload
void tryLoad(const Poco::Util::AbstractConfiguration & config);
/// A callback for OpenSSL
int setCertificate(SSL * ssl);
private:
CertificateReloader()
{
}
Poco::Logger * log = &Poco::Logger::get("CertificateReloader");
struct File
{
const char * description;
File(const char * description_) : description(description_) {}
std::string path;
std::filesystem::file_time_type modification_time;
bool changeIfModified(std::string new_path, Poco::Logger * logger);
};
File cert_file{"certificate"};
File key_file{"key"};
struct Data
{
Poco::Crypto::X509Certificate cert;
Poco::Crypto::RSAKey key;
Data(std::string cert_path, std::string key_path);
};
MultiVersion<Data> data;
bool init_was_not_made = true;
};
}
#endif

View File

@ -300,7 +300,7 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDisk(
throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED);
if (hashing_out.count() != size)
throw Exception(ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART, "Unexpected size of file {}", path);
throw Exception(ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART, "Unexpected size of file {}, expected {} got {}", path, hashing_out.count(), size);
writePODBinary(hashing_out.getHash(), out);
@ -595,7 +595,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
CompressionCodecFactory::instance().get("NONE", {}));
part_out.write(block);
part_out.writeSuffixAndFinalizePart(new_projection_part);
part_out.finalizePart(new_projection_part, false);
new_projection_part->checksums.checkEqual(checksums, /* have_uncompressed = */ true);
new_data_part->addProjectionPart(projection_name, std::move(new_projection_part));
}
@ -619,7 +619,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
CompressionCodecFactory::instance().get("NONE", {}));
part_out.write(block);
part_out.writeSuffixAndFinalizePart(new_data_part);
part_out.finalizePart(new_data_part, false);
new_data_part->checksums.checkEqual(checksums, /* have_uncompressed = */ true);
return new_data_part;

View File

@ -99,7 +99,7 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Dis
initialized = true;
}
void IMergeTreeDataPart::MinMaxIndex::store(
IMergeTreeDataPart::MinMaxIndex::WrittenFiles IMergeTreeDataPart::MinMaxIndex::store(
const MergeTreeData & data, const DiskPtr & disk_, const String & part_path, Checksums & out_checksums) const
{
auto metadata_snapshot = data.getInMemoryMetadataPtr();
@ -108,10 +108,10 @@ void IMergeTreeDataPart::MinMaxIndex::store(
auto minmax_column_names = data.getMinMaxColumnsNames(partition_key);
auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key);
store(minmax_column_names, minmax_column_types, disk_, part_path, out_checksums);
return store(minmax_column_names, minmax_column_types, disk_, part_path, out_checksums);
}
void IMergeTreeDataPart::MinMaxIndex::store(
IMergeTreeDataPart::MinMaxIndex::WrittenFiles IMergeTreeDataPart::MinMaxIndex::store(
const Names & column_names,
const DataTypes & data_types,
const DiskPtr & disk_,
@ -122,6 +122,8 @@ void IMergeTreeDataPart::MinMaxIndex::store(
throw Exception("Attempt to store uninitialized MinMax index for part " + part_path + ". This is a bug.",
ErrorCodes::LOGICAL_ERROR);
WrittenFiles written_files;
for (size_t i = 0; i < column_names.size(); ++i)
{
String file_name = "minmax_" + escapeForFileName(column_names[i]) + ".idx";
@ -134,8 +136,11 @@ void IMergeTreeDataPart::MinMaxIndex::store(
out_hashing.next();
out_checksums.files[file_name].file_size = out_hashing.count();
out_checksums.files[file_name].file_hash = out_hashing.getHash();
out->finalize();
out->preFinalize();
written_files.emplace_back(std::move(out));
}
return written_files;
}
void IMergeTreeDataPart::MinMaxIndex::update(const Block & block, const Names & column_names)
@ -1280,6 +1285,7 @@ void IMergeTreeDataPart::remove() const
try
{
/// Remove each expected file in directory, then remove directory itself.
IDisk::RemoveBatchRequest request;
#if !defined(__clang__)
# pragma GCC diagnostic push
@ -1288,18 +1294,19 @@ void IMergeTreeDataPart::remove() const
for (const auto & [file, _] : checksums.files)
{
if (projection_directories.find(file) == projection_directories.end())
disk->removeSharedFile(fs::path(to) / file, *keep_shared_data);
request.emplace_back(fs::path(to) / file);
}
#if !defined(__clang__)
# pragma GCC diagnostic pop
#endif
for (const auto & file : {"checksums.txt", "columns.txt"})
disk->removeSharedFile(fs::path(to) / file, *keep_shared_data);
request.emplace_back(fs::path(to) / file);
disk->removeSharedFileIfExists(fs::path(to) / DEFAULT_COMPRESSION_CODEC_FILE_NAME, *keep_shared_data);
disk->removeSharedFileIfExists(fs::path(to) / DELETE_ON_DESTROY_MARKER_FILE_NAME, *keep_shared_data);
request.emplace_back(fs::path(to) / DEFAULT_COMPRESSION_CODEC_FILE_NAME, true);
request.emplace_back(fs::path(to) / DELETE_ON_DESTROY_MARKER_FILE_NAME, true);
disk->removeSharedFiles(request, *keep_shared_data);
disk->removeDirectory(to);
}
catch (...)
@ -1333,22 +1340,24 @@ void IMergeTreeDataPart::projectionRemove(const String & parent_to, bool keep_sh
try
{
/// Remove each expected file in directory, then remove directory itself.
IDisk::RemoveBatchRequest request;
#if !defined(__clang__)
# pragma GCC diagnostic push
# pragma GCC diagnostic ignored "-Wunused-variable"
#endif
for (const auto & [file, _] : checksums.files)
disk->removeSharedFile(fs::path(to) / file, keep_shared_data);
request.emplace_back(fs::path(to) / file);
#if !defined(__clang__)
# pragma GCC diagnostic pop
#endif
for (const auto & file : {"checksums.txt", "columns.txt"})
disk->removeSharedFile(fs::path(to) / file, keep_shared_data);
disk->removeSharedFileIfExists(fs::path(to) / DEFAULT_COMPRESSION_CODEC_FILE_NAME, keep_shared_data);
disk->removeSharedFileIfExists(fs::path(to) / DELETE_ON_DESTROY_MARKER_FILE_NAME, keep_shared_data);
request.emplace_back(fs::path(to) / file);
request.emplace_back(fs::path(to) / DEFAULT_COMPRESSION_CODEC_FILE_NAME, true);
request.emplace_back(fs::path(to) / DELETE_ON_DESTROY_MARKER_FILE_NAME, true);
disk->removeSharedFiles(request, keep_shared_data);
disk->removeSharedRecursive(to, keep_shared_data);
}
catch (...)

View File

@ -300,9 +300,11 @@ public:
{
}
using WrittenFiles = std::vector<std::unique_ptr<WriteBufferFromFileBase>>;
void load(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path);
void store(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const;
void store(const Names & column_names, const DataTypes & data_types, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const;
[[nodiscard]] WrittenFiles store(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const;
[[nodiscard]] WrittenFiles store(const Names & column_names, const DataTypes & data_types, const DiskPtr & disk_, const String & part_path, Checksums & checksums) const;
void update(const Block & block, const Names & column_names);
void merge(const MinMaxIndex & other);

View File

@ -32,7 +32,9 @@ public:
virtual void write(const Block & block, const IColumn::Permutation * permutation) = 0;
virtual void finish(IMergeTreeDataPart::Checksums & checksums, bool sync) = 0;
virtual void fillChecksums(IMergeTreeDataPart::Checksums & checksums) = 0;
virtual void finish(bool sync) = 0;
Columns releaseIndexColumns();
const MergeTreeIndexGranularity & getIndexGranularity() const { return index_granularity; }

View File

@ -478,9 +478,11 @@ void MergeTask::VerticalMergeStage::finalizeVerticalMergeForOneColumn() const
throw Exception("Cancelled merging parts", ErrorCodes::ABORTED);
ctx->executor.reset();
auto changed_checksums = ctx->column_to->writeSuffixAndGetChecksums(global_ctx->new_data_part, global_ctx->checksums_gathered_columns, ctx->need_sync);
auto changed_checksums = ctx->column_to->fillChecksums(global_ctx->new_data_part, global_ctx->checksums_gathered_columns);
global_ctx->checksums_gathered_columns.add(std::move(changed_checksums));
ctx->delayed_streams.emplace_back(std::move(ctx->column_to));
if (global_ctx->rows_written != ctx->column_elems_written)
{
throw Exception("Written " + toString(ctx->column_elems_written) + " elements of column " + column_name +
@ -505,9 +507,8 @@ void MergeTask::VerticalMergeStage::finalizeVerticalMergeForOneColumn() const
bool MergeTask::VerticalMergeStage::finalizeVerticalMergeForAllColumns() const
{
/// No need to execute this part if it is horizontal merge.
if (global_ctx->chosen_merge_algorithm != MergeAlgorithm::Vertical)
return false;
for (auto & stream : ctx->delayed_streams)
stream->finish(ctx->need_sync);
return false;
}
@ -633,9 +634,9 @@ bool MergeTask::MergeProjectionsStage::finalizeProjectionsAndWholeMerge() const
}
if (global_ctx->chosen_merge_algorithm != MergeAlgorithm::Vertical)
global_ctx->to->writeSuffixAndFinalizePart(global_ctx->new_data_part, ctx->need_sync);
global_ctx->to->finalizePart(global_ctx->new_data_part, ctx->need_sync);
else
global_ctx->to->writeSuffixAndFinalizePart(
global_ctx->to->finalizePart(
global_ctx->new_data_part, ctx->need_sync, &global_ctx->storage_columns, &global_ctx->checksums_gathered_columns);
global_ctx->promise.set_value(global_ctx->new_data_part);

View File

@ -268,6 +268,7 @@ private:
Float64 progress_before = 0;
std::unique_ptr<MergedColumnOnlyOutputStream> column_to{nullptr};
std::vector<std::unique_ptr<MergedColumnOnlyOutputStream>> delayed_streams;
size_t column_elems_written{0};
QueryPipeline column_parts_pipeline;
std::unique_ptr<PullingPipelineExecutor> executor;

View File

@ -125,12 +125,12 @@ void MergeTreeDataPartInMemory::flushToDisk(const String & base_path, const Stri
projection_compression_codec);
projection_out.write(projection_part->block);
projection_out.writeSuffixAndFinalizePart(projection_data_part);
projection_out.finalizePart(projection_data_part, false);
new_data_part->addProjectionPart(projection_name, std::move(projection_data_part));
}
}
out.writeSuffixAndFinalizePart(new_data_part);
out.finalizePart(new_data_part, false);
}
void MergeTreeDataPartInMemory::makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const

View File

@ -218,7 +218,7 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G
}
}
void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync)
void MergeTreeDataPartWriterCompact::fillDataChecksums(IMergeTreeDataPart::Checksums & checksums)
{
if (columns_buffer.size() != 0)
{
@ -253,6 +253,12 @@ void MergeTreeDataPartWriterCompact::finishDataSerialization(IMergeTreeDataPart:
marks.next();
addToChecksums(checksums);
plain_file->preFinalize();
marks_file->preFinalize();
}
void MergeTreeDataPartWriterCompact::finishDataSerialization(bool sync)
{
plain_file->finalize();
marks_file->finalize();
if (sync)
@ -356,16 +362,28 @@ size_t MergeTreeDataPartWriterCompact::ColumnsBuffer::size() const
return accumulated_columns.at(0)->size();
}
void MergeTreeDataPartWriterCompact::finish(IMergeTreeDataPart::Checksums & checksums, bool sync)
void MergeTreeDataPartWriterCompact::fillChecksums(IMergeTreeDataPart::Checksums & checksums)
{
// If we don't have anything to write, skip finalization.
if (!columns_list.empty())
finishDataSerialization(checksums, sync);
fillDataChecksums(checksums);
if (settings.rewrite_primary_key)
finishPrimaryIndexSerialization(checksums, sync);
fillPrimaryIndexChecksums(checksums);
finishSkipIndicesSerialization(checksums, sync);
fillSkipIndicesChecksums(checksums);
}
void MergeTreeDataPartWriterCompact::finish(bool sync)
{
// If we don't have anything to write, skip finalization.
if (!columns_list.empty())
finishDataSerialization(sync);
if (settings.rewrite_primary_key)
finishPrimaryIndexSerialization(sync);
finishSkipIndicesSerialization(sync);
}
}

View File

@ -20,11 +20,13 @@ public:
void write(const Block & block, const IColumn::Permutation * permutation) override;
void finish(IMergeTreeDataPart::Checksums & checksums, bool sync) override;
void fillChecksums(IMergeTreeDataPart::Checksums & checksums) override;
void finish(bool sync) override;
private:
/// Finish serialization of the data. Flush rows in buffer to disk, compute checksums.
void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync);
void fillDataChecksums(IMergeTreeDataPart::Checksums & checksums);
void finishDataSerialization(bool sync);
void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) override;

View File

@ -76,7 +76,7 @@ void MergeTreeDataPartWriterInMemory::calculateAndSerializePrimaryIndex(const Bl
}
}
void MergeTreeDataPartWriterInMemory::finish(IMergeTreeDataPart::Checksums & checksums, bool /* sync */)
void MergeTreeDataPartWriterInMemory::fillChecksums(IMergeTreeDataPart::Checksums & checksums)
{
/// If part is empty we still need to initialize block by empty columns.
if (!part_in_memory->block)

View File

@ -18,7 +18,8 @@ public:
/// You can write only one block. In-memory part can be written only at INSERT.
void write(const Block & block, const IColumn::Permutation * permutation) override;
void finish(IMergeTreeDataPart::Checksums & checksums, bool sync) override;
void fillChecksums(IMergeTreeDataPart::Checksums & checksums) override;
void finish(bool /*sync*/) override {}
private:
void calculateAndSerializePrimaryIndex(const Block & primary_index_block);

View File

@ -2,6 +2,7 @@
#include <Common/MemoryTrackerBlockerInThread.h>
#include <utility>
#include "IO/WriteBufferFromFileDecorator.h"
namespace DB
{
@ -10,13 +11,24 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
void MergeTreeDataPartWriterOnDisk::Stream::finalize()
void MergeTreeDataPartWriterOnDisk::Stream::preFinalize()
{
compressed.next();
/// 'compressed_buf' doesn't call next() on underlying buffer ('plain_hashing'). We should do it manually.
plain_hashing.next();
marks.next();
plain_file->preFinalize();
marks_file->preFinalize();
is_prefinalized = true;
}
void MergeTreeDataPartWriterOnDisk::Stream::finalize()
{
if (!is_prefinalized)
preFinalize();
plain_file->finalize();
marks_file->finalize();
}
@ -245,8 +257,7 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block
}
}
void MergeTreeDataPartWriterOnDisk::finishPrimaryIndexSerialization(
MergeTreeData::DataPart::Checksums & checksums, bool sync)
void MergeTreeDataPartWriterOnDisk::fillPrimaryIndexChecksums(MergeTreeData::DataPart::Checksums & checksums)
{
bool write_final_mark = (with_final_mark && data_written);
if (write_final_mark && compute_granularity)
@ -269,6 +280,14 @@ void MergeTreeDataPartWriterOnDisk::finishPrimaryIndexSerialization(
index_stream->next();
checksums.files["primary.idx"].file_size = index_stream->count();
checksums.files["primary.idx"].file_hash = index_stream->getHash();
index_file_stream->preFinalize();
}
}
void MergeTreeDataPartWriterOnDisk::finishPrimaryIndexSerialization(bool sync)
{
if (index_stream)
{
index_file_stream->finalize();
if (sync)
index_file_stream->sync();
@ -276,8 +295,7 @@ void MergeTreeDataPartWriterOnDisk::finishPrimaryIndexSerialization(
}
}
void MergeTreeDataPartWriterOnDisk::finishSkipIndicesSerialization(
MergeTreeData::DataPart::Checksums & checksums, bool sync)
void MergeTreeDataPartWriterOnDisk::fillSkipIndicesChecksums(MergeTreeData::DataPart::Checksums & checksums)
{
for (size_t i = 0; i < skip_indices.size(); ++i)
{
@ -288,8 +306,16 @@ void MergeTreeDataPartWriterOnDisk::finishSkipIndicesSerialization(
for (auto & stream : skip_indices_streams)
{
stream->finalize();
stream->preFinalize();
stream->addToChecksums(checksums);
}
}
void MergeTreeDataPartWriterOnDisk::finishSkipIndicesSerialization(bool sync)
{
for (auto & stream : skip_indices_streams)
{
stream->finalize();
if (sync)
stream->sync();
}

View File

@ -71,6 +71,10 @@ public:
std::unique_ptr<WriteBufferFromFileBase> marks_file;
HashingWriteBuffer marks;
bool is_prefinalized = false;
void preFinalize();
void finalize();
void sync() const;
@ -107,9 +111,11 @@ protected:
void calculateAndSerializeSkipIndices(const Block & skip_indexes_block, const Granules & granules_to_write);
/// Finishes primary index serialization: write final primary index row (if required) and compute checksums
void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums, bool sync);
void fillPrimaryIndexChecksums(MergeTreeData::DataPart::Checksums & checksums);
void finishPrimaryIndexSerialization(bool sync);
/// Finishes skip indices serialization: write all accumulated data to disk and compute checksums
void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksums, bool sync);
void fillSkipIndicesChecksums(MergeTreeData::DataPart::Checksums & checksums);
void finishSkipIndicesSerialization(bool sync);
/// Get global number of the current which we are writing (or going to start to write)
size_t getCurrentMark() const { return current_mark; }

View File

@ -5,6 +5,7 @@
#include <DataTypes/Serializations/ISerialization.h>
#include <Common/escapeForFileName.h>
#include <Columns/ColumnSparse.h>
#include <base/logger_useful.h>
namespace DB
{
@ -514,7 +515,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai
}
void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync)
void MergeTreeDataPartWriterWide::fillDataChecksums(IMergeTreeDataPart::Checksums & checksums)
{
const auto & global_settings = storage.getContext()->getSettingsRef();
ISerialization::SerializeBinaryBulkSettings serialize_settings;
@ -547,10 +548,19 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Ch
writeFinalMark(*it, offset_columns, serialize_settings.path);
}
}
for (auto & stream : column_streams)
{
stream.second->preFinalize();
stream.second->addToChecksums(checksums);
}
}
void MergeTreeDataPartWriterWide::finishDataSerialization(bool sync)
{
for (auto & stream : column_streams)
{
stream.second->finalize();
stream.second->addToChecksums(checksums);
if (sync)
stream.second->sync();
}
@ -574,16 +584,28 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Ch
}
void MergeTreeDataPartWriterWide::finish(IMergeTreeDataPart::Checksums & checksums, bool sync)
void MergeTreeDataPartWriterWide::fillChecksums(IMergeTreeDataPart::Checksums & checksums)
{
// If we don't have anything to write, skip finalization.
if (!columns_list.empty())
finishDataSerialization(checksums, sync);
fillDataChecksums(checksums);
if (settings.rewrite_primary_key)
finishPrimaryIndexSerialization(checksums, sync);
fillPrimaryIndexChecksums(checksums);
finishSkipIndicesSerialization(checksums, sync);
fillSkipIndicesChecksums(checksums);
}
void MergeTreeDataPartWriterWide::finish(bool sync)
{
// If we don't have anything to write, skip finalization.
if (!columns_list.empty())
finishDataSerialization(sync);
if (settings.rewrite_primary_key)
finishPrimaryIndexSerialization(sync);
finishSkipIndicesSerialization(sync);
}
void MergeTreeDataPartWriterWide::writeFinalMark(

View File

@ -29,12 +29,15 @@ public:
void write(const Block & block, const IColumn::Permutation * permutation) override;
void finish(IMergeTreeDataPart::Checksums & checksums, bool sync) final;
void fillChecksums(IMergeTreeDataPart::Checksums & checksums) final;
void finish(bool sync) final;
private:
/// Finish serialization of data: write final mark if required and compute checksums
/// Also validate written data in debug mode
void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync);
void fillDataChecksums(IMergeTreeDataPart::Checksums & checksums);
void finishDataSerialization(bool sync);
/// Write data of one column.
/// Return how many marks were written and

View File

@ -137,6 +137,12 @@ void updateTTL(
}
void MergeTreeDataWriter::TemporaryPart::finalize()
{
for (auto & stream : streams)
stream.finalizer.finish();
}
BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(
const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context)
{
@ -270,9 +276,10 @@ Block MergeTreeDataWriter::mergeBlock(
return block.cloneWithColumns(status.chunk.getColumns());
}
MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(
MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart(
BlockWithPartition & block_with_partition, const StorageMetadataPtr & metadata_snapshot, ContextPtr context)
{
TemporaryPart temp_part;
Block & block = block_with_partition.block;
static const String TMP_PREFIX = "tmp_insert_";
@ -343,7 +350,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(
/// If optimize_on_insert is true, block may become empty after merge.
/// There is no need to create empty part.
if (expected_size == 0)
return nullptr;
return temp_part;
DB::IMergeTreeDataPart::TTLInfos move_ttl_infos;
const auto & move_ttl_entries = metadata_snapshot->getMoveTTLs();
@ -419,30 +426,37 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(
auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0);
const auto & index_factory = MergeTreeIndexFactory::instance();
MergedBlockOutputStream out(new_data_part, metadata_snapshot,columns,
auto out = std::make_unique<MergedBlockOutputStream>(new_data_part, metadata_snapshot,columns,
index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec);
bool sync_on_insert = data_settings->fsync_after_insert;
out.writeWithPermutation(block, perm_ptr);
out->writeWithPermutation(block, perm_ptr);
for (const auto & projection : metadata_snapshot->getProjections())
{
auto projection_block = projection.calculate(block, context);
if (projection_block.rows())
new_data_part->addProjectionPart(
projection.name, writeProjectionPart(data, log, projection_block, projection, new_data_part.get()));
{
auto proj_temp_part = writeProjectionPart(data, log, projection_block, projection, new_data_part.get());
new_data_part->addProjectionPart(projection.name, std::move(proj_temp_part.part));
for (auto & stream : proj_temp_part.streams)
temp_part.streams.emplace_back(std::move(stream));
}
}
out.writeSuffixAndFinalizePart(new_data_part, sync_on_insert);
auto finalizer = out->finalizePartAsync(new_data_part, data_settings->fsync_after_insert);
temp_part.part = new_data_part;
temp_part.streams.emplace_back(TemporaryPart::Stream{.stream = std::move(out), .finalizer = std::move(finalizer)});
/// out.finish(new_data_part, std::move(written_files), sync_on_insert);
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterRows, block.rows());
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterUncompressedBytes, block.bytes());
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterCompressedBytes, new_data_part->getBytesOnDisk());
return new_data_part;
return temp_part;
}
MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeProjectionPartImpl(
MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl(
const String & part_name,
MergeTreeDataPartType part_type,
const String & relative_path,
@ -453,6 +467,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeProjectionPartImpl(
Block block,
const ProjectionDescription & projection)
{
TemporaryPart temp_part;
const StorageMetadataPtr & metadata_snapshot = projection.metadata;
MergeTreePartInfo new_part_info("all", 0, 0, 0);
auto new_data_part = data.createPart(
@ -525,24 +540,28 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeProjectionPartImpl(
/// either default lz4 or compression method with zero thresholds on absolute and relative part size.
auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0);
MergedBlockOutputStream out(
auto out = std::make_unique<MergedBlockOutputStream>(
new_data_part,
metadata_snapshot,
columns,
{},
MergeTreeIndices{},
compression_codec);
out.writeWithPermutation(block, perm_ptr);
out.writeSuffixAndFinalizePart(new_data_part);
out->writeWithPermutation(block, perm_ptr);
auto finalizer = out->finalizePartAsync(new_data_part, false);
temp_part.part = new_data_part;
temp_part.streams.emplace_back(TemporaryPart::Stream{.stream = std::move(out), .finalizer = std::move(finalizer)});
// out.finish(new_data_part, std::move(written_files), false);
ProfileEvents::increment(ProfileEvents::MergeTreeDataProjectionWriterRows, block.rows());
ProfileEvents::increment(ProfileEvents::MergeTreeDataProjectionWriterUncompressedBytes, block.bytes());
ProfileEvents::increment(ProfileEvents::MergeTreeDataProjectionWriterCompressedBytes, new_data_part->getBytesOnDisk());
return new_data_part;
return temp_part;
}
MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeProjectionPart(
MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPart(
MergeTreeData & data, Poco::Logger * log, Block block, const ProjectionDescription & projection, const IMergeTreeDataPart * parent_part)
{
String part_name = projection.name;
@ -574,7 +593,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeProjectionPart(
/// This is used for projection materialization process which may contain multiple stages of
/// projection part merges.
MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempProjectionPart(
MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempProjectionPart(
MergeTreeData & data,
Poco::Logger * log,
Block block,
@ -609,7 +628,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempProjectionPart(
projection);
}
MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeInMemoryProjectionPart(
MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeInMemoryProjectionPart(
const MergeTreeData & data,
Poco::Logger * log,
Block block,

View File

@ -10,6 +10,7 @@
#include <Interpreters/sortBlock.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergedBlockOutputStream.h>
namespace DB
@ -46,11 +47,28 @@ public:
*/
MergeTreeData::MutableDataPartPtr writeTempPart(BlockWithPartition & block, const StorageMetadataPtr & metadata_snapshot, bool optimize_on_insert);
MergeTreeData::MutableDataPartPtr
writeTempPart(BlockWithPartition & block, const StorageMetadataPtr & metadata_snapshot, ContextPtr context);
/// This structure contains not completely written temporary part.
/// Some writes may happen asynchronously, e.g. for blob storages.
/// You should call finalize() to wait until all data is written.
struct TemporaryPart
{
MergeTreeData::MutableDataPartPtr part;
struct Stream
{
std::unique_ptr<MergedBlockOutputStream> stream;
MergedBlockOutputStream::Finalizer finalizer;
};
std::vector<Stream> streams;
void finalize();
};
TemporaryPart writeTempPart(BlockWithPartition & block, const StorageMetadataPtr & metadata_snapshot, ContextPtr context);
/// For insertion.
static MergeTreeData::MutableDataPartPtr writeProjectionPart(
static TemporaryPart writeProjectionPart(
MergeTreeData & data,
Poco::Logger * log,
Block block,
@ -58,7 +76,7 @@ public:
const IMergeTreeDataPart * parent_part);
/// For mutation: MATERIALIZE PROJECTION.
static MergeTreeData::MutableDataPartPtr writeTempProjectionPart(
static TemporaryPart writeTempProjectionPart(
MergeTreeData & data,
Poco::Logger * log,
Block block,
@ -67,7 +85,7 @@ public:
size_t block_num);
/// For WriteAheadLog AddPart.
static MergeTreeData::MutableDataPartPtr writeInMemoryProjectionPart(
static TemporaryPart writeInMemoryProjectionPart(
const MergeTreeData & data,
Poco::Logger * log,
Block block,
@ -82,7 +100,7 @@ public:
const MergeTreeData::MergingParams & merging_params);
private:
static MergeTreeData::MutableDataPartPtr writeProjectionPartImpl(
static TemporaryPart writeProjectionPartImpl(
const String & part_name,
MergeTreeDataPartType part_type,
const String & relative_path,

View File

@ -375,17 +375,17 @@ void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & dis
partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *file);
}
void MergeTreePartition::store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const
std::unique_ptr<WriteBufferFromFileBase> MergeTreePartition::store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const
{
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage.getContext()).sample_block;
store(partition_key_sample, disk, part_path, checksums);
return store(partition_key_sample, disk, part_path, checksums);
}
void MergeTreePartition::store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const
std::unique_ptr<WriteBufferFromFileBase> MergeTreePartition::store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const
{
if (!partition_key_sample)
return;
return nullptr;
auto out = disk->writeFile(part_path + "partition.dat");
HashingWriteBuffer out_hashing(*out);
@ -395,7 +395,8 @@ void MergeTreePartition::store(const Block & partition_key_sample, const DiskPtr
out_hashing.next();
checksums.files["partition.dat"].file_size = out_hashing.count();
checksums.files["partition.dat"].file_hash = out_hashing.getHash();
out->finalize();
out->preFinalize();
return out;
}
void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context)

View File

@ -38,8 +38,10 @@ public:
void serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const;
void load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path);
void store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const;
void store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const;
/// Store functions return write buffer with written but not finalized data.
/// User must call finish() for returned object.
[[nodiscard]] std::unique_ptr<WriteBufferFromFileBase> store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const;
[[nodiscard]] std::unique_ptr<WriteBufferFromFileBase> store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const;
void assign(const MergeTreePartition & other) { value = other.value; }

View File

@ -7,6 +7,21 @@
namespace DB
{
MergeTreeSink::~MergeTreeSink() = default;
MergeTreeSink::MergeTreeSink(
StorageMergeTree & storage_,
StorageMetadataPtr metadata_snapshot_,
size_t max_parts_per_block_,
ContextPtr context_)
: SinkToStorage(metadata_snapshot_->getSampleBlock())
, storage(storage_)
, metadata_snapshot(metadata_snapshot_)
, max_parts_per_block(max_parts_per_block_)
, context(context_)
{
}
void MergeTreeSink::onStart()
{
/// Only check "too many parts" before write,
@ -14,22 +29,42 @@ void MergeTreeSink::onStart()
storage.delayInsertOrThrowIfNeeded();
}
void MergeTreeSink::onFinish()
{
finishDelayedChunk();
}
struct MergeTreeSink::DelayedChunk
{
struct Partition
{
MergeTreeDataWriter::TemporaryPart temp_part;
UInt64 elapsed_ns;
String block_dedup_token;
};
std::vector<Partition> partitions;
};
void MergeTreeSink::consume(Chunk chunk)
{
auto block = getHeader().cloneWithColumns(chunk.detachColumns());
String block_dedup_token;
auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context);
std::vector<MergeTreeSink::DelayedChunk::Partition> partitions;
for (auto & current_block : part_blocks)
{
Stopwatch watch;
String block_dedup_token;
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, metadata_snapshot, context);
auto temp_part = storage.writer.writeTempPart(current_block, metadata_snapshot, context);
UInt64 elapsed_ns = watch.elapsed();
/// If optimize_on_insert setting is true, current_block could become empty after merge
/// and we didn't create part.
if (!part)
if (!temp_part.part)
continue;
if (storage.getDeduplicationLog())
@ -44,15 +79,41 @@ void MergeTreeSink::consume(Chunk chunk)
}
}
/// Part can be deduplicated, so increment counters and add to part log only if it's really added
if (storage.renameTempPartAndAdd(part, &storage.increment, nullptr, storage.getDeduplicationLog(), block_dedup_token))
partitions.emplace_back(MergeTreeSink::DelayedChunk::Partition
{
PartLog::addNewPart(storage.getContext(), part, watch.elapsed());
.temp_part = std::move(temp_part),
.elapsed_ns = elapsed_ns,
.block_dedup_token = std::move(block_dedup_token)
});
}
finishDelayedChunk();
delayed_chunk = std::make_unique<MergeTreeSink::DelayedChunk>();
delayed_chunk->partitions = std::move(partitions);
}
void MergeTreeSink::finishDelayedChunk()
{
if (!delayed_chunk)
return;
for (auto & partition : delayed_chunk->partitions)
{
partition.temp_part.finalize();
auto & part = partition.temp_part.part;
/// Part can be deduplicated, so increment counters and add to part log only if it's really added
if (storage.renameTempPartAndAdd(part, &storage.increment, nullptr, storage.getDeduplicationLog(), partition.block_dedup_token))
{
PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns);
/// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'.
storage.background_operations_assignee.trigger();
}
}
delayed_chunk.reset();
}
}

View File

@ -16,20 +16,16 @@ class MergeTreeSink : public SinkToStorage
public:
MergeTreeSink(
StorageMergeTree & storage_,
const StorageMetadataPtr metadata_snapshot_,
StorageMetadataPtr metadata_snapshot_,
size_t max_parts_per_block_,
ContextPtr context_)
: SinkToStorage(metadata_snapshot_->getSampleBlock())
, storage(storage_)
, metadata_snapshot(metadata_snapshot_)
, max_parts_per_block(max_parts_per_block_)
, context(context_)
{
}
ContextPtr context_);
~MergeTreeSink() override;
String getName() const override { return "MergeTreeSink"; }
void consume(Chunk chunk) override;
void onStart() override;
void onFinish() override;
private:
StorageMergeTree & storage;
@ -37,6 +33,12 @@ private:
size_t max_parts_per_block;
ContextPtr context;
uint64_t chunk_dedup_seqnum = 0; /// input chunk ordinal number in case of dedup token
/// We can delay processing for previous chunk and start writing a new one.
struct DelayedChunk;
std::unique_ptr<DelayedChunk> delayed_chunk;
void finishDelayedChunk();
};
}

View File

@ -210,12 +210,12 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor
for (const auto & projection : metadata_snapshot->getProjections())
{
auto projection_block = projection.calculate(block, context);
auto temp_part = MergeTreeDataWriter::writeInMemoryProjectionPart(storage, log, projection_block, projection, part.get());
temp_part.finalize();
if (projection_block.rows())
part->addProjectionPart(
projection.name,
MergeTreeDataWriter::writeInMemoryProjectionPart(storage, log, projection_block, projection, part.get()));
part->addProjectionPart(projection.name, std::move(temp_part.part));
}
part_out.writeSuffixAndFinalizePart(part);
part_out.finalizePart(part, false);
min_block_number = std::min(min_block_number, part->info.min_block);
max_block_number = std::max(max_block_number, part->info.max_block);

View File

@ -51,7 +51,67 @@ void MergedBlockOutputStream::writeWithPermutation(const Block & block, const IC
writeImpl(block, permutation);
}
void MergedBlockOutputStream::writeSuffixAndFinalizePart(
struct MergedBlockOutputStream::Finalizer::Impl
{
IMergeTreeDataPartWriter & writer;
MergeTreeData::MutableDataPartPtr part;
std::vector<std::unique_ptr<WriteBufferFromFileBase>> written_files;
bool sync;
Impl(IMergeTreeDataPartWriter & writer_, MergeTreeData::MutableDataPartPtr part_, bool sync_)
: writer(writer_), part(std::move(part_)), sync(sync_) {}
void finish();
};
void MergedBlockOutputStream::Finalizer::finish()
{
if (impl)
impl->finish();
impl.reset();
}
void MergedBlockOutputStream::Finalizer::Impl::finish()
{
writer.finish(sync);
for (auto & file : written_files)
{
file->finalize();
if (sync)
file->sync();
}
part->storage.lockSharedData(*part);
}
MergedBlockOutputStream::Finalizer::~Finalizer()
{
try
{
finish();
}
catch (...)
{
tryLogCurrentException("MergedBlockOutputStream");
}
}
MergedBlockOutputStream::Finalizer::Finalizer(Finalizer &&) = default;
MergedBlockOutputStream::Finalizer & MergedBlockOutputStream::Finalizer::operator=(Finalizer &&) = default;
MergedBlockOutputStream::Finalizer::Finalizer(std::unique_ptr<Impl> impl_) : impl(std::move(impl_)) {}
void MergedBlockOutputStream::finalizePart(
MergeTreeData::MutableDataPartPtr & new_part,
bool sync,
const NamesAndTypesList * total_columns_list,
MergeTreeData::DataPart::Checksums * additional_column_checksums)
{
finalizePartAsync(new_part, sync, total_columns_list, additional_column_checksums).finish();
}
MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync(
MergeTreeData::MutableDataPartPtr & new_part,
bool sync,
const NamesAndTypesList * total_columns_list,
@ -64,7 +124,9 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart(
checksums = std::move(*additional_column_checksums);
/// Finish columns serialization.
writer->finish(checksums, sync);
writer->fillChecksums(checksums);
LOG_TRACE(&Poco::Logger::get("MergedBlockOutputStream"), "filled checksums {}", new_part->getNameWithState());
for (const auto & [projection_name, projection_part] : new_part->getProjectionParts())
checksums.addFile(
@ -84,8 +146,9 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart(
new_part->setSerializationInfos(serialization_infos);
}
auto finalizer = std::make_unique<Finalizer::Impl>(*writer, new_part, sync);
if (new_part->isStoredOnDisk())
finalizePartOnDisk(new_part, checksums, sync);
finalizer->written_files = finalizePartOnDisk(new_part, checksums);
new_part->rows_count = rows_count;
new_part->modification_time = time(nullptr);
@ -97,15 +160,15 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart(
if (default_codec != nullptr)
new_part->default_codec = default_codec;
new_part->storage.lockSharedData(*new_part);
return Finalizer(std::move(finalizer));
}
void MergedBlockOutputStream::finalizePartOnDisk(
MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDisk(
const MergeTreeData::DataPartPtr & new_part,
MergeTreeData::DataPart::Checksums & checksums,
bool sync)
MergeTreeData::DataPart::Checksums & checksums)
{
WrittenFiles written_files;
if (new_part->isProjectionPart())
{
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || isCompactPart(new_part))
@ -116,6 +179,8 @@ void MergedBlockOutputStream::finalizePartOnDisk(
count_out_hashing.next();
checksums.files["count.txt"].file_size = count_out_hashing.count();
checksums.files["count.txt"].file_hash = count_out_hashing.getHash();
count_out->preFinalize();
written_files.emplace_back(std::move(count_out));
}
}
else
@ -127,16 +192,21 @@ void MergedBlockOutputStream::finalizePartOnDisk(
writeUUIDText(new_part->uuid, out_hashing);
checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_size = out_hashing.count();
checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_hash = out_hashing.getHash();
out->finalize();
if (sync)
out->sync();
out->preFinalize();
written_files.emplace_back(std::move(out));
}
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
{
new_part->partition.store(storage, volume->getDisk(), part_path, checksums);
if (auto file = new_part->partition.store(storage, volume->getDisk(), part_path, checksums))
written_files.emplace_back(std::move(file));
if (new_part->minmax_idx->initialized)
new_part->minmax_idx->store(storage, volume->getDisk(), part_path, checksums);
{
auto files = new_part->minmax_idx->store(storage, volume->getDisk(), part_path, checksums);
for (auto & file : files)
written_files.emplace_back(std::move(file));
}
else if (rows_count)
throw Exception("MinMax index was not initialized for new non-empty part " + new_part->name
+ ". It is a bug.", ErrorCodes::LOGICAL_ERROR);
@ -149,9 +219,8 @@ void MergedBlockOutputStream::finalizePartOnDisk(
count_out_hashing.next();
checksums.files["count.txt"].file_size = count_out_hashing.count();
checksums.files["count.txt"].file_hash = count_out_hashing.getHash();
count_out->finalize();
if (sync)
count_out->sync();
count_out->preFinalize();
written_files.emplace_back(std::move(count_out));
}
}
@ -163,9 +232,8 @@ void MergedBlockOutputStream::finalizePartOnDisk(
new_part->ttl_infos.write(out_hashing);
checksums.files["ttl.txt"].file_size = out_hashing.count();
checksums.files["ttl.txt"].file_hash = out_hashing.getHash();
out->finalize();
if (sync)
out->sync();
out->preFinalize();
written_files.emplace_back(std::move(out));
}
if (!new_part->getSerializationInfos().empty())
@ -175,25 +243,24 @@ void MergedBlockOutputStream::finalizePartOnDisk(
new_part->getSerializationInfos().writeJSON(out_hashing);
checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_size = out_hashing.count();
checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_hash = out_hashing.getHash();
out->finalize();
if (sync)
out->sync();
out->preFinalize();
written_files.emplace_back(std::move(out));
}
{
/// Write a file with a description of columns.
auto out = volume->getDisk()->writeFile(fs::path(part_path) / "columns.txt", 4096);
new_part->getColumns().writeText(*out);
out->finalize();
if (sync)
out->sync();
out->preFinalize();
written_files.emplace_back(std::move(out));
}
if (default_codec != nullptr)
{
auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME, 4096);
DB::writeText(queryToString(default_codec->getFullCodecDesc()), *out);
out->finalize();
out->preFinalize();
written_files.emplace_back(std::move(out));
}
else
{
@ -205,10 +272,11 @@ void MergedBlockOutputStream::finalizePartOnDisk(
/// Write file with checksums.
auto out = volume->getDisk()->writeFile(fs::path(part_path) / "checksums.txt", 4096);
checksums.write(*out);
out->finalize();
if (sync)
out->sync();
out->preFinalize();
written_files.emplace_back(std::move(out));
}
return written_files;
}
void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Permutation * permutation)

View File

@ -32,11 +32,33 @@ public:
*/
void writeWithPermutation(const Block & block, const IColumn::Permutation * permutation);
/// Finalizer is a structure which is returned from by finalizePart().
/// Files from part may be written asynchronously, e.g. for blob storages.
/// You should call finish() to wait until all data is written.
struct Finalizer
{
struct Impl;
std::unique_ptr<Impl> impl;
explicit Finalizer(std::unique_ptr<Impl> impl_);
~Finalizer();
Finalizer(Finalizer &&);
Finalizer & operator=(Finalizer &&);
void finish();
};
/// Finalize writing part and fill inner structures
/// If part is new and contains projections, they should be added before invoking this method.
void writeSuffixAndFinalizePart(
Finalizer finalizePartAsync(
MergeTreeData::MutableDataPartPtr & new_part,
bool sync = false,
bool sync,
const NamesAndTypesList * total_columns_list = nullptr,
MergeTreeData::DataPart::Checksums * additional_column_checksums = nullptr);
void finalizePart(
MergeTreeData::MutableDataPartPtr & new_part,
bool sync,
const NamesAndTypesList * total_columns_list = nullptr,
MergeTreeData::DataPart::Checksums * additional_column_checksums = nullptr);
@ -46,10 +68,10 @@ private:
*/
void writeImpl(const Block & block, const IColumn::Permutation * permutation);
void finalizePartOnDisk(
using WrittenFiles = std::vector<std::unique_ptr<WriteBufferFromFileBase>>;
WrittenFiles finalizePartOnDisk(
const MergeTreeData::DataPartPtr & new_part,
MergeTreeData::DataPart::Checksums & checksums,
bool sync);
MergeTreeData::DataPart::Checksums & checksums);
NamesAndTypesList columns_list;
IMergeTreeDataPart::MinMaxIndex minmax_idx;

View File

@ -55,14 +55,13 @@ void MergedColumnOnlyOutputStream::write(const Block & block)
}
MergeTreeData::DataPart::Checksums
MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums(
MergedColumnOnlyOutputStream::fillChecksums(
MergeTreeData::MutableDataPartPtr & new_part,
MergeTreeData::DataPart::Checksums & all_checksums,
bool sync)
MergeTreeData::DataPart::Checksums & all_checksums)
{
/// Finish columns serialization.
MergeTreeData::DataPart::Checksums checksums;
writer->finish(checksums, sync);
writer->fillChecksums(checksums);
for (const auto & [projection_name, projection_part] : new_part->getProjectionParts())
checksums.addFile(
@ -85,4 +84,9 @@ MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums(
return checksums;
}
void MergedColumnOnlyOutputStream::finish(bool sync)
{
writer->finish(sync);
}
}

View File

@ -25,8 +25,11 @@ public:
Block getHeader() const { return header; }
void write(const Block & block) override;
MergeTreeData::DataPart::Checksums
writeSuffixAndGetChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums, bool sync = false);
fillChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums);
void finish(bool sync);
private:
Block header;

View File

@ -804,8 +804,12 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections()
const auto & projection = *ctx->projections_to_build[i];
auto projection_block = projection_squashes[i].add(projection.calculate(cur_block, ctx->context));
if (projection_block)
projection_parts[projection.name].emplace_back(MergeTreeDataWriter::writeTempProjectionPart(
*ctx->data, ctx->log, projection_block, projection, ctx->new_data_part.get(), ++block_num));
{
auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart(
*ctx->data, ctx->log, projection_block, projection, ctx->new_data_part.get(), ++block_num);
tmp_part.finalize();
projection_parts[projection.name].emplace_back(std::move(tmp_part.part));
}
}
(*ctx->mutate_entry)->rows_written += cur_block.rows();
@ -823,8 +827,10 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections()
auto projection_block = projection_squash.add({});
if (projection_block)
{
projection_parts[projection.name].emplace_back(MergeTreeDataWriter::writeTempProjectionPart(
*ctx->data, ctx->log, projection_block, projection, ctx->new_data_part.get(), ++block_num));
auto temp_part = MergeTreeDataWriter::writeTempProjectionPart(
*ctx->data, ctx->log, projection_block, projection, ctx->new_data_part.get(), ++block_num);
temp_part.finalize();
projection_parts[projection.name].emplace_back(std::move(temp_part.part));
}
}
@ -976,7 +982,7 @@ private:
ctx->mutating_executor.reset();
ctx->mutating_pipeline.reset();
static_pointer_cast<MergedBlockOutputStream>(ctx->out)->writeSuffixAndFinalizePart(ctx->new_data_part, ctx->need_sync);
static_pointer_cast<MergedBlockOutputStream>(ctx->out)->finalizePart(ctx->new_data_part, ctx->need_sync);
ctx->out.reset();
}
@ -1132,9 +1138,11 @@ private:
ctx->mutating_pipeline.reset();
auto changed_checksums =
static_pointer_cast<MergedColumnOnlyOutputStream>(ctx->out)->writeSuffixAndGetChecksums(
ctx->new_data_part, ctx->new_data_part->checksums, ctx->need_sync);
static_pointer_cast<MergedColumnOnlyOutputStream>(ctx->out)->fillChecksums(
ctx->new_data_part, ctx->new_data_part->checksums);
ctx->new_data_part->checksums.add(std::move(changed_checksums));
static_pointer_cast<MergedColumnOnlyOutputStream>(ctx->out)->finish(ctx->need_sync);
}
for (const auto & [rename_from, rename_to] : ctx->files_to_rename)

View File

@ -32,6 +32,17 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
struct ReplicatedMergeTreeSink::DelayedChunk
{
struct Partition
{
MergeTreeDataWriter::TemporaryPart temp_part;
UInt64 elapsed_ns;
String block_id;
};
std::vector<Partition> partitions;
};
ReplicatedMergeTreeSink::ReplicatedMergeTreeSink(
StorageReplicatedMergeTree & storage_,
@ -60,6 +71,8 @@ ReplicatedMergeTreeSink::ReplicatedMergeTreeSink(
quorum = 0;
}
ReplicatedMergeTreeSink::~ReplicatedMergeTreeSink() = default;
/// Allow to verify that the session in ZooKeeper is still alive.
static void assertSessionIsNotExpired(zkutil::ZooKeeperPtr & zookeeper)
@ -126,8 +139,6 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk)
{
auto block = getHeader().cloneWithColumns(chunk.detachColumns());
last_block_is_duplicate = false;
auto zookeeper = storage.getZooKeeper();
assertSessionIsNotExpired(zookeeper);
@ -140,6 +151,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk)
checkQuorumPrecondition(zookeeper);
auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context);
std::vector<ReplicatedMergeTreeSink::DelayedChunk::Partition> partitions;
String block_dedup_token;
for (auto & current_block : part_blocks)
@ -148,11 +160,11 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk)
/// Write part to the filesystem under temporary name. Calculate a checksum.
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, metadata_snapshot, context);
auto temp_part = storage.writer.writeTempPart(current_block, metadata_snapshot, context);
/// If optimize_on_insert setting is true, current_block could become empty after merge
/// and we didn't create part.
if (!part)
if (!temp_part.part)
continue;
String block_id;
@ -170,7 +182,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk)
block_dedup_token = fmt::format("{}_{}", dedup_token, chunk_dedup_seqnum);
++chunk_dedup_seqnum;
}
block_id = part->getZeroLevelPartBlockID(block_dedup_token);
block_id = temp_part.part->getZeroLevelPartBlockID(block_dedup_token);
LOG_DEBUG(log, "Wrote block with ID '{}', {} rows", block_id, current_block.block.rows());
}
else
@ -178,27 +190,63 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk)
LOG_DEBUG(log, "Wrote block with {} rows", current_block.block.rows());
}
UInt64 elapsed_ns = watch.elapsed();
partitions.emplace_back(ReplicatedMergeTreeSink::DelayedChunk::Partition{
.temp_part = std::move(temp_part),
.elapsed_ns = elapsed_ns,
.block_id = std::move(block_id)
});
}
finishDelayedChunk(zookeeper);
delayed_chunk = std::make_unique<ReplicatedMergeTreeSink::DelayedChunk>();
delayed_chunk->partitions = std::move(partitions);
/// If deduplicated data should not be inserted into MV, we need to set proper
/// value for `last_block_is_duplicate`, which is possible only after the part is committed.
/// Othervide we can delay commit.
/// TODO: we can also delay commit if there is no MVs.
if (!context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views)
finishDelayedChunk(zookeeper);
}
void ReplicatedMergeTreeSink::finishDelayedChunk(zkutil::ZooKeeperPtr & zookeeper)
{
if (!delayed_chunk)
return;
last_block_is_duplicate = false;
for (auto & partition : delayed_chunk->partitions)
{
partition.temp_part.finalize();
auto & part = partition.temp_part.part;
try
{
commitPart(zookeeper, part, block_id);
commitPart(zookeeper, part, partition.block_id);
last_block_is_duplicate = last_block_is_duplicate || part->is_duplicate;
/// Set a special error code if the block is duplicate
int error = (deduplicate && last_block_is_duplicate) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0;
PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus(error));
int error = (deduplicate && part->is_duplicate) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0;
PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus(error));
}
catch (...)
{
PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentException(__PRETTY_FUNCTION__));
PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentException(__PRETTY_FUNCTION__));
throw;
}
}
delayed_chunk.reset();
}
void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPtr & part)
{
last_block_is_duplicate = false;
/// NOTE: No delay in this case. That's Ok.
auto zookeeper = storage.getZooKeeper();
@ -356,7 +404,6 @@ void ReplicatedMergeTreeSink::commitPart(
if (storage.getActiveContainingPart(existing_part_name))
{
part->is_duplicate = true;
last_block_is_duplicate = true;
ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks);
if (quorum)
{
@ -531,6 +578,12 @@ void ReplicatedMergeTreeSink::onStart()
storage.delayInsertOrThrowIfNeeded(&storage.partial_shutdown_event);
}
void ReplicatedMergeTreeSink::onFinish()
{
auto zookeeper = storage.getZooKeeper();
assertSessionIsNotExpired(zookeeper);
finishDelayedChunk(zookeeper);
}
void ReplicatedMergeTreeSink::waitForQuorum(
zkutil::ZooKeeperPtr & zookeeper,

View File

@ -35,8 +35,11 @@ public:
// needed to set the special LogEntryType::ATTACH_PART
bool is_attach_ = false);
~ReplicatedMergeTreeSink() override;
void onStart() override;
void consume(Chunk chunk) override;
void onFinish() override;
String getName() const override { return "ReplicatedMergeTreeSink"; }
@ -90,6 +93,12 @@ private:
ContextPtr context;
UInt64 chunk_dedup_seqnum = 0; /// input chunk ordinal number in case of dedup token
/// We can delay processing for previous chunk and start writing a new one.
struct DelayedChunk;
std::unique_ptr<DelayedChunk> delayed_chunk;
void finishDelayedChunk(zkutil::ZooKeeperPtr & zookeeper);
};
}

View File

@ -6,6 +6,8 @@
#include <Interpreters/replaceAliasColumnsInQuery.h>
#include <Functions/IFunction.h>
#include <Interpreters/TableJoin.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTFunction.h>
namespace DB
{
@ -15,13 +17,152 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
namespace
{
ASTPtr getFixedPoint(const ASTPtr & ast)
{
const auto * func = ast->as<ASTFunction>();
if (!func || func->name != "equals")
return nullptr;
const auto & lhs = func->arguments->children[0];
const auto & rhs = func->arguments->children[1];
if (lhs->as<ASTLiteral>())
return rhs;
if (rhs->as<ASTLiteral>())
return lhs;
return nullptr;
}
size_t calculateFixedPrefixSize(
const ASTSelectQuery & query, const Names & sorting_key_columns)
{
ASTPtr condition;
if (query.where() && query.prewhere())
condition = makeASTFunction("and", query.where(), query.prewhere());
else if (query.where())
condition = query.where();
else if (query.prewhere())
condition = query.prewhere();
if (!condition)
return 0;
/// Convert condition to CNF for more convenient analysis.
auto cnf = TreeCNFConverter::tryConvertToCNF(condition);
if (!cnf)
return 0;
NameSet fixed_points;
/// If we met expression like 'column = x', where 'x' is literal,
/// in clause of size 1 in CNF, then we can guarantee
/// that in all filtered rows 'column' will be equal to 'x'.
cnf->iterateGroups([&](const auto & group)
{
if (group.size() == 1 && !group.begin()->negative)
{
auto fixed_point = getFixedPoint(group.begin()->ast);
if (fixed_point)
fixed_points.insert(fixed_point->getColumnName());
}
});
size_t prefix_size = 0;
for (const auto & column_name : sorting_key_columns)
{
if (!fixed_points.contains(column_name))
break;
++prefix_size;
}
return prefix_size;
}
/// Optimize in case of exact match with order key element
/// or in some simple cases when order key element is wrapped into monotonic function.
/// Returns on of {-1, 0, 1} - direction of the match. 0 means - doesn't match.
int matchSortDescriptionAndKey(
const ExpressionActions::Actions & actions,
const SortColumnDescription & sort_column,
const String & sorting_key_column)
{
/// If required order depend on collation, it cannot be matched with primary key order.
/// Because primary keys cannot have collations.
if (sort_column.collator)
return 0;
int current_direction = sort_column.direction;
/// For the path: order by (sort_column, ...)
if (sort_column.column_name == sorting_key_column)
return current_direction;
/// For the path: order by (function(sort_column), ...)
/// Allow only one simple monotonic functions with one argument
/// Why not allow multi monotonic functions?
bool found_function = false;
for (const auto & action : actions)
{
if (action.node->type != ActionsDAG::ActionType::FUNCTION)
continue;
if (found_function)
{
current_direction = 0;
break;
}
else
{
found_function = true;
}
if (action.node->children.size() != 1 || action.node->children.at(0)->result_name != sorting_key_column)
{
current_direction = 0;
break;
}
const auto & func = *action.node->function_base;
if (!func.hasInformationAboutMonotonicity())
{
current_direction = 0;
break;
}
auto monotonicity = func.getMonotonicityForRange(*func.getArgumentTypes().at(0), {}, {});
if (!monotonicity.is_monotonic)
{
current_direction = 0;
break;
}
else if (!monotonicity.is_positive)
{
current_direction *= -1;
}
}
if (!found_function)
current_direction = 0;
return current_direction;
}
}
ReadInOrderOptimizer::ReadInOrderOptimizer(
const ASTSelectQuery & query_,
const ManyExpressionActions & elements_actions_,
const SortDescription & required_sort_description_,
const TreeRewriterResultPtr & syntax_result)
: elements_actions(elements_actions_)
, required_sort_description(required_sort_description_)
, query(query_)
{
if (elements_actions.size() != required_sort_description.size())
throw Exception("Sizes of sort description and actions are mismatched", ErrorCodes::LOGICAL_ERROR);
@ -35,126 +176,88 @@ ReadInOrderOptimizer::ReadInOrderOptimizer(
array_join_result_to_source = syntax_result->array_join_result_to_source;
}
InputOrderInfoPtr ReadInOrderOptimizer::getInputOrder(const StorageMetadataPtr & metadata_snapshot, ContextPtr context, UInt64 limit) const
InputOrderInfoPtr ReadInOrderOptimizer::getInputOrderImpl(
const StorageMetadataPtr & metadata_snapshot,
const SortDescription & description,
const ManyExpressionActions & actions,
UInt64 limit) const
{
Names sorting_key_columns = metadata_snapshot->getSortingKeyColumns();
if (!metadata_snapshot->hasSortingKey())
return {};
auto sorting_key_columns = metadata_snapshot->getSortingKeyColumns();
int read_direction = description.at(0).direction;
size_t fixed_prefix_size = calculateFixedPrefixSize(query, sorting_key_columns);
size_t descr_prefix_size = std::min(description.size(), sorting_key_columns.size() - fixed_prefix_size);
SortDescription order_key_prefix_descr;
int read_direction = required_sort_description.at(0).direction;
order_key_prefix_descr.reserve(descr_prefix_size);
size_t prefix_size = std::min(required_sort_description.size(), sorting_key_columns.size());
auto aliased_columns = metadata_snapshot->getColumns().getAliases();
for (size_t i = 0; i < prefix_size; ++i)
for (size_t i = 0; i < descr_prefix_size; ++i)
{
if (forbidden_columns.count(required_sort_description[i].column_name))
if (forbidden_columns.count(description[i].column_name))
break;
/// Optimize in case of exact match with order key element
/// or in some simple cases when order key element is wrapped into monotonic function.
auto apply_order_judge = [&] (const ExpressionActions::Actions & actions, const String & sort_column)
{
/// If required order depend on collation, it cannot be matched with primary key order.
/// Because primary keys cannot have collations.
if (required_sort_description[i].collator)
return false;
int current_direction = matchSortDescriptionAndKey(
actions[i]->getActions(), description[i], sorting_key_columns[i + fixed_prefix_size]);
int current_direction = required_sort_description[i].direction;
/// For the path: order by (sort_column, ...)
if (sort_column == sorting_key_columns[i] && current_direction == read_direction)
{
return true;
}
/// For the path: order by (function(sort_column), ...)
/// Allow only one simple monotonic functions with one argument
/// Why not allow multi monotonic functions?
else
{
bool found_function = false;
for (const auto & action : actions)
{
if (action.node->type != ActionsDAG::ActionType::FUNCTION)
{
continue;
}
if (found_function)
{
current_direction = 0;
break;
}
else
found_function = true;
if (action.node->children.size() != 1 || action.node->children.at(0)->result_name != sorting_key_columns[i])
{
current_direction = 0;
break;
}
const auto & func = *action.node->function_base;
if (!func.hasInformationAboutMonotonicity())
{
current_direction = 0;
break;
}
auto monotonicity = func.getMonotonicityForRange(*func.getArgumentTypes().at(0), {}, {});
if (!monotonicity.is_monotonic)
{
current_direction = 0;
break;
}
else if (!monotonicity.is_positive)
current_direction *= -1;
}
if (!found_function)
current_direction = 0;
if (!current_direction || (i > 0 && current_direction != read_direction))
return false;
if (i == 0)
read_direction = current_direction;
return true;
}
};
const auto & actions = elements_actions[i]->getActions();
bool ok;
/// check if it's alias column
/// currently we only support alias column without any function wrapper
/// ie: `order by aliased_column` can have this optimization, but `order by function(aliased_column)` can not.
/// This suits most cases.
if (context->getSettingsRef().optimize_respect_aliases && aliased_columns.contains(required_sort_description[i].column_name))
{
auto column_expr = metadata_snapshot->getColumns().get(required_sort_description[i].column_name).default_desc.expression->clone();
replaceAliasColumnsInQuery(column_expr, metadata_snapshot->getColumns(), array_join_result_to_source, context);
auto syntax_analyzer_result = TreeRewriter(context).analyze(column_expr, metadata_snapshot->getColumns().getAll());
const auto expression_analyzer = ExpressionAnalyzer(column_expr, syntax_analyzer_result, context).getActions(true);
const auto & alias_actions = expression_analyzer->getActions();
ok = apply_order_judge(alias_actions, column_expr->getColumnName());
}
else
ok = apply_order_judge(actions, required_sort_description[i].column_name);
if (ok)
order_key_prefix_descr.push_back(required_sort_description[i]);
else
if (!current_direction || (i > 0 && current_direction != read_direction))
break;
if (i == 0)
read_direction = current_direction;
order_key_prefix_descr.push_back(required_sort_description[i]);
}
if (order_key_prefix_descr.empty())
return {};
return std::make_shared<InputOrderInfo>(std::move(order_key_prefix_descr), read_direction, limit);
SortDescription order_key_fixed_prefix_descr;
order_key_fixed_prefix_descr.reserve(fixed_prefix_size);
for (size_t i = 0; i < fixed_prefix_size; ++i)
order_key_fixed_prefix_descr.emplace_back(sorting_key_columns[i], read_direction);
return std::make_shared<InputOrderInfo>(
std::move(order_key_fixed_prefix_descr),
std::move(order_key_prefix_descr),
read_direction, limit);
}
InputOrderInfoPtr ReadInOrderOptimizer::getInputOrder(
const StorageMetadataPtr & metadata_snapshot, ContextPtr context, UInt64 limit) const
{
if (!metadata_snapshot->hasSortingKey())
return {};
auto aliased_columns = metadata_snapshot->getColumns().getAliases();
/// Replace alias column with proper expressions.
/// Currently we only support alias column without any function wrapper,
/// i.e.: `order by aliased_column` can have this optimization, but `order by function(aliased_column)` can not.
/// This suits most cases.
if (context->getSettingsRef().optimize_respect_aliases && !aliased_columns.empty())
{
SortDescription aliases_sort_description = required_sort_description;
ManyExpressionActions aliases_actions = elements_actions;
for (size_t i = 0; i < required_sort_description.size(); ++i)
{
if (!aliased_columns.contains(required_sort_description[i].column_name))
continue;
auto column_expr = metadata_snapshot->getColumns().get(required_sort_description[i].column_name).default_desc.expression->clone();
replaceAliasColumnsInQuery(column_expr, metadata_snapshot->getColumns(), array_join_result_to_source, context);
auto syntax_analyzer_result = TreeRewriter(context).analyze(column_expr, metadata_snapshot->getColumns().getAll());
auto expression_analyzer = ExpressionAnalyzer(column_expr, syntax_analyzer_result, context);
aliases_sort_description[i].column_name = column_expr->getColumnName();
aliases_actions[i] = expression_analyzer.getActions(true);
}
return getInputOrderImpl(metadata_snapshot, aliases_sort_description, aliases_actions, limit);
}
return getInputOrderImpl(metadata_snapshot, required_sort_description, elements_actions, limit);
}
}

View File

@ -18,6 +18,7 @@ class ReadInOrderOptimizer
{
public:
ReadInOrderOptimizer(
const ASTSelectQuery & query,
const ManyExpressionActions & elements_actions,
const SortDescription & required_sort_description,
const TreeRewriterResultPtr & syntax_result);
@ -25,10 +26,17 @@ public:
InputOrderInfoPtr getInputOrder(const StorageMetadataPtr & metadata_snapshot, ContextPtr context, UInt64 limit = 0) const;
private:
InputOrderInfoPtr getInputOrderImpl(
const StorageMetadataPtr & metadata_snapshot,
const SortDescription & description,
const ManyExpressionActions & actions,
UInt64 limit) const;
/// Actions for every element of order expression to analyze functions for monotonicity
ManyExpressionActions elements_actions;
NameSet forbidden_columns;
NameToNameMap array_join_result_to_source;
SortDescription required_sort_description;
const ASTSelectQuery & query;
};
}

View File

@ -87,19 +87,22 @@ struct FilterDAGInfo
struct InputOrderInfo
{
SortDescription order_key_fixed_prefix_descr;
SortDescription order_key_prefix_descr;
int direction;
UInt64 limit;
InputOrderInfo(const SortDescription & order_key_prefix_descr_, int direction_, UInt64 limit_)
: order_key_prefix_descr(order_key_prefix_descr_), direction(direction_), limit(limit_) {}
bool operator ==(const InputOrderInfo & other) const
InputOrderInfo(
const SortDescription & order_key_fixed_prefix_descr_,
const SortDescription & order_key_prefix_descr_,
int direction_, UInt64 limit_)
: order_key_fixed_prefix_descr(order_key_fixed_prefix_descr_)
, order_key_prefix_descr(order_key_prefix_descr_)
, direction(direction_), limit(limit_)
{
return order_key_prefix_descr == other.order_key_prefix_descr && direction == other.direction;
}
bool operator !=(const InputOrderInfo & other) const { return !(*this == other); }
bool operator==(const InputOrderInfo &) const = default;
};
class IMergeTreeDataPart;

View File

@ -7534,7 +7534,7 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP
/// TODO(ab): What projections should we add to the empty part? How can we make sure that it
/// won't block future merges? Perhaps we should also check part emptiness when selecting parts
/// to merge.
out.writeSuffixAndFinalizePart(new_data_part, sync_on_insert);
out.finalizePart(new_data_part, sync_on_insert);
try
{

View File

@ -2,8 +2,9 @@
import sys
import logging
import re
from github import Github
from typing import Tuple
from github import Github
from env_helper import GITHUB_RUN_ID, GITHUB_REPOSITORY, GITHUB_SERVER_URL
from pr_info import PRInfo
from get_robot_token import get_best_robot_token
@ -17,8 +18,10 @@ TRUSTED_ORG_IDS = {
54801242, # clickhouse
}
OK_TEST_LABEL = set(["can be tested", "release", "pr-documentation", "pr-doc-fix"])
OK_SKIP_LABELS = {"release", "pr-documentation", "pr-doc-fix"}
CAN_BE_TESTED_LABEL = "can be tested"
DO_NOT_TEST_LABEL = "do not test"
FORCE_TESTS_LABEL = "force tests"
# Individual trusted contirbutors who are not in any trusted organization.
# Can be changed in runtime: we will append users that we learned to be in
@ -100,29 +103,29 @@ def pr_is_by_trusted_user(pr_user_login, pr_user_orgs):
# Returns whether we should look into individual checks for this PR. If not, it
# can be skipped entirely.
def should_run_checks_for_pr(pr_info):
# Returns can_run, description, labels_state
def should_run_checks_for_pr(pr_info: PRInfo) -> Tuple[bool, str, str]:
# Consider the labels and whether the user is trusted.
print("Got labels", pr_info.labels)
force_labels = set(["force tests"]).intersection(pr_info.labels)
if force_labels:
return True, "Labeled '{}'".format(", ".join(force_labels))
if FORCE_TESTS_LABEL in pr_info.labels:
return True, f"Labeled '{FORCE_TESTS_LABEL}'", "pending"
if "do not test" in pr_info.labels:
return False, "Labeled 'do not test'"
if DO_NOT_TEST_LABEL in pr_info.labels:
return False, f"Labeled '{DO_NOT_TEST_LABEL}'", "success"
if "can be tested" not in pr_info.labels and not pr_is_by_trusted_user(
if CAN_BE_TESTED_LABEL not in pr_info.labels and not pr_is_by_trusted_user(
pr_info.user_login, pr_info.user_orgs
):
return False, "Needs 'can be tested' label"
return False, "Needs 'can be tested' label", "failure"
if (
"release" in pr_info.labels
or "pr-backport" in pr_info.labels
or "pr-cherrypick" in pr_info.labels
):
return False, "Don't try new checks for release/backports/cherry-picks"
if OK_SKIP_LABELS.intersection(pr_info.labels):
return (
False,
"Don't try new checks for release/backports/cherry-picks",
"success",
)
return True, "No special conditions apply"
return True, "No special conditions apply", "pending"
def check_pr_description(pr_info):
@ -205,7 +208,7 @@ if __name__ == "__main__":
logging.basicConfig(level=logging.INFO)
pr_info = PRInfo(need_orgs=True, pr_event_from_api=True)
can_run, description = should_run_checks_for_pr(pr_info)
can_run, description, labels_state = should_run_checks_for_pr(pr_info)
gh = Github(get_best_robot_token())
commit = get_commit(gh, pr_info.sha)
@ -231,7 +234,7 @@ if __name__ == "__main__":
if not can_run:
print("::notice ::Cannot run")
commit.create_status(
context=NAME, description=description, state="failure", target_url=url
context=NAME, description=description, state=labels_state, target_url=url
)
sys.exit(1)
else:

View File

@ -86,12 +86,18 @@ if __name__ == "__main__":
docker_image = get_image_with_version(temp_path, "clickhouse/style-test")
s3_helper = S3Helper("https://s3.amazonaws.com")
subprocess.check_output(
cmd = (
f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE "
f"--volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output "
f"{docker_image}",
f"{docker_image}"
)
logging.info("Is going to run the command: %s", cmd)
subprocess.check_call(
cmd,
shell=True,
)
state, description, test_results, additional_files = process_result(temp_path)
ch_helper = ClickHouseHelper()
mark_flaky_tests(ch_helper, NAME, test_results)

View File

@ -1,4 +1,5 @@
from bottle import request, route, run, response
from threading import Lock
# Endpoint can be configured to throw 500 error on N-th request attempt.
@ -6,6 +7,7 @@ from bottle import request, route, run, response
# Dict to the number of request should be failed.
cache = {}
mutex = Lock()
@route('/fail_request/<_request_number>')
@ -38,23 +40,34 @@ def delete(_bucket):
@route('/<_bucket>/<_path:path>', ['GET', 'POST', 'PUT', 'DELETE'])
def server(_bucket, _path):
if cache.get('request_number', None):
request_number = cache.pop('request_number') - 1
if request_number > 0:
cache['request_number'] = request_number
else:
response.status = 500
response.content_type = 'text/xml'
return '<?xml version="1.0" encoding="UTF-8"?><Error><Code>ExpectedError</Code><Message>Expected Error</Message><RequestId>txfbd566d03042474888193-00608d7537</RequestId></Error>'
if cache.get('throttle_request_number', None):
request_number = cache.pop('throttle_request_number') - 1
if request_number > 0:
cache['throttle_request_number'] = request_number
else:
response.status = 429
response.content_type = 'text/xml'
return '<?xml version="1.0" encoding="UTF-8"?><Error><Code>TooManyRequestsException</Code><Message>Please reduce your request rate.</Message><RequestId>txfbd566d03042474888193-00608d7538</RequestId></Error>'
# It's delete query for failed part
if _path.endswith('delete'):
response.set_header("Location", "http://minio1:9001/" + _bucket + '/' + _path)
response.status = 307
return 'Redirected'
mutex.acquire()
try:
if cache.get('request_number', None):
request_number = cache.pop('request_number') - 1
if request_number > 0:
cache['request_number'] = request_number
else:
response.status = 500
response.content_type = 'text/xml'
return '<?xml version="1.0" encoding="UTF-8"?><Error><Code>ExpectedError</Code><Message>Expected Error</Message><RequestId>txfbd566d03042474888193-00608d7537</RequestId></Error>'
if cache.get('throttle_request_number', None):
request_number = cache.pop('throttle_request_number') - 1
if request_number > 0:
cache['throttle_request_number'] = request_number
else:
response.status = 429
response.content_type = 'text/xml'
return '<?xml version="1.0" encoding="UTF-8"?><Error><Code>TooManyRequestsException</Code><Message>Please reduce your request rate.</Message><RequestId>txfbd566d03042474888193-00608d7538</RequestId></Error>'
finally:
mutex.release()
response.set_header("Location", "http://minio1:9001/" + _bucket + '/' + _path)
response.status = 307

View File

@ -0,0 +1,14 @@
<?xml version="1.0"?>
<clickhouse>
<https_port>8443</https_port>
<openSSL>
<server>
<certificateFile>/etc/clickhouse-server/config.d/first.crt</certificateFile>
<privateKeyFile>/etc/clickhouse-server/config.d/first.key</privateKeyFile>
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
</server>
</openSSL>
</clickhouse>

View File

@ -0,0 +1,19 @@
-----BEGIN CERTIFICATE-----
MIIDCzCCAfOgAwIBAgIUcA+y3LQyfpxlBzL7IQVKUfnhRncwDQYJKoZIhvcNAQEL
BQAwFDESMBAGA1UEAwwJbG9jYWxob3N0MCAXDTIyMDEyODExNTA1NloYDzIzMTIw
NDE4MTE1MDU2WjAUMRIwEAYDVQQDDAlsb2NhbGhvc3QwggEiMA0GCSqGSIb3DQEB
AQUAA4IBDwAwggEKAoIBAQCumguFo+M0AQ6SgyL8K2Kep3YFYaJUjU3Mm7rrYXP7
aGnm0Cvh3dqituSHF1ZoLgThqdBR4e/e5SZvS7ShCnFCBZpWSuhodp7qZy4ETqa8
1/TJUr2hQLH+GpldAeGxPuDJwsdEEk100l4UHWQYg0+kqAmkijWlXDrxJYzeZ5Q5
r/qxJN1kGxFnGYtlFjM3IpunXsREjcxjJmE4pDHp+Bkvkp0znajPJo8AE4pZ0zEQ
K/LfQSHh5BWSLw3SwGzHTTsHkn7KduaIppbYCG1j8/VEGKJIZMWUP4UbBfZ5Pl1+
tm7sPOKho+pu35pA/7keYEP1XxGSbHy4e8xO2DkIDSBRAgMBAAGjUzBRMB0GA1Ud
DgQWBBQQ1RwP+LO8L9WmeW9xijBbG93jTjAfBgNVHSMEGDAWgBQQ1RwP+LO8L9Wm
eW9xijBbG93jTjAPBgNVHRMBAf8EBTADAQH/MA0GCSqGSIb3DQEBCwUAA4IBAQAV
hqUMF75V8dEqCF61GsHizcL0R+KN36AjFckzZfMCk/H5UFIQ2L6vCsm8tmi6DlZQ
4VXlq+gCuAXuKWDe5yGNBxveKgdIltCJ85ZcySU1KON0OJIICW0lmOsFflqjD6cZ
tnP0FvFgoRRHW+lEQcgklXDF4taV9cF40xhNQ+TnzXSEmeg9fJQeC2rKLdwbpSAe
xfFPUvaKTF6w4caUqBojvTk0eKahva+kEtoeJ6KKPoBxkJsUzdL63V7P6FlktWut
0O3H4oFTZ6fNYKBKRwYeSZY30o8N6Lib0LihScLF7HWQNs0eyQ6m6v93KcJyK42F
wweXxFoWbdsITFYOdBmD
-----END CERTIFICATE-----

View File

@ -0,0 +1,28 @@
-----BEGIN PRIVATE KEY-----
MIIEvgIBADANBgkqhkiG9w0BAQEFAASCBKgwggSkAgEAAoIBAQCumguFo+M0AQ6S
gyL8K2Kep3YFYaJUjU3Mm7rrYXP7aGnm0Cvh3dqituSHF1ZoLgThqdBR4e/e5SZv
S7ShCnFCBZpWSuhodp7qZy4ETqa81/TJUr2hQLH+GpldAeGxPuDJwsdEEk100l4U
HWQYg0+kqAmkijWlXDrxJYzeZ5Q5r/qxJN1kGxFnGYtlFjM3IpunXsREjcxjJmE4
pDHp+Bkvkp0znajPJo8AE4pZ0zEQK/LfQSHh5BWSLw3SwGzHTTsHkn7KduaIppbY
CG1j8/VEGKJIZMWUP4UbBfZ5Pl1+tm7sPOKho+pu35pA/7keYEP1XxGSbHy4e8xO
2DkIDSBRAgMBAAECggEAZSiW2Fy1fCHIoZYcpOE2CBmZxVBlznr3wj3PtCQIIHbE
NJgTdI8m5vLzwFkDFOTkqyHJskcmxIsbE4xXIJ5+M/QvESPhNvTS6ZfSD2jKLcso
5aNsfoqPFVuv0zUN37VAY2TYMlYwTii7nQfSQGmDsTAyNgRlRGMFO0W4Mfrs4+Zd
ysoxdb+562DfKnqzTaqWIGXB7kW4bdUmQwK5dCmuj4m5yh0TknPM2w+jtI/O5mA9
pTG8p/te8b8qkrVaPyrApVNuBEonIOBfesFjnvjqIMquCutysii/hMkP6LbkXE+0
bpcVV8Rs1W0I1zU6veAh3ValDYpiWRGX/9IALfIMAQKBgQDjEqK0Qzh8kMhO2g2E
+zh32ZsMmt7/wmhYhfmunhYKgjFyLVmAsDVHDoC/wtimItljSdKJdPenBKloTzxh
L/EuP5Fqt6BIlkrwdiXurTXEVWrntNenzpUBxGeXSmJ4B4BFJhNpQj1ewASuKjrM
CrIwwhIJRq0MjsG8aOwWHOYNMQKBgQDE2DU+jjHN12rvF+68+8HE6Gx/op6POSiW
Jb+IJRFGSrntLEXRQgvj6Tybm+dbrTy+OtXAZRo7W9hUjf5eYav1lnLj/YguvAhy
/9x97edZ9CJjvW/fEpkRBXdNkvKfqaR8qQaQSlAJRu6syreXJbPgAQOwQWVIXGa8
N+TGIhz9IQKBgQCfLvY+to0HzhuOI5Css8yPQE5QlNVVqHyr6ifyAMLk1QZCy4Xe
ECkZShJ52+cy+GU7FIpycDwYqszz4fArFYfW6xtPG7FSkYGxdrH60xRJMbRDAOTZ
r5mH5p7UUYIcMO38C8g51wTcwnHFgrc7SRhH1BT+ybwQfJdWNJukmNexUQKBgQCI
eRHpLfKvsMNtwtz9X1qHZ1EZ6KgfylQuTTuOa4yffF2NZt186FqQB/vCMwPjVqc/
iFD8E9xs/Q9uCAgsbXEoUseS9Ar/w9PjzyqSkGeOwSk6l3NBaIaA+5YsTU4zjg0B
dLqdPThiRjBh0iYY/8XG700cXSqYUZ/UrLfK+om4oQKBgBWSH0guEZyx37kzrN9z
bIsjSgkYxuoD18fpifAyVDmJGhi3JK4fSNz9gvsCEpPTVYMh2++HB2Pm45oNeO/m
XxqJQS+/tOcFxLC3Goy274jH6LK+3AXE++W2jV/G9Rkgv3et7KiKXbwWnYW3qIA0
Sgm0PSmXIcMhsb03LCqfsQa1
-----END PRIVATE KEY-----

View File

@ -0,0 +1,19 @@
-----BEGIN CERTIFICATE-----
MIIDCzCCAfOgAwIBAgIUc5pIrYIv905gvuYtD/Y4LPDeXKswDQYJKoZIhvcNAQEL
BQAwFDESMBAGA1UEAwwJbG9jYWxob3N0MCAXDTIyMDEyODExNTE0NVoYDzIzMTIw
NDE4MTE1MTQ1WjAUMRIwEAYDVQQDDAlsb2NhbGhvc3QwggEiMA0GCSqGSIb3DQEB
AQUAA4IBDwAwggEKAoIBAQDrQQCvvoSpNjl1HvPgI9ST+UbjZGEYHLVPlLTVh/Q/
RIQg/zxVOWmuHOnFXRAyb2b7B8uuNBC8wzHaubLkP6wk8nBbhc5kK4ohhiFA3DD8
6DGXpxfZhlZ/x/mnQnb8T+PFSPXNfJxTer1RttBBzHSiRcG0cTkCPH0oIYBRbNAO
Ig7/76EGHhNNBLDgU7CaMpvOeefMVJ1qd5SYRDgLRvZa4Y3KWtA9WrTQmDTH6YzH
+bLnVBfUV5rs9nyM4B8pGNrezb/deFlzB9c8+FhFxxm8UjeOZZhzyro+7eToVpDf
btuYcmjgju7O3142/s2P29RTogCteA8PP4KHc6oekztDAgMBAAGjUzBRMB0GA1Ud
DgQWBBQ5/fotxXkvhLkBUMfzjurEbwukOzAfBgNVHSMEGDAWgBQ5/fotxXkvhLkB
UMfzjurEbwukOzAPBgNVHRMBAf8EBTADAQH/MA0GCSqGSIb3DQEBCwUAA4IBAQB+
PWMgxHoN0xqDdLq/3cqGoO+4iG4CZGUIG7VspnojjGrHKTA/0jvC2lu62q/Dxwsv
EXL/0Gt3ycfP0pbtVIcPjB2j3u+c9iJJ2R47Q/6FRwSiN7LpUohJvCMnTDpK19wj
p5TNZL5DCFzrOqINewOZBaAn4TpkdSfmZUA65KZe8qrOmw7YbVxX8HOP0RPPtA96
zbUtc2VjHT0lNMZmiTuuLYtKxV+dyyAqX48KxthNCCs0zP414nUXxymdE0MCSUss
565FXf+FrMA+owe1SlacX/IjCkgN2oauRMDXN+JDLXJUwDKVKOb3yObXyIJ0/b6E
+cGmwo/7m6CE5hoCSncE
-----END CERTIFICATE-----

View File

@ -0,0 +1,28 @@
-----BEGIN PRIVATE KEY-----
MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQDrQQCvvoSpNjl1
HvPgI9ST+UbjZGEYHLVPlLTVh/Q/RIQg/zxVOWmuHOnFXRAyb2b7B8uuNBC8wzHa
ubLkP6wk8nBbhc5kK4ohhiFA3DD86DGXpxfZhlZ/x/mnQnb8T+PFSPXNfJxTer1R
ttBBzHSiRcG0cTkCPH0oIYBRbNAOIg7/76EGHhNNBLDgU7CaMpvOeefMVJ1qd5SY
RDgLRvZa4Y3KWtA9WrTQmDTH6YzH+bLnVBfUV5rs9nyM4B8pGNrezb/deFlzB9c8
+FhFxxm8UjeOZZhzyro+7eToVpDfbtuYcmjgju7O3142/s2P29RTogCteA8PP4KH
c6oekztDAgMBAAECggEANQeZFQSYOOB9QTZx+ON6xsRZQ2bcMChAgqjdvoh/+UcD
lcCTJA7mEJZ558Bbp1LPXuTZ9/HKmBJUCZ70gVkM/+Mairb12ESsRXRLyKgZ7tiU
XUAQMzuCAhnc3+QumB+WE2Gn7uMZBgRT6riP51UkMXQR/w/KrwNdnw82MqSZnaWH
G2U+VWzrwloGDn4wdTm0egPMUCoF9kTW04HCTRZBg2a1DGCIOd2QcnYrsLiRQHmK
J8hfVVHDk+aaDmkYhJqyruTan51ifWziMOAcxTxuhDDtGOGXtMMlJOF9sskVT8CF
ToCSMJ8Rs+TPHqdBTy9l9n6kSki9Z/JFylMBClQsAQKBgQD6/FF5EF7WaVjs6gJt
DcOCPBAo8Se1YxFQ+lHIflswJs9O8u/wOej06o3c2S0zqj8v6aomhw4hYds2RaSa
fJ1jOS/7l+b3A4QndmZ6ZPBXc4mdWeUwcJ1snzi8of7D+BInAoXa9mwmOYp4u8DB
x+udumDr9zorR3aI6LXARF8+AQKBgQDv9DlBKXO7iqf2n3peb5QaLuxLvaiQM2+I
kA5xTakrODWE8+nZ0fTjrq5gVsYoIGaci6FlvX1yKsWiwy79ng+XBPNcz14FEhrn
xQFDWa/t2xEMruar3cQ0eqotlwe4yanM+/5SYu5kgm8qs9CUdr14vrV3MYB0moKb
YAg2qmgBQwKBgQDavGPU+qtsecuCTj9nA4PMUMRUqjdNIdXJmR8FePnH8UrjJ15t
Iksgh/qy6qM2T71Z6G7dvP5XoY0Gs5NNACW6f/CNeElWJb5bFhkhui6sSIk6lUnk
+YB5VhqAaz45VE2dqdk2h2Shu6wupJLNT4rMn84wV/peFZ38m7MqqWvIAQKBgCh4
CxP3VsKBfxR0DyJQNS05TrbzdLNlSWFB0n2/eFGGuFgE/yKya1ffBR/QYrkvxb6P
Ohg7niWcGxr5SjqR5tU0i4rSmmvGgu0l57GhNa+q67Q050iDLW0gZwUrXK0Ire+Z
bGoer1AaQ39zNjFj2U6880P4AE8qI+7qglgd406bAoGAVhnd9sA+hU6pG+WJ7JbX
TQoUUmO2BF3M2C3F6dje2LNsgtuZg+YkhG4RyBFuKOCZKdN6IhfoMhBw08qskB8J
a9vFyzygqEH4X2yhgmsMpb1dHOyCIvyPwlZIeGkzzJcXlcdx6cOQvmt1NLXPwrAz
GJbH6utej4bup+u4gDT5wEk=
-----END PRIVATE KEY-----

View File

@ -0,0 +1,75 @@
import pytest
import os
from helpers.cluster import ClickHouseCluster
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', main_configs=["configs/first.crt", "configs/first.key",
"configs/second.crt", "configs/second.key",
"configs/cert.xml"])
@pytest.fixture(scope="module", autouse=True)
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def change_config_to_key(name):
'''
* Generate config with certificate/key name from args.
* Reload config.
'''
node.exec_in_container(["bash", "-c" , """cat > /etc/clickhouse-server/config.d/cert.xml << EOF
<?xml version="1.0"?>
<clickhouse>
<https_port>8443</https_port>
<openSSL>
<server>
<certificateFile>/etc/clickhouse-server/config.d/{cur_name}.crt</certificateFile>
<privateKeyFile>/etc/clickhouse-server/config.d/{cur_name}.key</privateKeyFile>
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
</server>
</openSSL>
</clickhouse>
EOF""".format(cur_name=name)])
node.query("SYSTEM RELOAD CONFIG")
def test_first_than_second_cert():
''' Consistently set first key and check that only it will be accepted, then repeat same for second key. '''
# Set first key
change_config_to_key('first')
# Command with correct certificate
assert node.exec_in_container(['curl', '--silent', '--cacert', '/etc/clickhouse-server/config.d/{cur_name}.crt'.format(cur_name='first'),
'https://localhost:8443/']) == 'Ok.\n'
# Command with wrong certificate
# This command don't use option '-k', so it will lead to error while execution.
# That's why except will always work
try:
node.exec_in_container(['curl', '--silent', '--cacert', '/etc/clickhouse-server/config.d/{cur_name}.crt'.format(cur_name='second'),
'https://localhost:8443/'])
assert False
except:
assert True
# Change to other key
change_config_to_key('second')
# Command with correct certificate
assert node.exec_in_container(['curl', '--silent', '--cacert', '/etc/clickhouse-server/config.d/{cur_name}.crt'.format(cur_name='second'),
'https://localhost:8443/']) == 'Ok.\n'
# Command with wrong certificate
# Same as previous
try:
node.exec_in_container(['curl', '--silent', '--cacert', '/etc/clickhouse-server/config.d/{cur_name}.crt'.format(cur_name='first'),
'https://localhost:8443/'])
assert False
except:
assert True

View File

@ -28,6 +28,11 @@
<query>SELECT quantile(d64), quantileExact(d64), quantileExactWeighted(d64, 2) FROM (SELECT * FROM t LIMIT 1000000)</query>
<query>SELECT quantile(d128), quantileExact(d128), quantileExactWeighted(d128, 2) FROM (SELECT * FROM t LIMIT 1000000)</query>
<query>SELECT quantilesExactLow(0.5)(d32) FROM (SELECT * FROM t LIMIT 10000000)</query>
<query>SELECT quantilesExactHigh(0.5)(d32) FROM (SELECT * FROM t LIMIT 10000000)</query>
<query>SELECT quantilesExactLow(0.1, 0.5, 0.9)(d32) FROM (SELECT * FROM t LIMIT 10000000)</query>
<query>SELECT quantilesExactHigh(0.1, 0.5, 0.9)(d32) FROM (SELECT * FROM t LIMIT 10000000)</query>
<query>SELECT quantilesExact(0.1, 0.9)(d32), quantilesExactWeighted(0.1, 0.9)(d32, 2) FROM (SELECT * FROM t LIMIT 10000000)</query>
<query>SELECT quantilesExact(0.1, 0.9)(d64), quantilesExactWeighted(0.1, 0.9)(d64, 2) FROM (SELECT * FROM t LIMIT 1000000)</query>
<query>SELECT quantilesExact(0.1, 0.9)(d128), quantilesExactWeighted(0.1, 0.9)(d128, 2) FROM (SELECT * FROM t LIMIT 1000000)</query>

View File

@ -8,7 +8,9 @@
-0.5028215369187079 0.6152361677171103
14.971190998235835 5.898143508382202e-44
14.971190998235837 0
7.650530175770567 9.960200184229425
-2.610898982580138 0.00916587538237954
-2.610898982580134 0.0091658753823834
-28.740781574102936 7.667329672103986e-133
-28.74078157410298 0
-9.625938422388245 -8.395483817611758

File diff suppressed because one or more lines are too long

View File

@ -0,0 +1,155 @@
2020-10-01 0
2020-10-01 0
2020-10-01 0
2020-10-01 0
2020-10-01 0
(Expression)
ExpressionTransform
(Limit)
Limit
(Sorting)
MergingSortedTransform 2 → 1
(Expression)
ExpressionTransform × 2
(SettingQuotaAndLimits)
(ReadFromMergeTree)
MergeTreeInOrder × 2 0 → 1
2020-10-01 9
2020-10-01 9
2020-10-01 9
2020-10-01 9
2020-10-01 9
(Expression)
ExpressionTransform
(Limit)
Limit
(Sorting)
MergingSortedTransform 2 → 1
(Expression)
ExpressionTransform × 2
(SettingQuotaAndLimits)
(ReadFromMergeTree)
ReverseTransform
MergeTreeReverse 0 → 1
ReverseTransform
MergeTreeReverse 0 → 1
2020-10-01 9
2020-10-01 9
2020-10-01 9
2020-10-01 9
2020-10-01 9
(Expression)
ExpressionTransform
(Limit)
Limit
(Sorting)
FinishSortingTransform
PartialSortingTransform
MergingSortedTransform 2 → 1
(Expression)
ExpressionTransform × 2
(SettingQuotaAndLimits)
(ReadFromMergeTree)
MergeTreeInOrder × 2 0 → 1
2020-10-11 0
2020-10-11 0
2020-10-11 0
2020-10-11 0
2020-10-11 0
(Expression)
ExpressionTransform
(Limit)
Limit
(Sorting)
(Expression)
ExpressionTransform
(Filter)
FilterTransform
(SettingQuotaAndLimits)
(ReadFromMergeTree)
MergeTreeInOrder 0 → 1
2020-10-11 0 0
2020-10-11 0 10
2020-10-11 0 20
2020-10-11 0 30
2020-10-11 0 40
(Expression)
ExpressionTransform
(Limit)
Limit
(Sorting)
FinishSortingTransform
PartialSortingTransform
(Expression)
ExpressionTransform
(Filter)
FilterTransform
(SettingQuotaAndLimits)
(ReadFromMergeTree)
MergeTreeInOrder 0 → 1
2020-10-12 0
2020-10-12 1
2020-10-12 2
2020-10-12 3
2020-10-12 4
(Expression)
ExpressionTransform
(Limit)
Limit
(Sorting)
(Expression)
ExpressionTransform
(Filter)
FilterTransform
(SettingQuotaAndLimits)
(ReadFromMergeTree)
ReverseTransform
MergeTreeReverse 0 → 1
2020-10-12 99999
2020-10-12 99998
2020-10-12 99997
2020-10-12 99996
2020-10-12 99995
1 2
1 2
1 3
1 3
1 4
1 4
========
1 4
1 4
1 3
1 3
1 2
1 2
(Expression)
ExpressionTransform
(Limit)
Limit
(Sorting)
(Expression)
ExpressionTransform
(SettingQuotaAndLimits)
(ReadFromMergeTree)
MergeTreeInOrder 0 → 1
2020-10-10 00:00:00 0.01
2020-10-10 00:00:00 0.01
2020-10-10 00:00:00 0.01
2020-10-10 00:00:00 0.01
2020-10-10 00:00:00 0.01
(Expression)
ExpressionTransform
(Limit)
Limit
(Sorting)
(Expression)
ExpressionTransform
(SettingQuotaAndLimits)
(ReadFromMergeTree)
MergeTreeInOrder 0 → 1
2020-10-10 00:00:00 0.01
2020-10-10 00:00:00 0.01
2020-10-10 00:00:00 0.01
2020-10-10 00:00:00 0.01
2020-10-10 00:00:00 0.01

View File

@ -0,0 +1,58 @@
DROP TABLE IF EXISTS t_read_in_order;
CREATE TABLE t_read_in_order(date Date, i UInt64, v UInt64)
ENGINE = MergeTree ORDER BY (date, i);
INSERT INTO t_read_in_order SELECT '2020-10-10', number % 10, number FROM numbers(100000);
INSERT INTO t_read_in_order SELECT '2020-10-11', number % 10, number FROM numbers(100000);
SELECT toStartOfMonth(date) as d, i FROM t_read_in_order ORDER BY d, i LIMIT 5;
EXPLAIN PIPELINE SELECT toStartOfMonth(date) as d, i FROM t_read_in_order ORDER BY d, i LIMIT 5;
SELECT toStartOfMonth(date) as d, i FROM t_read_in_order ORDER BY d DESC, -i LIMIT 5;
EXPLAIN PIPELINE SELECT toStartOfMonth(date) as d, i FROM t_read_in_order ORDER BY d DESC, -i LIMIT 5;
-- Here FinishSorting is used, because directions don't match.
SELECT toStartOfMonth(date) as d, i FROM t_read_in_order ORDER BY d, -i LIMIT 5;
EXPLAIN PIPELINE SELECT toStartOfMonth(date) as d, i FROM t_read_in_order ORDER BY d, -i LIMIT 5;
SELECT date, i FROM t_read_in_order WHERE date = '2020-10-11' ORDER BY i LIMIT 5;
EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order WHERE date = '2020-10-11' ORDER BY i LIMIT 5;
SELECT * FROM t_read_in_order WHERE date = '2020-10-11' ORDER BY i, v LIMIT 5;
EXPLAIN PIPELINE SELECT * FROM t_read_in_order WHERE date = '2020-10-11' ORDER BY i, v LIMIT 5;
INSERT INTO t_read_in_order SELECT '2020-10-12', number, number FROM numbers(100000);
SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i LIMIT 5;
EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i DESC LIMIT 5;
SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i DESC LIMIT 5;
DROP TABLE IF EXISTS t_read_in_order;
CREATE TABLE t_read_in_order(a UInt32, b UInt32)
ENGINE = MergeTree ORDER BY (a, b)
SETTINGS index_granularity = 3;
SYSTEM STOP MERGES t_read_in_order;
INSERT INTO t_read_in_order VALUES (0, 100), (1, 2), (1, 3), (1, 4), (2, 5);
INSERT INTO t_read_in_order VALUES (0, 100), (1, 2), (1, 3), (1, 4), (2, 5);
SELECT a, b FROM t_read_in_order WHERE a = 1 ORDER BY b SETTINGS read_in_order_two_level_merge_threshold = 1;
SELECT '========';
SELECT a, b FROM t_read_in_order WHERE a = 1 ORDER BY b DESC SETTINGS read_in_order_two_level_merge_threshold = 1;
DROP TABLE t_read_in_order;
CREATE TABLE t_read_in_order(dt DateTime, d Decimal64(5), v UInt64)
ENGINE = MergeTree ORDER BY (toStartOfDay(dt), d);
INSERT INTO t_read_in_order SELECT toDateTime('2020-10-10 00:00:00') + number, 1 / (number % 100 + 1), number FROM numbers(1000);
EXPLAIN PIPELINE SELECT toStartOfDay(dt) as date, d FROM t_read_in_order ORDER BY date, round(d) LIMIT 5;
SELECT toStartOfDay(dt) as date, d FROM t_read_in_order ORDER BY date, round(d) LIMIT 5;
EXPLAIN PIPELINE SELECT toStartOfDay(dt) as date, d FROM t_read_in_order ORDER BY date, round(d) LIMIT 5;
SELECT toStartOfDay(dt) as date, d FROM t_read_in_order WHERE date = '2020-10-10' ORDER BY round(d) LIMIT 5;

View File

@ -0,0 +1,5 @@
MergeSorting
MergeSorting
MergeSorting
MergeSorting
MergeSorting

View File

@ -0,0 +1,20 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_read_in_order_2";
$CLICKHOUSE_CLIENT -q "CREATE TABLE t_read_in_order_2(date Date, i UInt64, v UInt64) ENGINE = MergeTree ORDER BY (date, i)"
$CLICKHOUSE_CLIENT -q "INSERT INTO t_read_in_order_2 SELECT '2020-10-10', number % 10, number FROM numbers(100000)"
$CLICKHOUSE_CLIENT -q "INSERT INTO t_read_in_order_2 SELECT '2020-10-11', number % 10, number FROM numbers(100000)"
$CLICKHOUSE_CLIENT -q "EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order_2 WHERE date = '2020-10-11' OR date = '2020-10-12' ORDER BY i DESC LIMIT 10" | grep -o "MergeSorting"
$CLICKHOUSE_CLIENT -q "EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order_2 WHERE date >= '2020-10-11' ORDER BY i DESC LIMIT 10" | grep -o "MergeSorting"
$CLICKHOUSE_CLIENT -q "EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order_2 WHERE date = '2020-10-11' OR v = 100 ORDER BY i DESC LIMIT 10" | grep -o "MergeSorting"
$CLICKHOUSE_CLIENT -q "EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order_2 WHERE date != '2020-10-11' ORDER BY i DESC LIMIT 10" | grep -o "MergeSorting"
$CLICKHOUSE_CLIENT -q "EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order_2 WHERE NOT (date = '2020-10-11') ORDER BY i DESC LIMIT 10" | grep -o "MergeSorting"
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_read_in_order_2";

View File

@ -1,5 +1,5 @@
#!/usr/bin/expect -f
# Tags: no-fasttest
# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-debug, no-fasttest
log_user 0
set timeout 60

3
utils/check-style/check-workflows Normal file → Executable file
View File

@ -1,5 +1,6 @@
#!/usr/bin/env bash
act --list 1>/dev/null 2>&1 || act --list 2>&1
GIT_ROOT=$(git rev-parse --show-cdup)
act --list --directory="$GIT_ROOT" 1>/dev/null 2>&1 || act --list --directory="$GIT_ROOT" 2>&1
actionlint

View File

@ -193,3 +193,11 @@ a.btn-outline-yellow {
.height-20 {
height: 20px;
}
ol.default{
margin-top: 10px;
}
ol.default li{
margin-bottom: 10px;
}

View File

@ -0,0 +1,27 @@
{% set prefetch_items = [
('/docs/en/', 'document')
] %}
{% extends "templates/base.html" %}
{% block extra_meta %}
<meta name="robots" content="noindex">
{% include "templates/common_fonts.html" %}
{% endblock %}
{% block nav %}
{% include "templates/global/nav.html" %}
{% endblock %}
{% block content %}
{% include "templates/support/policy-hero.html" %}
{% include "templates/support/policy-content.html" %}
{% include "templates/global/newsletter.html" %}
{% include "templates/global/github_stars.html" %}
{% endblock %}

View File

@ -0,0 +1,55 @@
<div id="contact" class="section pt-5">
<div class="container">
<p><strong>Effective Date: January 28, 2022</strong></p>
<p>This ClickHouse Support Services Policy sets forth ClickHouses policy for the provision of Support Services to its Subscription customers (each a “Customer”). Capitalized terms used but not defined herein, have the definition set forth in the applicable Subscription Agreement between ClickHouse and Customer (the “Agreement”).</p>
<ol class="default">
<li><strong>Defined Terms</strong>
<ol class="default" style="list-style-type:lower-alpha">
<li>"<u>Business Day</u>" means Monday through Friday other than a day designated from time to time as a national holiday in the place from which Support Services may be provided.</li>
<li><u>Long-Term Supported Release</u>” means the official release for the Software which has been designated as a release for long term support. ClickHouse designates its as Long-Term Supported Releases by including the letters“lts” in the release number, which are typically March and August releases.</li>
<li><u>Regular Stable Sequential Release</u>” means the official software release for the Software which has not been designated as a Long Term Supported Release.</li>
<li><u>Support</u>” means technical support by telephone, chat, or email provided by ClickHouse to a Support Contact concerning a Support Incident.</li>
<li><u>Support Contact</u>” means a single named individual who is authorized to contact ClickHouse to use the Support Services.</li>
<li>"<u>Support Incident</u>" means a single issue or error with the Software that is raised with ClickHouse by a Support Contact. Each Support Incident will be assigned a unique ID by ClickHouse. In the situation where multiple similar or equivalent cases are opened for a single Support Incident, ClickHouse may choose to consolidate these into a single support case, in which case it shall promptly notify Customer.</li>
</ol>
</li>
<li><strong>Scope and Performance of Support Services.</strong>
<p class="mt-2">The scope of the Support Services provided to Customer includes general assistance and support regarding the installation of the Software and basic technical configuration of the Software, as well as operational or development assistance on how to use the Software. ClickHouse shall use commercially reasonable efforts to meet the applicable target response times set forth in Section 3 below. Customer acknowledges that the time required for resolution of issues may vary depending on the specific circumstances of each problem, including, without limitation, the nature of the incident/problem, the extent and accuracy of information available about the incident/problem, and the level of Customer's cooperation and responsiveness in providing materials, information, access and support reasonably required by ClickHouse to achieve problem resolution.</p>
<ul>
<li><strong>Normal Business Hours:</strong>
<ul>
<li>24x7x365 for Severity 1 Issues</li>
<li>Monday 08:00 CEST through Friday 17:00 US Pacific time for Severity 2 and Severity 3 issues</li>
</ul>
</li>
<li><strong>Number of Support Contacts: unlimited</strong> </li>
<li><strong>Emergency Patches: yes</strong></li>
<li><strong>Annual Support Incidents: unlimited</strong></li>
</ul>
</li>
<li class="mt-2"><strong>Severity Levels and ClickHouse Efforts.</strong>
<p class="mt-2"><strong>Severity Level 1</strong></p>
<p>A Severity Level 1 issue is a major production error within the software that severely impacts the Customer's use of the software for production purposes, such as the loss of production data or where production systems are not functioning and no work-around exists. ClickHouse will use continuous efforts during applicable Normal Business Hours to provide a resolution for any Level 1 issues as soon as is commercially reasonable.</p>
<p><strong>Severity Level 2</strong></p>
<p>A Severity Level 2 issue is an error within the software where the customer's system is functioning for production purposes but in a reduced capacity, such as a problem that is causing significant impact to portions of the customer's business operations and productivity, or where the software is exposed to potential loss or interruption of service. ClickHouse will use continuous efforts during the Normal Business Hours to provide a resolution for any Severity Level 2 issues.</p>
<p><strong>Severity Level 3</strong></p>
<p>A Severity Level 3 issue is a medium-to-low impact error that involves partial and/or non-critical loss of functionality for production purposes or development purposes, such as a problem that impairs some operations but allows the customer's operations to continue to function. Errors for which there is limited or no loss or functionality or impact to the customer's operation and for which there is an easy work-around qualify as Severity Level 3. General questions are also Severity Level issues. ClickHouse will use reasonable efforts to provide a resolution for any Severity Level 3 issues in time for an upcoming release of the software. All inbound production email cases shall have an initial status of Severity Level 3.</p>
</li>
<li><strong>Customer Obligations.</strong>
<p class="mt-2">Customer agrees to provide ClickHouse with reasonable: (i) detail of the nature of and circumstances surrounding the issue, (ii) access to Customer's environment as necessary to enable ClickHouse to provide Support Services; and (iii) cooperation in the diagnosis and resolution of any issues.</p>
</li>
<li><strong>Supported Versions.</strong>
<p class="mt-2">Notwithstanding anything else, ClickHouse will support the current Regular Stable Release in conjunction with the two (2) prior Regular Stable Releases of the Software or a minimum period of three (3) months from the date of the current Regular Stable Release. ClickHouse will support the current Long-Term Supported Release in conjunction with the one (1) prior Long-Term Supported Release of the Software or a minimum period of one (1) year from the date of the current Long-Term Supported Release. </p>
</li>
<li><strong>Support Service Exclusions.</strong>
<p class="mt-2">ClickHouse will have no obligation to provide Support Services to Customer in the event that (i) the Software has been changed, modified or damaged by Customer or anyone other than ClickHouse, (ii) the problem is caused by Customer's negligence, misconduct, or misuse of the Software, a hardware malfunction, or other causes beyond the reasonable control of ClickHouse, (iii) the problem is due to third party software, (iv) the Software is being hosted by a third party that is offering the Software as a service (v) Customer has not installed or implemented any Software releases made generally available or is not running a then supported version of the Software as provided by ClickHouse or (vi) information requested by Customer could reasonably be expected to assist in the development, deployment, enablement and/or maintenance of any non-ClickHouse software that competes with ClickHouse's commercial software products. The Support Services do not cover the support of any third party software which integrates with the Software or the investigation into a potential or actual security incident in a Customer environment, including but not limited to the analysis and response to security events and signals. In addition, the Support Services do not include the following: (a) use of any version of a Software that is not designated as a production release (such as a milestone or release candidate or code contained in the sandbox or any other repository that is not packaged into a production release distribution); (b) Customer's failure to comply with operating instructions contained in the documentation for the Software; (c) installation, configuration, management and operation of Customer's applications; (d) APIs, interfaces or data formats other than those included with the Software; or (e) any training. </p>
</li>
<!-- <li><strong>Supported Platforms</strong>
<p class="mt-2">A list of supported platforms are set forth <a href="/support/platforms/">here.</a></p>
</li> -->
</ol>
</div>
</div>

View File

@ -0,0 +1,10 @@
<div class="hero bg-primary-light d-flex align-items-center base-hero">
<div class="hero-bg other-hero"></div>
<div class="container pt-8 pt-lg-10 pt-xl-15 pb-8 pb-lg-10 pb-xl-15">
<h1 class="display-1 mb-0">
{{ _('ClickHouse Support Services Policy') }}
</h1>
</div>
</div>