mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-15 02:41:59 +00:00
Merge branch 'master' of github.com:ClickHouse/ClickHouse into s3queue
This commit is contained in:
commit
841987a0a3
@ -731,7 +731,13 @@ The names given to the described entities can be found in the system tables, [sy
|
||||
|
||||
### Configuration {#table_engine-mergetree-multiple-volumes_configure}
|
||||
|
||||
Disks, volumes and storage policies should be declared inside the `<storage_configuration>` tag either in the main file `config.xml` or in a distinct file in the `config.d` directory.
|
||||
Disks, volumes and storage policies should be declared inside the `<storage_configuration>` tag either in a file in the `config.d` directory.
|
||||
|
||||
:::tip
|
||||
Disks can also be declared in the `SETTINGS` section of a query. This is useful
|
||||
for adhoc analysis to temporarily attach a disk that is, for example, hosted at a URL.
|
||||
See [dynamic storage](#dynamic-storage) for more details.
|
||||
:::
|
||||
|
||||
Configuration structure:
|
||||
|
||||
@ -876,6 +882,87 @@ You could change storage policy after table creation with [ALTER TABLE ... MODIF
|
||||
|
||||
The number of threads performing background moves of data parts can be changed by [background_move_pool_size](/docs/en/operations/server-configuration-parameters/settings.md/#background_move_pool_size) setting.
|
||||
|
||||
### Dynamic Storage
|
||||
|
||||
This example query shows how to attach a table stored at a URL and configure the
|
||||
remote storage within the query. The web storage is not configured in the ClickHouse
|
||||
configuration files; all the settings are in the CREATE/ATTACH query.
|
||||
|
||||
:::note
|
||||
The example uses `type=web`, but any disk type can be configured as dynamic, even Local disk. Local disks require a path argument to be inside the server config parameter `custom_local_disks_base_directory`, which has no default, so set that also when using local disk.
|
||||
:::
|
||||
|
||||
```sql
|
||||
ATTACH TABLE uk_price_paid UUID 'cf712b4f-2ca8-435c-ac23-c4393efe52f7'
|
||||
(
|
||||
price UInt32,
|
||||
date Date,
|
||||
postcode1 LowCardinality(String),
|
||||
postcode2 LowCardinality(String),
|
||||
type Enum8('other' = 0, 'terraced' = 1, 'semi-detached' = 2, 'detached' = 3, 'flat' = 4),
|
||||
is_new UInt8,
|
||||
duration Enum8('unknown' = 0, 'freehold' = 1, 'leasehold' = 2),
|
||||
addr1 String,
|
||||
addr2 String,
|
||||
street LowCardinality(String),
|
||||
locality LowCardinality(String),
|
||||
town LowCardinality(String),
|
||||
district LowCardinality(String),
|
||||
county LowCardinality(String)
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY (postcode1, postcode2, addr1, addr2)
|
||||
# highlight-start
|
||||
SETTINGS disk = disk(
|
||||
type=web,
|
||||
endpoint='https://raw.githubusercontent.com/ClickHouse/web-tables-demo/main/web/'
|
||||
);
|
||||
# highlight-end
|
||||
```
|
||||
|
||||
### Nested Dynamic Storage
|
||||
|
||||
This example query builds on the above dynamic disk configuration and shows how to
|
||||
use a local disk to cache data from a table stored at a URL. Neither the cache disk
|
||||
nor the web storage is configured in the ClickHouse configuration files; both are
|
||||
configured in the CREATE/ATTACH query settings.
|
||||
|
||||
In the settings highlighted below notice that the disk of `type=web` is nested within
|
||||
the disk of `type=cache`.
|
||||
|
||||
```sql
|
||||
ATTACH TABLE uk_price_paid UUID 'cf712b4f-2ca8-435c-ac23-c4393efe52f7'
|
||||
(
|
||||
price UInt32,
|
||||
date Date,
|
||||
postcode1 LowCardinality(String),
|
||||
postcode2 LowCardinality(String),
|
||||
type Enum8('other' = 0, 'terraced' = 1, 'semi-detached' = 2, 'detached' = 3, 'flat' = 4),
|
||||
is_new UInt8,
|
||||
duration Enum8('unknown' = 0, 'freehold' = 1, 'leasehold' = 2),
|
||||
addr1 String,
|
||||
addr2 String,
|
||||
street LowCardinality(String),
|
||||
locality LowCardinality(String),
|
||||
town LowCardinality(String),
|
||||
district LowCardinality(String),
|
||||
county LowCardinality(String)
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY (postcode1, postcode2, addr1, addr2)
|
||||
# highlight-start
|
||||
SETTINGS disk = disk(
|
||||
type=cache,
|
||||
max_size='1Gi',
|
||||
path='/var/lib/clickhouse/custom_disk_cache/',
|
||||
disk=disk(
|
||||
type=web,
|
||||
endpoint='https://raw.githubusercontent.com/ClickHouse/web-tables-demo/main/web/'
|
||||
)
|
||||
);
|
||||
# highlight-end
|
||||
```
|
||||
|
||||
### Details {#details}
|
||||
|
||||
In the case of `MergeTree` tables, data is getting to disk in different ways:
|
||||
|
@ -1324,7 +1324,7 @@ The trailing slash is mandatory.
|
||||
<path>/var/lib/clickhouse/</path>
|
||||
```
|
||||
|
||||
## prometheus {#server_configuration_parameters-prometheus}
|
||||
## Prometheus {#server_configuration_parameters-prometheus}
|
||||
|
||||
Exposing metrics data for scraping from [Prometheus](https://prometheus.io).
|
||||
|
||||
@ -1339,13 +1339,25 @@ Settings:
|
||||
**Example**
|
||||
|
||||
``` xml
|
||||
<prometheus>
|
||||
<endpoint>/metrics</endpoint>
|
||||
<port>9363</port>
|
||||
<metrics>true</metrics>
|
||||
<events>true</events>
|
||||
<asynchronous_metrics>true</asynchronous_metrics>
|
||||
</prometheus>
|
||||
<clickhouse>
|
||||
<listen_host>0.0.0.0</listen_host>
|
||||
<http_port>8123</http_port>
|
||||
<tcp_port>9000</tcp_port>
|
||||
<!-- highlight-start -->
|
||||
<prometheus>
|
||||
<endpoint>/metrics</endpoint>
|
||||
<port>9363</port>
|
||||
<metrics>true</metrics>
|
||||
<events>true</events>
|
||||
<asynchronous_metrics>true</asynchronous_metrics>
|
||||
</prometheus>
|
||||
<!-- highlight-end -->
|
||||
</clickhouse>
|
||||
```
|
||||
|
||||
Check (replace `127.0.0.1` with the IP addr or hostname of your ClickHouse server):
|
||||
```bash
|
||||
curl 127.0.0.1:9363/metrics
|
||||
```
|
||||
|
||||
## query_log {#server_configuration_parameters-query-log}
|
||||
|
@ -27,7 +27,7 @@ ClickHouse data types include:
|
||||
- **Aggregation function types**: use [`SimpleAggregateFunction`](./simpleaggregatefunction.md) and [`AggregateFunction`](./aggregatefunction.md) for storing the intermediate status of aggregate function results
|
||||
- **Nested data structures**: A [`Nested` data structure](./nested-data-structures/index.md) is like a table inside a cell
|
||||
- **Tuples**: A [`Tuple` of elements](./tuple.md), each having an individual type.
|
||||
- **Nullable**: [`Nullbale`](./nullable.md) allows you to store a value as `NULL` when a value is "missing" (instead of the column gettings its default value for the data type)
|
||||
- **Nullable**: [`Nullable`](./nullable.md) allows you to store a value as `NULL` when a value is "missing" (instead of the column gettings its default value for the data type)
|
||||
- **IP addresses**: use [`IPv4`](./domains/ipv4.md) and [`IPv6`](./domains/ipv6.md) to efficiently store IP addresses
|
||||
- **Geo types**: for[ geographical data](./geo.md), including `Point`, `Ring`, `Polygon` and `MultiPolygon`
|
||||
- **Special data types**: including [`Expression`](./special-data-types/expression.md), [`Set`](./special-data-types/set.md), [`Nothing`](./special-data-types/nothing.md) and [`Interval`](./special-data-types/interval.md)
|
@ -319,6 +319,9 @@ struct CheckRequest : virtual Request
|
||||
String path;
|
||||
int32_t version = -1;
|
||||
|
||||
/// should it check if a node DOES NOT exist
|
||||
bool not_exists = false;
|
||||
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
|
||||
@ -524,7 +527,7 @@ public:
|
||||
const Requests & requests,
|
||||
MultiCallback callback) = 0;
|
||||
|
||||
virtual DB::KeeperApiVersion getApiVersion() = 0;
|
||||
virtual DB::KeeperApiVersion getApiVersion() const = 0;
|
||||
|
||||
/// Expire session and finish all pending requests
|
||||
virtual void finalize(const String & reason) = 0;
|
||||
|
@ -91,7 +91,7 @@ public:
|
||||
|
||||
void finalize(const String & reason) override;
|
||||
|
||||
DB::KeeperApiVersion getApiVersion() override
|
||||
DB::KeeperApiVersion getApiVersion() const override
|
||||
{
|
||||
return KeeperApiVersion::ZOOKEEPER_COMPATIBLE;
|
||||
}
|
||||
|
@ -821,7 +821,7 @@ bool ZooKeeper::expired()
|
||||
return impl->isExpired();
|
||||
}
|
||||
|
||||
DB::KeeperApiVersion ZooKeeper::getApiVersion()
|
||||
DB::KeeperApiVersion ZooKeeper::getApiVersion() const
|
||||
{
|
||||
return impl->getApiVersion();
|
||||
}
|
||||
@ -1282,7 +1282,6 @@ Coordination::RequestPtr makeExistsRequest(const std::string & path)
|
||||
return request;
|
||||
}
|
||||
|
||||
|
||||
std::string normalizeZooKeeperPath(std::string zookeeper_path, bool check_starts_with_slash, Poco::Logger * log)
|
||||
{
|
||||
if (!zookeeper_path.empty() && zookeeper_path.back() == '/')
|
||||
|
@ -215,7 +215,7 @@ public:
|
||||
/// Returns true, if the session has expired.
|
||||
bool expired();
|
||||
|
||||
DB::KeeperApiVersion getApiVersion();
|
||||
DB::KeeperApiVersion getApiVersion() const;
|
||||
|
||||
/// Create a znode.
|
||||
/// Throw an exception if something went wrong.
|
||||
@ -674,4 +674,20 @@ bool hasZooKeeperConfig(const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
String getZooKeeperConfigName(const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
template <typename Client>
|
||||
void addCheckNotExistsRequest(Coordination::Requests & requests, const Client & client, const std::string & path)
|
||||
{
|
||||
if (client.getApiVersion() >= DB::KeeperApiVersion::WITH_CHECK_NOT_EXISTS)
|
||||
{
|
||||
auto request = std::make_shared<Coordination::CheckRequest>();
|
||||
request->path = path;
|
||||
request->not_exists = true;
|
||||
requests.push_back(std::move(request));
|
||||
return;
|
||||
}
|
||||
|
||||
requests.push_back(makeCreateRequest(path, "", zkutil::CreateMode::Persistent));
|
||||
requests.push_back(makeRemoveRequest(path, -1));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -666,7 +666,15 @@ ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return setTime(
|
||||
ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperSetResponse>()); }
|
||||
ZooKeeperResponsePtr ZooKeeperListRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperListResponse>()); }
|
||||
ZooKeeperResponsePtr ZooKeeperSimpleListRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperSimpleListResponse>()); }
|
||||
ZooKeeperResponsePtr ZooKeeperCheckRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperCheckResponse>()); }
|
||||
|
||||
ZooKeeperResponsePtr ZooKeeperCheckRequest::makeResponse() const
|
||||
{
|
||||
if (not_exists)
|
||||
return setTime(std::make_shared<ZooKeeperCheckNotExistsResponse>());
|
||||
|
||||
return setTime(std::make_shared<ZooKeeperCheckResponse>());
|
||||
}
|
||||
|
||||
ZooKeeperResponsePtr ZooKeeperMultiRequest::makeResponse() const
|
||||
{
|
||||
std::shared_ptr<ZooKeeperMultiResponse> response;
|
||||
@ -931,6 +939,8 @@ void registerZooKeeperRequest(ZooKeeperRequestFactory & factory)
|
||||
res->operation_type = ZooKeeperMultiRequest::OperationType::Read;
|
||||
else if constexpr (num == OpNum::Multi)
|
||||
res->operation_type = ZooKeeperMultiRequest::OperationType::Write;
|
||||
else if constexpr (num == OpNum::CheckNotExists)
|
||||
res->not_exists = true;
|
||||
|
||||
return res;
|
||||
});
|
||||
@ -956,6 +966,7 @@ ZooKeeperRequestFactory::ZooKeeperRequestFactory()
|
||||
registerZooKeeperRequest<OpNum::GetACL, ZooKeeperGetACLRequest>(*this);
|
||||
registerZooKeeperRequest<OpNum::SetACL, ZooKeeperSetACLRequest>(*this);
|
||||
registerZooKeeperRequest<OpNum::FilteredList, ZooKeeperFilteredListRequest>(*this);
|
||||
registerZooKeeperRequest<OpNum::CheckNotExists, ZooKeeperCheckRequest>(*this);
|
||||
}
|
||||
|
||||
PathMatchResult matchPath(std::string_view path, std::string_view match_to)
|
||||
|
@ -390,12 +390,12 @@ struct ZooKeeperSimpleListResponse final : ZooKeeperListResponse
|
||||
size_t bytesSize() const override { return ZooKeeperListResponse::bytesSize() - sizeof(stat); }
|
||||
};
|
||||
|
||||
struct ZooKeeperCheckRequest final : CheckRequest, ZooKeeperRequest
|
||||
struct ZooKeeperCheckRequest : CheckRequest, ZooKeeperRequest
|
||||
{
|
||||
ZooKeeperCheckRequest() = default;
|
||||
explicit ZooKeeperCheckRequest(const CheckRequest & base) : CheckRequest(base) {}
|
||||
|
||||
OpNum getOpNum() const override { return OpNum::Check; }
|
||||
OpNum getOpNum() const override { return not_exists ? OpNum::CheckNotExists : OpNum::Check; }
|
||||
void writeImpl(WriteBuffer & out) const override;
|
||||
void readImpl(ReadBuffer & in) override;
|
||||
std::string toStringImpl() const override;
|
||||
@ -408,7 +408,7 @@ struct ZooKeeperCheckRequest final : CheckRequest, ZooKeeperRequest
|
||||
void createLogElements(LogElements & elems) const override;
|
||||
};
|
||||
|
||||
struct ZooKeeperCheckResponse final : CheckResponse, ZooKeeperResponse
|
||||
struct ZooKeeperCheckResponse : CheckResponse, ZooKeeperResponse
|
||||
{
|
||||
void readImpl(ReadBuffer &) override {}
|
||||
void writeImpl(WriteBuffer &) const override {}
|
||||
@ -417,6 +417,12 @@ struct ZooKeeperCheckResponse final : CheckResponse, ZooKeeperResponse
|
||||
size_t bytesSize() const override { return CheckResponse::bytesSize() + sizeof(xid) + sizeof(zxid); }
|
||||
};
|
||||
|
||||
struct ZooKeeperCheckNotExistsResponse : public ZooKeeperCheckResponse
|
||||
{
|
||||
OpNum getOpNum() const override { return OpNum::CheckNotExists; }
|
||||
using ZooKeeperCheckResponse::ZooKeeperCheckResponse;
|
||||
};
|
||||
|
||||
/// This response may be received only as an element of responses in MultiResponse.
|
||||
struct ZooKeeperErrorResponse final : ErrorResponse, ZooKeeperResponse
|
||||
{
|
||||
|
@ -26,6 +26,7 @@ static const std::unordered_set<int32_t> VALID_OPERATIONS =
|
||||
static_cast<int32_t>(OpNum::SetACL),
|
||||
static_cast<int32_t>(OpNum::GetACL),
|
||||
static_cast<int32_t>(OpNum::FilteredList),
|
||||
static_cast<int32_t>(OpNum::CheckNotExists),
|
||||
};
|
||||
|
||||
std::string toString(OpNum op_num)
|
||||
@ -70,6 +71,8 @@ std::string toString(OpNum op_num)
|
||||
return "GetACL";
|
||||
case OpNum::FilteredList:
|
||||
return "FilteredList";
|
||||
case OpNum::CheckNotExists:
|
||||
return "CheckNotExists";
|
||||
}
|
||||
int32_t raw_op = static_cast<int32_t>(op_num);
|
||||
throw Exception("Operation " + std::to_string(raw_op) + " is unknown", Error::ZUNIMPLEMENTED);
|
||||
|
@ -36,6 +36,7 @@ enum class OpNum : int32_t
|
||||
|
||||
// CH Keeper specific operations
|
||||
FilteredList = 500,
|
||||
CheckNotExists = 501,
|
||||
|
||||
SessionID = 997, /// Special internal request
|
||||
};
|
||||
|
@ -1085,7 +1085,7 @@ void ZooKeeper::pushRequest(RequestInfo && info)
|
||||
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
|
||||
}
|
||||
|
||||
KeeperApiVersion ZooKeeper::getApiVersion()
|
||||
KeeperApiVersion ZooKeeper::getApiVersion() const
|
||||
{
|
||||
return keeper_api_version;
|
||||
}
|
||||
|
@ -179,7 +179,7 @@ public:
|
||||
const Requests & requests,
|
||||
MultiCallback callback) override;
|
||||
|
||||
DB::KeeperApiVersion getApiVersion() override;
|
||||
DB::KeeperApiVersion getApiVersion() const override;
|
||||
|
||||
/// Without forcefully invalidating (finalizing) ZooKeeper session before
|
||||
/// establishing a new one, there was a possibility that server is using
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Common/ZooKeeper/ZooKeeperCommon.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include "Coordination/KeeperConstants.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -381,6 +382,11 @@ public:
|
||||
ephemeral_nodes.clear();
|
||||
}
|
||||
|
||||
KeeperApiVersion getApiVersion() const
|
||||
{
|
||||
return keeper->getApiVersion();
|
||||
}
|
||||
|
||||
private:
|
||||
void faultInjectionBefore(std::function<void()> fault_cleanup)
|
||||
{
|
||||
|
@ -9,10 +9,11 @@ enum class KeeperApiVersion : uint8_t
|
||||
{
|
||||
ZOOKEEPER_COMPATIBLE = 0,
|
||||
WITH_FILTERED_LIST,
|
||||
WITH_MULTI_READ
|
||||
WITH_MULTI_READ,
|
||||
WITH_CHECK_NOT_EXISTS,
|
||||
};
|
||||
|
||||
inline constexpr auto current_keeper_api_version = KeeperApiVersion::WITH_MULTI_READ;
|
||||
inline constexpr auto current_keeper_api_version = KeeperApiVersion::WITH_CHECK_NOT_EXISTS;
|
||||
|
||||
const std::string keeper_system_path = "/keeper";
|
||||
const std::string keeper_api_version_path = keeper_system_path + "/api_version";
|
||||
|
@ -1449,24 +1449,39 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc
|
||||
|
||||
struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestProcessor
|
||||
{
|
||||
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
|
||||
explicit KeeperStorageCheckRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_)
|
||||
: KeeperStorageRequestProcessor(zk_request_)
|
||||
{
|
||||
return storage.checkACL(zk_request->getPath(), Coordination::ACL::Read, session_id, is_local);
|
||||
check_not_exists = zk_request->getOpNum() == Coordination::OpNum::CheckNotExists;
|
||||
}
|
||||
|
||||
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
|
||||
{
|
||||
auto path = zk_request->getPath();
|
||||
return storage.checkACL(check_not_exists ? parentPath(path) : path, Coordination::ACL::Read, session_id, is_local);
|
||||
}
|
||||
|
||||
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
|
||||
std::vector<KeeperStorage::Delta>
|
||||
preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::KeeperCheckRequest);
|
||||
|
||||
Coordination::ZooKeeperCheckRequest & request = dynamic_cast<Coordination::ZooKeeperCheckRequest &>(*zk_request);
|
||||
|
||||
if (!storage.uncommitted_state.getNode(request.path))
|
||||
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}};
|
||||
|
||||
auto node = storage.uncommitted_state.getNode(request.path);
|
||||
if (request.version != -1 && request.version != node->stat.version)
|
||||
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}};
|
||||
if (check_not_exists)
|
||||
{
|
||||
if (node && (request.version == -1 || request.version == node->stat.version))
|
||||
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNODEEXISTS}};
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!node)
|
||||
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}};
|
||||
|
||||
if (request.version != -1 && request.version != node->stat.version)
|
||||
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}};
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
@ -1497,17 +1512,22 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro
|
||||
|
||||
auto & container = storage.container;
|
||||
auto node_it = container.find(request.path);
|
||||
if (node_it == container.end())
|
||||
|
||||
if (check_not_exists)
|
||||
{
|
||||
on_error(Coordination::Error::ZNONODE);
|
||||
}
|
||||
else if (request.version != -1 && request.version != node_it->value.stat.version)
|
||||
{
|
||||
on_error(Coordination::Error::ZBADVERSION);
|
||||
if (node_it != container.end() && (request.version == -1 || request.version == node_it->value.stat.version))
|
||||
on_error(Coordination::Error::ZNODEEXISTS);
|
||||
else
|
||||
response.error = Coordination::Error::ZOK;
|
||||
}
|
||||
else
|
||||
{
|
||||
response.error = Coordination::Error::ZOK;
|
||||
if (node_it == container.end())
|
||||
on_error(Coordination::Error::ZNONODE);
|
||||
else if (request.version != -1 && request.version != node_it->value.stat.version)
|
||||
on_error(Coordination::Error::ZBADVERSION);
|
||||
else
|
||||
response.error = Coordination::Error::ZOK;
|
||||
}
|
||||
|
||||
return response_ptr;
|
||||
@ -1523,6 +1543,9 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro
|
||||
ProfileEvents::increment(ProfileEvents::KeeperCheckRequest);
|
||||
return processImpl<true>(storage, zxid);
|
||||
}
|
||||
|
||||
private:
|
||||
bool check_not_exists;
|
||||
};
|
||||
|
||||
|
||||
@ -1716,6 +1739,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro
|
||||
concrete_requests.push_back(std::make_shared<KeeperStorageSetRequestProcessor>(sub_zk_request));
|
||||
break;
|
||||
case Coordination::OpNum::Check:
|
||||
case Coordination::OpNum::CheckNotExists:
|
||||
check_operation_type(OperationType::Write);
|
||||
concrete_requests.push_back(std::make_shared<KeeperStorageCheckRequestProcessor>(sub_zk_request));
|
||||
break;
|
||||
@ -1971,6 +1995,7 @@ KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFactory()
|
||||
registerKeeperRequestProcessor<Coordination::OpNum::MultiRead, KeeperStorageMultiRequestProcessor>(*this);
|
||||
registerKeeperRequestProcessor<Coordination::OpNum::SetACL, KeeperStorageSetACLRequestProcessor>(*this);
|
||||
registerKeeperRequestProcessor<Coordination::OpNum::GetACL, KeeperStorageGetACLRequestProcessor>(*this);
|
||||
registerKeeperRequestProcessor<Coordination::OpNum::CheckNotExists, KeeperStorageCheckRequestProcessor>(*this);
|
||||
}
|
||||
|
||||
|
||||
|
@ -2451,6 +2451,78 @@ TEST_P(CoordinationTest, ChangelogTestMaxLogSize)
|
||||
|
||||
}
|
||||
|
||||
TEST_P(CoordinationTest, TestCheckNotExistsRequest)
|
||||
{
|
||||
using namespace DB;
|
||||
using namespace Coordination;
|
||||
|
||||
KeeperStorage storage{500, "", keeper_context};
|
||||
|
||||
int32_t zxid = 0;
|
||||
|
||||
const auto create_path = [&](const auto & path)
|
||||
{
|
||||
const auto create_request = std::make_shared<ZooKeeperCreateRequest>();
|
||||
int new_zxid = ++zxid;
|
||||
create_request->path = path;
|
||||
storage.preprocessRequest(create_request, 1, 0, new_zxid);
|
||||
auto responses = storage.processRequest(create_request, 1, new_zxid);
|
||||
|
||||
EXPECT_GE(responses.size(), 1);
|
||||
EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to create " << path;
|
||||
};
|
||||
|
||||
const auto check_request = std::make_shared<ZooKeeperCheckRequest>();
|
||||
check_request->path = "/test_node";
|
||||
check_request->not_exists = true;
|
||||
|
||||
{
|
||||
SCOPED_TRACE("CheckNotExists returns ZOK");
|
||||
int new_zxid = ++zxid;
|
||||
storage.preprocessRequest(check_request, 1, 0, new_zxid);
|
||||
auto responses = storage.processRequest(check_request, 1, new_zxid);
|
||||
EXPECT_GE(responses.size(), 1);
|
||||
auto error = responses[0].response->error;
|
||||
EXPECT_EQ(error, Coordination::Error::ZOK) << "CheckNotExists returned invalid result: " << errorMessage(error);
|
||||
}
|
||||
|
||||
create_path("/test_node");
|
||||
auto node_it = storage.container.find("/test_node");
|
||||
ASSERT_NE(node_it, storage.container.end());
|
||||
auto node_version = node_it->value.stat.version;
|
||||
|
||||
{
|
||||
SCOPED_TRACE("CheckNotExists returns ZNODEEXISTS");
|
||||
int new_zxid = ++zxid;
|
||||
storage.preprocessRequest(check_request, 1, 0, new_zxid);
|
||||
auto responses = storage.processRequest(check_request, 1, new_zxid);
|
||||
EXPECT_GE(responses.size(), 1);
|
||||
auto error = responses[0].response->error;
|
||||
EXPECT_EQ(error, Coordination::Error::ZNODEEXISTS) << "CheckNotExists returned invalid result: " << errorMessage(error);
|
||||
}
|
||||
|
||||
{
|
||||
SCOPED_TRACE("CheckNotExists returns ZNODEEXISTS for same version");
|
||||
int new_zxid = ++zxid;
|
||||
check_request->version = node_version;
|
||||
storage.preprocessRequest(check_request, 1, 0, new_zxid);
|
||||
auto responses = storage.processRequest(check_request, 1, new_zxid);
|
||||
EXPECT_GE(responses.size(), 1);
|
||||
auto error = responses[0].response->error;
|
||||
EXPECT_EQ(error, Coordination::Error::ZNODEEXISTS) << "CheckNotExists returned invalid result: " << errorMessage(error);
|
||||
}
|
||||
|
||||
{
|
||||
SCOPED_TRACE("CheckNotExists returns ZOK for different version");
|
||||
int new_zxid = ++zxid;
|
||||
check_request->version = node_version + 1;
|
||||
storage.preprocessRequest(check_request, 1, 0, new_zxid);
|
||||
auto responses = storage.processRequest(check_request, 1, new_zxid);
|
||||
EXPECT_GE(responses.size(), 1);
|
||||
auto error = responses[0].response->error;
|
||||
EXPECT_EQ(error, Coordination::Error::ZOK) << "CheckNotExists returned invalid result: " << errorMessage(error);
|
||||
}
|
||||
}
|
||||
|
||||
INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite,
|
||||
CoordinationTest,
|
||||
|
@ -29,11 +29,6 @@
|
||||
#define DEFAULT_INSERT_BLOCK_SIZE \
|
||||
1048449 /// 1048576 - PADDING_FOR_SIMD - (PADDING_FOR_SIMD - 1) bytes padding that we usually have in arrays
|
||||
|
||||
/** The same, but for merge operations. Less DEFAULT_BLOCK_SIZE for saving RAM (since all the columns are read).
|
||||
* Significantly less, since there are 10-way mergers.
|
||||
*/
|
||||
#define DEFAULT_MERGE_BLOCK_SIZE 8192
|
||||
|
||||
#define DEFAULT_PERIODIC_LIVE_VIEW_REFRESH_SEC 60
|
||||
#define SHOW_CHARS_ON_SYNTAX_ERROR ptrdiff_t(160)
|
||||
#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES 3
|
||||
@ -83,4 +78,3 @@
|
||||
#else
|
||||
#define QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS 0
|
||||
#endif
|
||||
|
||||
|
@ -49,6 +49,8 @@ class IColumn;
|
||||
M(MaxThreads, max_download_threads, 4, "The maximum number of threads to download data (e.g. for URL engine).", 0) \
|
||||
M(UInt64, max_download_buffer_size, 10*1024*1024, "The maximal size of buffer for parallel downloading (e.g. for URL engine) per each thread.", 0) \
|
||||
M(UInt64, max_read_buffer_size, DBMS_DEFAULT_BUFFER_SIZE, "The maximum size of the buffer to read from the filesystem.", 0) \
|
||||
M(UInt64, max_read_buffer_size_local_fs, 128*1024, "The maximum size of the buffer to read from local filesystem. If set to 0 then max_read_buffer_size will be used.", 0) \
|
||||
M(UInt64, max_read_buffer_size_remote_fs, 0, "The maximum size of the buffer to read from remote filesystem. If set to 0 then max_read_buffer_size will be used.", 0) \
|
||||
M(UInt64, max_distributed_connections, 1024, "The maximum number of connections for distributed processing of one query (should be greater than max_threads).", 0) \
|
||||
M(UInt64, max_query_size, DBMS_DEFAULT_MAX_QUERY_SIZE, "The maximum number of bytes of a query string parsed by the SQL parser. Data in the VALUES clause of INSERT queries is processed by a separate stream parser (that consumes O(1) RAM) and not affected by this restriction.", 0) \
|
||||
M(UInt64, interactive_delay, 100000, "The interval in microseconds to check if the request is cancelled, and to send progress info.", 0) \
|
||||
|
@ -493,11 +493,6 @@ bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext()
|
||||
|
||||
chassert(file_offset_of_buffer_end > completed_range.right);
|
||||
|
||||
if (read_type == ReadType::CACHED)
|
||||
{
|
||||
chassert(current_file_segment->getDownloadedSize(true) == current_file_segment->range().size());
|
||||
}
|
||||
|
||||
file_segments->popFront();
|
||||
if (file_segments->empty())
|
||||
return false;
|
||||
|
@ -3,6 +3,7 @@
|
||||
#if USE_BROTLI
|
||||
# include <brotli/decode.h>
|
||||
# include "BrotliReadBuffer.h"
|
||||
# include <IO/WithFileName.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -60,7 +61,10 @@ bool BrotliReadBuffer::nextImpl()
|
||||
|
||||
if (brotli->result == BROTLI_DECODER_RESULT_NEEDS_MORE_INPUT && (!in_available || in->eof()))
|
||||
{
|
||||
throw Exception(ErrorCodes::BROTLI_READ_FAILED, "brotli decode error");
|
||||
throw Exception(
|
||||
ErrorCodes::BROTLI_READ_FAILED,
|
||||
"brotli decode error{}",
|
||||
getExceptionEntryWithFileName(*in));
|
||||
}
|
||||
|
||||
out_capacity = internal_buffer.size();
|
||||
@ -83,13 +87,19 @@ bool BrotliReadBuffer::nextImpl()
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::BROTLI_READ_FAILED, "brotli decode error");
|
||||
throw Exception(
|
||||
ErrorCodes::BROTLI_READ_FAILED,
|
||||
"brotli decode error{}",
|
||||
getExceptionEntryWithFileName(*in));
|
||||
}
|
||||
}
|
||||
|
||||
if (brotli->result == BROTLI_DECODER_RESULT_ERROR)
|
||||
{
|
||||
throw Exception(ErrorCodes::BROTLI_READ_FAILED, "brotli decode error");
|
||||
throw Exception(
|
||||
ErrorCodes::BROTLI_READ_FAILED,
|
||||
"brotli decode error{}",
|
||||
getExceptionEntryWithFileName(*in));
|
||||
}
|
||||
|
||||
return true;
|
||||
|
@ -3,6 +3,7 @@
|
||||
#if USE_BZIP2
|
||||
# include <IO/Bzip2ReadBuffer.h>
|
||||
# include <bzlib.h>
|
||||
# include <IO/WithFileName.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -118,13 +119,17 @@ bool Bzip2ReadBuffer::nextImpl()
|
||||
if (ret != BZ_OK)
|
||||
throw Exception(
|
||||
ErrorCodes::BZIP2_STREAM_DECODER_FAILED,
|
||||
"bzip2 stream decoder failed: error code: {}",
|
||||
ret);
|
||||
"bzip2 stream decoder failed: error code: {}{}",
|
||||
ret,
|
||||
getExceptionEntryWithFileName(*in));
|
||||
|
||||
if (in->eof())
|
||||
{
|
||||
eof_flag = true;
|
||||
throw Exception(ErrorCodes::UNEXPECTED_END_OF_FILE, "Unexpected end of bzip2 archive");
|
||||
throw Exception(
|
||||
ErrorCodes::UNEXPECTED_END_OF_FILE,
|
||||
"Unexpected end of bzip2 archive{}",
|
||||
getExceptionEntryWithFileName(*in));
|
||||
}
|
||||
|
||||
return true;
|
||||
|
@ -11,6 +11,8 @@
|
||||
|
||||
#include "HadoopSnappyReadBuffer.h"
|
||||
|
||||
#include <IO/WithFileName.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
@ -89,9 +91,8 @@ inline HadoopSnappyDecoder::Status HadoopSnappyDecoder::readCompressedLength(siz
|
||||
{
|
||||
auto status = readLength(avail_in, next_in, &compressed_length);
|
||||
if (unlikely(compressed_length > 0 && static_cast<size_t>(compressed_length) > sizeof(buffer)))
|
||||
throw Exception(ErrorCodes::SNAPPY_UNCOMPRESS_FAILED,
|
||||
"Too large snappy compressed block. buffer size: {}, compressed block size: {}",
|
||||
sizeof(buffer), compressed_length);
|
||||
return Status::TOO_LARGE_COMPRESSED_BLOCK;
|
||||
|
||||
return status;
|
||||
}
|
||||
return Status::OK;
|
||||
@ -196,7 +197,11 @@ bool HadoopSnappyReadBuffer::nextImpl()
|
||||
|
||||
if (decoder->result == Status::NEEDS_MORE_INPUT && (!in_available || in->eof()))
|
||||
{
|
||||
throw Exception(ErrorCodes::SNAPPY_UNCOMPRESS_FAILED, "hadoop snappy decode error: {}", statusToString(decoder->result));
|
||||
throw Exception(
|
||||
ErrorCodes::SNAPPY_UNCOMPRESS_FAILED,
|
||||
"hadoop snappy decode error: {}{}",
|
||||
statusToString(decoder->result),
|
||||
getExceptionEntryWithFileName(*in));
|
||||
}
|
||||
|
||||
out_capacity = internal_buffer.size();
|
||||
@ -219,9 +224,13 @@ bool HadoopSnappyReadBuffer::nextImpl()
|
||||
}
|
||||
return true;
|
||||
}
|
||||
else if (decoder->result == Status::INVALID_INPUT || decoder->result == Status::BUFFER_TOO_SMALL)
|
||||
else if (decoder->result != Status::NEEDS_MORE_INPUT)
|
||||
{
|
||||
throw Exception(ErrorCodes::SNAPPY_UNCOMPRESS_FAILED, "hadoop snappy decode error: {}", statusToString(decoder->result));
|
||||
throw Exception(
|
||||
ErrorCodes::SNAPPY_UNCOMPRESS_FAILED,
|
||||
"hadoop snappy decode error: {}{}",
|
||||
statusToString(decoder->result),
|
||||
getExceptionEntryWithFileName(*in));
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
@ -29,6 +29,7 @@ public:
|
||||
INVALID_INPUT = 1,
|
||||
BUFFER_TOO_SMALL = 2,
|
||||
NEEDS_MORE_INPUT = 3,
|
||||
TOO_LARGE_COMPRESSED_BLOCK = 4,
|
||||
};
|
||||
|
||||
HadoopSnappyDecoder() = default;
|
||||
@ -84,6 +85,8 @@ public:
|
||||
return "BUFFER_TOO_SMALL";
|
||||
case Status::NEEDS_MORE_INPUT:
|
||||
return "NEEDS_MORE_INPUT";
|
||||
case Status::TOO_LARGE_COMPRESSED_BLOCK:
|
||||
return "TOO_LARGE_COMPRESSED_BLOCK";
|
||||
}
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <IO/LZMAInflatingReadBuffer.h>
|
||||
#include <IO/WithFileName.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -78,18 +79,20 @@ bool LZMAInflatingReadBuffer::nextImpl()
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LZMA_STREAM_DECODER_FAILED,
|
||||
"lzma decoder finished, but input stream has not exceeded: error code: {}; lzma version: {}",
|
||||
"lzma decoder finished, but input stream has not exceeded: error code: {}; lzma version: {}{}",
|
||||
ret,
|
||||
LZMA_VERSION_STRING);
|
||||
LZMA_VERSION_STRING,
|
||||
getExceptionEntryWithFileName(*in));
|
||||
}
|
||||
}
|
||||
|
||||
if (ret != LZMA_OK)
|
||||
throw Exception(
|
||||
ErrorCodes::LZMA_STREAM_DECODER_FAILED,
|
||||
"lzma_stream_decoder failed: error code: error codeL {}; lzma version: {}",
|
||||
"lzma_stream_decoder failed: error code: error code {}; lzma version: {}{}",
|
||||
ret,
|
||||
LZMA_VERSION_STRING);
|
||||
LZMA_VERSION_STRING,
|
||||
getExceptionEntryWithFileName(*in));
|
||||
|
||||
return true;
|
||||
}
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <IO/Lz4InflatingReadBuffer.h>
|
||||
#include <IO/WithFileName.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -72,9 +73,10 @@ bool Lz4InflatingReadBuffer::nextImpl()
|
||||
if (LZ4F_isError(ret))
|
||||
throw Exception(
|
||||
ErrorCodes::LZ4_DECODER_FAILED,
|
||||
"LZ4 decompression failed. LZ4F version: {}. Error: {}",
|
||||
"LZ4 decompression failed. LZ4F version: {}. Error: {}{}",
|
||||
LZ4F_VERSION,
|
||||
LZ4F_getErrorName(ret));
|
||||
LZ4F_getErrorName(ret),
|
||||
getExceptionEntryWithFileName(*in));
|
||||
|
||||
if (in->eof())
|
||||
{
|
||||
|
@ -68,7 +68,9 @@ struct ReadSettings
|
||||
/// Method to use reading from remote filesystem.
|
||||
RemoteFSReadMethod remote_fs_method = RemoteFSReadMethod::threadpool;
|
||||
|
||||
size_t local_fs_buffer_size = DBMS_DEFAULT_BUFFER_SIZE;
|
||||
/// https://eklitzke.org/efficient-file-copying-on-linux
|
||||
size_t local_fs_buffer_size = 128 * 1024;
|
||||
|
||||
size_t remote_fs_buffer_size = DBMS_DEFAULT_BUFFER_SIZE;
|
||||
size_t prefetch_buffer_size = DBMS_DEFAULT_BUFFER_SIZE;
|
||||
|
||||
|
@ -26,4 +26,14 @@ String getFileNameFromReadBuffer(const ReadBuffer & in)
|
||||
return getFileName(in);
|
||||
}
|
||||
|
||||
String getExceptionEntryWithFileName(const ReadBuffer & in)
|
||||
{
|
||||
auto filename = getFileNameFromReadBuffer(in);
|
||||
|
||||
if (filename.empty())
|
||||
return "";
|
||||
|
||||
return fmt::format(": While reading from: {}", filename);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -14,5 +14,6 @@ public:
|
||||
};
|
||||
|
||||
String getFileNameFromReadBuffer(const ReadBuffer & in);
|
||||
String getExceptionEntryWithFileName(const ReadBuffer & in);
|
||||
|
||||
}
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <IO/ZlibInflatingReadBuffer.h>
|
||||
|
||||
#include <IO/WithFileName.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -99,14 +99,22 @@ bool ZlibInflatingReadBuffer::nextImpl()
|
||||
{
|
||||
rc = inflateReset(&zstr);
|
||||
if (rc != Z_OK)
|
||||
throw Exception(ErrorCodes::ZLIB_INFLATE_FAILED, "inflateReset failed: {}", zError(rc));
|
||||
throw Exception(
|
||||
ErrorCodes::ZLIB_INFLATE_FAILED,
|
||||
"inflateReset failed: {}{}",
|
||||
zError(rc),
|
||||
getExceptionEntryWithFileName(*in));
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
/// If it is not end and not OK, something went wrong, throw exception
|
||||
if (rc != Z_OK)
|
||||
throw Exception(ErrorCodes::ZLIB_INFLATE_FAILED, "inflate failed: {}", zError(rc));
|
||||
throw Exception(
|
||||
ErrorCodes::ZLIB_INFLATE_FAILED,
|
||||
"inflate failed: {}{}",
|
||||
zError(rc),
|
||||
getExceptionEntryWithFileName(*in));
|
||||
}
|
||||
while (working_buffer.empty());
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <IO/ZstdInflatingReadBuffer.h>
|
||||
#include <IO/WithFileName.h>
|
||||
#include <zstd_errors.h>
|
||||
|
||||
|
||||
@ -61,12 +62,13 @@ bool ZstdInflatingReadBuffer::nextImpl()
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::ZSTD_DECODER_FAILED,
|
||||
"ZSTD stream decoding failed: error '{}'{}; ZSTD version: {}",
|
||||
"ZSTD stream decoding failed: error '{}'{}; ZSTD version: {}{}",
|
||||
ZSTD_getErrorName(ret),
|
||||
ZSTD_error_frameParameter_windowTooLarge == ret
|
||||
? ". You can increase the maximum window size with the 'zstd_window_log_max' setting in ClickHouse. Example: 'SET zstd_window_log_max = 31'"
|
||||
: "",
|
||||
ZSTD_VERSION_STRING);
|
||||
ZSTD_VERSION_STRING,
|
||||
getExceptionEntryWithFileName(*in));
|
||||
}
|
||||
|
||||
/// Check that something has changed after decompress (input or output position)
|
||||
|
@ -278,8 +278,8 @@ void CacheMetadata::doCleanup()
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
chassert(false);
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
chassert(false);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -4288,8 +4288,10 @@ ReadSettings Context::getReadSettings() const
|
||||
"Invalid value '{}' for max_read_buffer_size", settings.max_read_buffer_size);
|
||||
}
|
||||
|
||||
res.local_fs_buffer_size = settings.max_read_buffer_size;
|
||||
res.remote_fs_buffer_size = settings.max_read_buffer_size;
|
||||
res.local_fs_buffer_size
|
||||
= settings.max_read_buffer_size_local_fs ? settings.max_read_buffer_size_local_fs : settings.max_read_buffer_size;
|
||||
res.remote_fs_buffer_size
|
||||
= settings.max_read_buffer_size_remote_fs ? settings.max_read_buffer_size_remote_fs : settings.max_read_buffer_size;
|
||||
res.prefetch_buffer_size = settings.prefetch_buffer_size;
|
||||
res.direct_io_threshold = settings.min_bytes_to_use_direct_io;
|
||||
res.mmap_threshold = settings.min_bytes_to_use_mmap_io;
|
||||
|
@ -1066,13 +1066,6 @@ static std::shared_ptr<IJoin> chooseJoinAlgorithm(
|
||||
{
|
||||
const auto & settings = context->getSettings();
|
||||
|
||||
Block left_sample_block(left_sample_columns);
|
||||
for (auto & column : left_sample_block)
|
||||
{
|
||||
if (!column.column)
|
||||
column.column = column.type->createColumn();
|
||||
}
|
||||
|
||||
Block right_sample_block = joined_plan->getCurrentDataStream().header;
|
||||
|
||||
std::vector<String> tried_algorithms;
|
||||
@ -1118,7 +1111,10 @@ static std::shared_ptr<IJoin> chooseJoinAlgorithm(
|
||||
if (analyzed_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH))
|
||||
{
|
||||
tried_algorithms.push_back(toString(JoinAlgorithm::GRACE_HASH));
|
||||
if (GraceHashJoin::isSupported(analyzed_join))
|
||||
|
||||
// Grace hash join requires that columns exist in left_sample_block.
|
||||
Block left_sample_block(left_sample_columns);
|
||||
if (sanitizeBlock(left_sample_block, false) && GraceHashJoin::isSupported(analyzed_join))
|
||||
return std::make_shared<GraceHashJoin>(context, analyzed_join, left_sample_block, right_sample_block, context->getTempDataOnDisk());
|
||||
}
|
||||
|
||||
|
@ -165,6 +165,7 @@ SortedBlocksWriter::TmpFilePtr SortedBlocksWriter::flush(const BlocksList & bloc
|
||||
pipeline.getNumStreams(),
|
||||
sort_description,
|
||||
rows_in_block,
|
||||
/*max_block_size_bytes=*/0,
|
||||
SortingQueueStrategy::Default);
|
||||
|
||||
pipeline.addTransform(std::move(transform));
|
||||
@ -220,6 +221,7 @@ SortedBlocksWriter::PremergedFiles SortedBlocksWriter::premerge()
|
||||
pipeline.getNumStreams(),
|
||||
sort_description,
|
||||
rows_in_block,
|
||||
/*max_block_size_bytes=*/0,
|
||||
SortingQueueStrategy::Default);
|
||||
|
||||
pipeline.addTransform(std::move(transform));
|
||||
@ -254,6 +256,7 @@ SortedBlocksWriter::SortedFiles SortedBlocksWriter::finishMerge(std::function<vo
|
||||
pipeline.getNumStreams(),
|
||||
sort_description,
|
||||
rows_in_block,
|
||||
/*max_block_size_bytes=*/0,
|
||||
SortingQueueStrategy::Default);
|
||||
|
||||
pipeline.addTransform(std::move(transform));
|
||||
@ -331,6 +334,7 @@ Block SortedBlocksBuffer::mergeBlocks(Blocks && blocks) const
|
||||
builder.getNumStreams(),
|
||||
sort_description,
|
||||
num_rows,
|
||||
/*max_block_size_bytes=*/0,
|
||||
SortingQueueStrategy::Default);
|
||||
|
||||
builder.addTransform(std::move(transform));
|
||||
|
@ -147,6 +147,7 @@ void TableJoin::addDisjunct()
|
||||
void TableJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast)
|
||||
{
|
||||
addKey(left_table_ast->getColumnName(), right_table_ast->getAliasOrColumnName(), left_table_ast, right_table_ast);
|
||||
right_key_aliases[right_table_ast->getColumnName()] = right_table_ast->getAliasOrColumnName();
|
||||
}
|
||||
|
||||
/// @return how many times right key appears in ON section.
|
||||
@ -662,6 +663,14 @@ String TableJoin::renamedRightColumnName(const String & name) const
|
||||
return name;
|
||||
}
|
||||
|
||||
String TableJoin::renamedRightColumnNameWithAlias(const String & name) const
|
||||
{
|
||||
auto renamed = renamedRightColumnName(name);
|
||||
if (const auto it = right_key_aliases.find(renamed); it != right_key_aliases.end())
|
||||
return it->second;
|
||||
return renamed;
|
||||
}
|
||||
|
||||
void TableJoin::setRename(const String & from, const String & to)
|
||||
{
|
||||
renames[from] = to;
|
||||
|
@ -156,6 +156,13 @@ private:
|
||||
/// Original name -> name. Only renamed columns.
|
||||
std::unordered_map<String, String> renames;
|
||||
|
||||
/// Map column name to actual key name that can be an alias.
|
||||
/// Example: SELECT r.id as rid from t JOIN r ON t.id = rid
|
||||
/// Map: r.id -> rid
|
||||
/// Required only for StorageJoin to map join keys back to original column names.
|
||||
/// (workaround for ExpressionAnalyzer)
|
||||
std::unordered_map<String, String> right_key_aliases;
|
||||
|
||||
VolumePtr tmp_volume;
|
||||
|
||||
std::shared_ptr<StorageJoin> right_storage_join;
|
||||
@ -333,6 +340,7 @@ public:
|
||||
Block getRequiredRightKeys(const Block & right_table_keys, std::vector<String> & keys_sources) const;
|
||||
|
||||
String renamedRightColumnName(const String & name) const;
|
||||
String renamedRightColumnNameWithAlias(const String & name) const;
|
||||
void setRename(const String & from, const String & to);
|
||||
|
||||
void resetKeys();
|
||||
|
@ -87,6 +87,7 @@ NamesAndTypesList ZooKeeperLogElement::getNamesAndTypes()
|
||||
{"Auth", static_cast<Int16>(Coordination::OpNum::Auth)},
|
||||
{"SessionID", static_cast<Int16>(Coordination::OpNum::SessionID)},
|
||||
{"FilteredList", static_cast<Int16>(Coordination::OpNum::FilteredList)},
|
||||
{"CheckNotExists", static_cast<Int16>(Coordination::OpNum::CheckNotExists)},
|
||||
});
|
||||
|
||||
auto error_enum = getCoordinationErrorCodesEnumType();
|
||||
|
@ -1429,10 +1429,12 @@ bool ParserAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
if (!allow_alias_without_as_keyword && !has_as_word)
|
||||
return false;
|
||||
|
||||
bool is_quoted = pos->type == TokenType::QuotedIdentifier;
|
||||
|
||||
if (!id_p.parse(pos, node, expected))
|
||||
return false;
|
||||
|
||||
if (!has_as_word)
|
||||
if (!has_as_word && !is_quoted)
|
||||
{
|
||||
/** In this case, the alias can not match the keyword -
|
||||
* so that in the query "SELECT x FROM t", the word FROM was not considered an alias,
|
||||
|
@ -13,14 +13,18 @@ class AggregatingSortedTransform final : public IMergingTransform<AggregatingSor
|
||||
{
|
||||
public:
|
||||
AggregatingSortedTransform(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, size_t max_block_size)
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
size_t max_block_size_rows,
|
||||
size_t max_block_size_bytes)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
max_block_size)
|
||||
max_block_size_rows,
|
||||
max_block_size_bytes)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -159,8 +159,11 @@ AggregatingSortedAlgorithm::SimpleAggregateDescription::~SimpleAggregateDescript
|
||||
|
||||
|
||||
AggregatingSortedAlgorithm::AggregatingMergedData::AggregatingMergedData(
|
||||
MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def_)
|
||||
: MergedData(std::move(columns_), false, max_block_size_), def(def_)
|
||||
MutableColumns columns_,
|
||||
UInt64 max_block_size_rows_,
|
||||
UInt64 max_block_size_bytes_,
|
||||
ColumnsDefinition & def_)
|
||||
: MergedData(std::move(columns_), false, max_block_size_rows_, max_block_size_bytes_), def(def_)
|
||||
{
|
||||
initAggregateDescription();
|
||||
|
||||
@ -257,10 +260,14 @@ void AggregatingSortedAlgorithm::AggregatingMergedData::initAggregateDescription
|
||||
|
||||
|
||||
AggregatingSortedAlgorithm::AggregatingSortedAlgorithm(
|
||||
const Block & header_, size_t num_inputs, SortDescription description_, size_t max_block_size)
|
||||
const Block & header_,
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
size_t max_block_size_rows_,
|
||||
size_t max_block_size_bytes_)
|
||||
: IMergingAlgorithmWithDelayedChunk(header_, num_inputs, description_)
|
||||
, columns_definition(defineColumns(header_, description_))
|
||||
, merged_data(getMergedColumns(header_, columns_definition), max_block_size, columns_definition)
|
||||
, merged_data(getMergedColumns(header_, columns_definition), max_block_size_rows_, max_block_size_bytes_, columns_definition)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -18,8 +18,11 @@ class AggregatingSortedAlgorithm final : public IMergingAlgorithmWithDelayedChun
|
||||
{
|
||||
public:
|
||||
AggregatingSortedAlgorithm(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, size_t max_block_size);
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
size_t max_block_size_rows_,
|
||||
size_t max_block_size_bytes_);
|
||||
|
||||
void initialize(Inputs inputs) override;
|
||||
void consume(Input & input, size_t source_num) override;
|
||||
@ -96,7 +99,11 @@ private:
|
||||
using MergedData::insertRow;
|
||||
|
||||
public:
|
||||
AggregatingMergedData(MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def_);
|
||||
AggregatingMergedData(
|
||||
MutableColumns columns_,
|
||||
UInt64 max_block_size_rows_,
|
||||
UInt64 max_block_size_bytes_,
|
||||
ColumnsDefinition & def_);
|
||||
|
||||
/// Group is a group of rows with the same sorting key. It represents single row in result.
|
||||
/// Algorithm is: start group, add several rows, finish group.
|
||||
|
@ -26,12 +26,13 @@ CollapsingSortedAlgorithm::CollapsingSortedAlgorithm(
|
||||
SortDescription description_,
|
||||
const String & sign_column,
|
||||
bool only_positive_sign_,
|
||||
size_t max_block_size,
|
||||
size_t max_block_size_rows_,
|
||||
size_t max_block_size_bytes_,
|
||||
Poco::Logger * log_,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
bool use_average_block_sizes)
|
||||
: IMergingAlgorithmWithSharedChunks(header_, num_inputs, std::move(description_), out_row_sources_buf_, max_row_refs)
|
||||
, merged_data(header_.cloneEmptyColumns(), use_average_block_sizes, max_block_size)
|
||||
, merged_data(header_.cloneEmptyColumns(), use_average_block_sizes, max_block_size_rows_, max_block_size_bytes_)
|
||||
, sign_column_number(header_.getPositionByName(sign_column))
|
||||
, only_positive_sign(only_positive_sign_)
|
||||
, log(log_)
|
||||
|
@ -32,7 +32,8 @@ public:
|
||||
SortDescription description_,
|
||||
const String & sign_column,
|
||||
bool only_positive_sign_, /// For select final. Skip rows with sum(sign) < 0.
|
||||
size_t max_block_size,
|
||||
size_t max_block_size_rows_,
|
||||
size_t max_block_size_bytes_,
|
||||
Poco::Logger * log_,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool use_average_block_sizes = false);
|
||||
@ -74,4 +75,3 @@ private:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -30,9 +30,9 @@ FinishAggregatingInOrderAlgorithm::FinishAggregatingInOrderAlgorithm(
|
||||
size_t num_inputs_,
|
||||
AggregatingTransformParamsPtr params_,
|
||||
const SortDescription & description_,
|
||||
size_t max_block_size_,
|
||||
size_t max_block_bytes_)
|
||||
: header(header_), num_inputs(num_inputs_), params(params_), max_block_size(max_block_size_), max_block_bytes(max_block_bytes_)
|
||||
size_t max_block_size_rows_,
|
||||
size_t max_block_size_bytes_)
|
||||
: header(header_), num_inputs(num_inputs_), params(params_), max_block_size_rows(max_block_size_rows_), max_block_size_bytes(max_block_size_bytes_)
|
||||
{
|
||||
for (const auto & column_description : description_)
|
||||
description.emplace_back(column_description, header_.getPositionByName(column_description.column_name));
|
||||
@ -118,7 +118,7 @@ IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge()
|
||||
inputs_to_update.pop_back();
|
||||
|
||||
/// Do not merge blocks, if there are too few rows or bytes.
|
||||
if (accumulated_rows >= max_block_size || accumulated_bytes >= max_block_bytes)
|
||||
if (accumulated_rows >= max_block_size_rows || accumulated_bytes >= max_block_size_bytes)
|
||||
status.chunk = prepareToMerge();
|
||||
|
||||
return status;
|
||||
|
@ -42,8 +42,8 @@ public:
|
||||
size_t num_inputs_,
|
||||
AggregatingTransformParamsPtr params_,
|
||||
const SortDescription & description_,
|
||||
size_t max_block_size_,
|
||||
size_t max_block_bytes_);
|
||||
size_t max_block_size_rows_,
|
||||
size_t max_block_size_bytes_);
|
||||
|
||||
void initialize(Inputs inputs) override;
|
||||
void consume(Input & input, size_t source_num) override;
|
||||
@ -79,8 +79,8 @@ private:
|
||||
size_t num_inputs;
|
||||
AggregatingTransformParamsPtr params;
|
||||
SortDescriptionWithPositions description;
|
||||
size_t max_block_size;
|
||||
size_t max_block_bytes;
|
||||
size_t max_block_size_rows;
|
||||
size_t max_block_size_bytes;
|
||||
|
||||
Inputs current_inputs;
|
||||
|
||||
|
@ -42,11 +42,12 @@ GraphiteRollupSortedAlgorithm::GraphiteRollupSortedAlgorithm(
|
||||
const Block & header_,
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
size_t max_block_size,
|
||||
size_t max_block_size_rows_,
|
||||
size_t max_block_size_bytes_,
|
||||
Graphite::Params params_,
|
||||
time_t time_of_merge_)
|
||||
: IMergingAlgorithmWithSharedChunks(header_, num_inputs, std::move(description_), nullptr, max_row_refs)
|
||||
, merged_data(header_.cloneEmptyColumns(), false, max_block_size)
|
||||
, merged_data(header_.cloneEmptyColumns(), false, max_block_size_rows_, max_block_size_bytes_)
|
||||
, params(std::move(params_))
|
||||
, time_of_merge(time_of_merge_)
|
||||
{
|
||||
|
@ -22,9 +22,13 @@ class GraphiteRollupSortedAlgorithm final : public IMergingAlgorithmWithSharedCh
|
||||
{
|
||||
public:
|
||||
GraphiteRollupSortedAlgorithm(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, size_t max_block_size,
|
||||
Graphite::Params params_, time_t time_of_merge_);
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
size_t max_block_size_rows_,
|
||||
size_t max_block_size_bytes_,
|
||||
Graphite::Params params_,
|
||||
time_t time_of_merge_);
|
||||
|
||||
Status merge() override;
|
||||
|
||||
|
@ -19,8 +19,8 @@ namespace ErrorCodes
|
||||
class MergedData
|
||||
{
|
||||
public:
|
||||
explicit MergedData(MutableColumns columns_, bool use_average_block_size_, UInt64 max_block_size_)
|
||||
: columns(std::move(columns_)), max_block_size(max_block_size_), use_average_block_size(use_average_block_size_)
|
||||
explicit MergedData(MutableColumns columns_, bool use_average_block_size_, UInt64 max_block_size_, UInt64 max_block_size_bytes_)
|
||||
: columns(std::move(columns_)), max_block_size(max_block_size_), max_block_size_bytes(max_block_size_bytes_), use_average_block_size(use_average_block_size_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -117,6 +117,16 @@ public:
|
||||
if (merged_rows >= max_block_size)
|
||||
return true;
|
||||
|
||||
/// Never return more than max_block_size_bytes
|
||||
if (max_block_size_bytes)
|
||||
{
|
||||
size_t merged_bytes = 0;
|
||||
for (const auto & column : columns)
|
||||
merged_bytes += column->allocatedBytes();
|
||||
if (merged_bytes >= max_block_size_bytes)
|
||||
return true;
|
||||
}
|
||||
|
||||
if (!use_average_block_size)
|
||||
return false;
|
||||
|
||||
@ -143,8 +153,9 @@ protected:
|
||||
UInt64 total_chunks = 0;
|
||||
UInt64 total_allocated_bytes = 0;
|
||||
|
||||
const UInt64 max_block_size;
|
||||
const bool use_average_block_size;
|
||||
const UInt64 max_block_size = 0;
|
||||
const UInt64 max_block_size_bytes = 0;
|
||||
const bool use_average_block_size = false;
|
||||
|
||||
bool need_flush = false;
|
||||
};
|
||||
|
@ -11,13 +11,14 @@ MergingSortedAlgorithm::MergingSortedAlgorithm(
|
||||
Block header_,
|
||||
size_t num_inputs,
|
||||
const SortDescription & description_,
|
||||
size_t max_block_size,
|
||||
size_t max_block_size_,
|
||||
size_t max_block_size_bytes_,
|
||||
SortingQueueStrategy sorting_queue_strategy_,
|
||||
UInt64 limit_,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
bool use_average_block_sizes)
|
||||
: header(std::move(header_))
|
||||
, merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size)
|
||||
, merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size_, max_block_size_bytes_)
|
||||
, description(description_)
|
||||
, limit(limit_)
|
||||
, out_row_sources_buf(out_row_sources_buf_)
|
||||
|
@ -17,7 +17,8 @@ public:
|
||||
Block header_,
|
||||
size_t num_inputs,
|
||||
const SortDescription & description_,
|
||||
size_t max_block_size,
|
||||
size_t max_block_size_,
|
||||
size_t max_block_size_bytes_,
|
||||
SortingQueueStrategy sorting_queue_strategy_,
|
||||
UInt64 limit_ = 0,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
|
@ -17,12 +17,13 @@ ReplacingSortedAlgorithm::ReplacingSortedAlgorithm(
|
||||
SortDescription description_,
|
||||
const String & is_deleted_column,
|
||||
const String & version_column,
|
||||
size_t max_block_size,
|
||||
size_t max_block_size_rows,
|
||||
size_t max_block_size_bytes,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
bool use_average_block_sizes,
|
||||
bool cleanup_)
|
||||
: IMergingAlgorithmWithSharedChunks(header_, num_inputs, std::move(description_), out_row_sources_buf_, max_row_refs)
|
||||
, merged_data(header_.cloneEmptyColumns(), use_average_block_sizes, max_block_size), cleanup(cleanup_)
|
||||
, merged_data(header_.cloneEmptyColumns(), use_average_block_sizes, max_block_size_rows, max_block_size_bytes), cleanup(cleanup_)
|
||||
{
|
||||
if (!is_deleted_column.empty())
|
||||
is_deleted_column_number = header_.getPositionByName(is_deleted_column);
|
||||
|
@ -23,7 +23,8 @@ public:
|
||||
SortDescription description_,
|
||||
const String & is_deleted_column,
|
||||
const String & version_column,
|
||||
size_t max_block_size,
|
||||
size_t max_block_size_rows,
|
||||
size_t max_block_size_bytes,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool use_average_block_sizes = false,
|
||||
bool cleanup = false);
|
||||
|
@ -497,8 +497,8 @@ static void setRow(Row & row, const ColumnRawPtrs & raw_columns, size_t row_num,
|
||||
|
||||
|
||||
SummingSortedAlgorithm::SummingMergedData::SummingMergedData(
|
||||
MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def_)
|
||||
: MergedData(std::move(columns_), false, max_block_size_)
|
||||
MutableColumns columns_, UInt64 max_block_size_rows_, UInt64 max_block_size_bytes_, ColumnsDefinition & def_)
|
||||
: MergedData(std::move(columns_), false, max_block_size_rows_, max_block_size_bytes_)
|
||||
, def(def_)
|
||||
{
|
||||
current_row.resize(def.column_names.size());
|
||||
@ -686,10 +686,11 @@ SummingSortedAlgorithm::SummingSortedAlgorithm(
|
||||
SortDescription description_,
|
||||
const Names & column_names_to_sum,
|
||||
const Names & partition_key_columns,
|
||||
size_t max_block_size)
|
||||
size_t max_block_size_rows,
|
||||
size_t max_block_size_bytes)
|
||||
: IMergingAlgorithmWithDelayedChunk(header_, num_inputs, std::move(description_))
|
||||
, columns_definition(defineColumns(header_, description, column_names_to_sum, partition_key_columns))
|
||||
, merged_data(getMergedDataColumns(header_, columns_definition), max_block_size, columns_definition)
|
||||
, merged_data(getMergedDataColumns(header_, columns_definition), max_block_size_rows, max_block_size_bytes, columns_definition)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -22,7 +22,8 @@ public:
|
||||
const Names & column_names_to_sum,
|
||||
/// List of partition key columns. They have to be excluded.
|
||||
const Names & partition_key_columns,
|
||||
size_t max_block_size);
|
||||
size_t max_block_size_rows,
|
||||
size_t max_block_size_bytes);
|
||||
|
||||
void initialize(Inputs inputs) override;
|
||||
void consume(Input & input, size_t source_num) override;
|
||||
@ -63,7 +64,7 @@ public:
|
||||
using MergedData::insertRow;
|
||||
|
||||
public:
|
||||
SummingMergedData(MutableColumns columns_, UInt64 max_block_size_, ColumnsDefinition & def_);
|
||||
SummingMergedData(MutableColumns columns_, UInt64 max_block_size_rows, UInt64 max_block_size_bytes_, ColumnsDefinition & def_);
|
||||
|
||||
void startGroup(ColumnRawPtrs & raw_columns, size_t row);
|
||||
void finishGroup();
|
||||
|
@ -12,13 +12,14 @@ VersionedCollapsingAlgorithm::VersionedCollapsingAlgorithm(
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
const String & sign_column_,
|
||||
size_t max_block_size,
|
||||
size_t max_block_size_rows_,
|
||||
size_t max_block_size_bytes_,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
bool use_average_block_sizes)
|
||||
: IMergingAlgorithmWithSharedChunks(header_, num_inputs, std::move(description_), out_row_sources_buf_, MAX_ROWS_IN_MULTIVERSION_QUEUE)
|
||||
, merged_data(header_.cloneEmptyColumns(), use_average_block_sizes, max_block_size)
|
||||
, merged_data(header_.cloneEmptyColumns(), use_average_block_sizes, max_block_size_rows_, max_block_size_bytes_)
|
||||
/// -1 for +1 in FixedSizeDequeWithGaps's internal buffer. 3 is a reasonable minimum size to collapse anything.
|
||||
, max_rows_in_queue(std::min(std::max<size_t>(3, max_block_size), MAX_ROWS_IN_MULTIVERSION_QUEUE) - 1)
|
||||
, max_rows_in_queue(std::min(std::max<size_t>(3, max_block_size_rows_), MAX_ROWS_IN_MULTIVERSION_QUEUE) - 1)
|
||||
, current_keys(max_rows_in_queue)
|
||||
{
|
||||
sign_column_number = header_.getPositionByName(sign_column_);
|
||||
|
@ -20,7 +20,8 @@ public:
|
||||
VersionedCollapsingAlgorithm(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, const String & sign_column_,
|
||||
size_t max_block_size,
|
||||
size_t max_block_size_rows,
|
||||
size_t max_block_size_bytes,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool use_average_block_sizes = false);
|
||||
|
||||
|
@ -16,7 +16,8 @@ public:
|
||||
SortDescription description_,
|
||||
const String & sign_column,
|
||||
bool only_positive_sign,
|
||||
size_t max_block_size,
|
||||
size_t max_block_size_rows,
|
||||
size_t max_block_size_bytes,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool use_average_block_sizes = false)
|
||||
: IMergingTransform(
|
||||
@ -26,7 +27,8 @@ public:
|
||||
std::move(description_),
|
||||
sign_column,
|
||||
only_positive_sign,
|
||||
max_block_size,
|
||||
max_block_size_rows,
|
||||
max_block_size_bytes,
|
||||
&Poco::Logger::get("CollapsingSortedTransform"),
|
||||
out_row_sources_buf_,
|
||||
use_average_block_sizes)
|
||||
|
@ -17,16 +17,16 @@ public:
|
||||
size_t num_inputs,
|
||||
AggregatingTransformParamsPtr params,
|
||||
SortDescription description,
|
||||
size_t max_block_size,
|
||||
size_t max_block_bytes)
|
||||
size_t max_block_size_rows,
|
||||
size_t max_block_size_bytes)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, {}, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false,
|
||||
header,
|
||||
num_inputs,
|
||||
params,
|
||||
std::move(description),
|
||||
max_block_size,
|
||||
max_block_bytes)
|
||||
max_block_size_rows,
|
||||
max_block_size_bytes)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -11,15 +11,20 @@ class GraphiteRollupSortedTransform final : public IMergingTransform<GraphiteRol
|
||||
{
|
||||
public:
|
||||
GraphiteRollupSortedTransform(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, size_t max_block_size,
|
||||
Graphite::Params params_, time_t time_of_merge_)
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
size_t max_block_size_rows,
|
||||
size_t max_block_size_bytes,
|
||||
Graphite::Params params_,
|
||||
time_t time_of_merge_)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
max_block_size,
|
||||
max_block_size_rows,
|
||||
max_block_size_bytes,
|
||||
std::move(params_),
|
||||
time_of_merge_)
|
||||
{
|
||||
|
@ -11,7 +11,8 @@ MergingSortedTransform::MergingSortedTransform(
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
const SortDescription & description_,
|
||||
size_t max_block_size,
|
||||
size_t max_block_size_rows,
|
||||
size_t max_block_size_bytes,
|
||||
SortingQueueStrategy sorting_queue_strategy,
|
||||
UInt64 limit_,
|
||||
bool always_read_till_end_,
|
||||
@ -29,7 +30,8 @@ MergingSortedTransform::MergingSortedTransform(
|
||||
header,
|
||||
num_inputs,
|
||||
description_,
|
||||
max_block_size,
|
||||
max_block_size_rows,
|
||||
max_block_size_bytes,
|
||||
sorting_queue_strategy,
|
||||
limit_,
|
||||
out_row_sources_buf_,
|
||||
|
@ -15,7 +15,8 @@ public:
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
const SortDescription & description,
|
||||
size_t max_block_size,
|
||||
size_t max_block_size_rows,
|
||||
size_t max_block_size_bytes,
|
||||
SortingQueueStrategy sorting_queue_strategy,
|
||||
UInt64 limit_ = 0,
|
||||
bool always_read_till_end_ = false,
|
||||
|
@ -15,7 +15,8 @@ public:
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_,
|
||||
const String & is_deleted_column, const String & version_column,
|
||||
size_t max_block_size,
|
||||
size_t max_block_size_rows,
|
||||
size_t max_block_size_bytes,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool use_average_block_sizes = false,
|
||||
bool cleanup = false)
|
||||
@ -26,7 +27,8 @@ public:
|
||||
std::move(description_),
|
||||
is_deleted_column,
|
||||
version_column,
|
||||
max_block_size,
|
||||
max_block_size_rows,
|
||||
max_block_size_bytes,
|
||||
out_row_sources_buf_,
|
||||
use_average_block_sizes,
|
||||
cleanup)
|
||||
|
@ -17,7 +17,9 @@ public:
|
||||
/// List of columns to be summed. If empty, all numeric columns that are not in the description are taken.
|
||||
const Names & column_names_to_sum,
|
||||
const Names & partition_key_columns,
|
||||
size_t max_block_size)
|
||||
size_t max_block_size_rows,
|
||||
size_t max_block_size_bytes
|
||||
)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false,
|
||||
header,
|
||||
@ -25,7 +27,8 @@ public:
|
||||
std::move(description_),
|
||||
column_names_to_sum,
|
||||
partition_key_columns,
|
||||
max_block_size)
|
||||
max_block_size_rows,
|
||||
max_block_size_bytes)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -15,7 +15,8 @@ public:
|
||||
VersionedCollapsingTransform(
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, const String & sign_column_,
|
||||
size_t max_block_size,
|
||||
size_t max_block_size_rows,
|
||||
size_t max_block_size_bytes,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool use_average_block_sizes = false)
|
||||
: IMergingTransform(
|
||||
@ -24,7 +25,8 @@ public:
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
sign_column_,
|
||||
max_block_size,
|
||||
max_block_size_rows,
|
||||
max_block_size_bytes,
|
||||
out_row_sources_buf_,
|
||||
use_average_block_sizes)
|
||||
{
|
||||
|
@ -852,7 +852,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
|
||||
if (pipe.numOutputPorts() > 1)
|
||||
{
|
||||
auto transform = std::make_shared<MergingSortedTransform>(
|
||||
pipe.getHeader(), pipe.numOutputPorts(), sort_description, max_block_size, SortingQueueStrategy::Batch);
|
||||
pipe.getHeader(), pipe.numOutputPorts(), sort_description, max_block_size, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch);
|
||||
|
||||
pipe.addTransform(std::move(transform));
|
||||
}
|
||||
@ -898,31 +898,31 @@ static void addMergingFinal(
|
||||
{
|
||||
case MergeTreeData::MergingParams::Ordinary:
|
||||
return std::make_shared<MergingSortedTransform>(header, num_outputs,
|
||||
sort_description, max_block_size, SortingQueueStrategy::Batch);
|
||||
sort_description, max_block_size, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch);
|
||||
|
||||
case MergeTreeData::MergingParams::Collapsing:
|
||||
return std::make_shared<CollapsingSortedTransform>(header, num_outputs,
|
||||
sort_description, merging_params.sign_column, true, max_block_size);
|
||||
sort_description, merging_params.sign_column, true, max_block_size, /*max_block_size_bytes=*/0);
|
||||
|
||||
case MergeTreeData::MergingParams::Summing:
|
||||
return std::make_shared<SummingSortedTransform>(header, num_outputs,
|
||||
sort_description, merging_params.columns_to_sum, partition_key_columns, max_block_size);
|
||||
sort_description, merging_params.columns_to_sum, partition_key_columns, max_block_size, /*max_block_size_bytes=*/0);
|
||||
|
||||
case MergeTreeData::MergingParams::Aggregating:
|
||||
return std::make_shared<AggregatingSortedTransform>(header, num_outputs,
|
||||
sort_description, max_block_size);
|
||||
sort_description, max_block_size, /*max_block_size_bytes=*/0);
|
||||
|
||||
case MergeTreeData::MergingParams::Replacing:
|
||||
return std::make_shared<ReplacingSortedTransform>(header, num_outputs,
|
||||
sort_description, merging_params.is_deleted_column, merging_params.version_column, max_block_size, /*out_row_sources_buf_*/ nullptr, /*use_average_block_sizes*/ false, /*cleanup*/ !merging_params.is_deleted_column.empty());
|
||||
sort_description, merging_params.is_deleted_column, merging_params.version_column, max_block_size, /*max_block_size_bytes=*/0, /*out_row_sources_buf_*/ nullptr, /*use_average_block_sizes*/ false, /*cleanup*/ !merging_params.is_deleted_column.empty());
|
||||
|
||||
case MergeTreeData::MergingParams::VersionedCollapsing:
|
||||
return std::make_shared<VersionedCollapsingTransform>(header, num_outputs,
|
||||
sort_description, merging_params.sign_column, max_block_size);
|
||||
sort_description, merging_params.sign_column, max_block_size, /*max_block_size_bytes=*/0);
|
||||
|
||||
case MergeTreeData::MergingParams::Graphite:
|
||||
return std::make_shared<GraphiteRollupSortedTransform>(header, num_outputs,
|
||||
sort_description, max_block_size, merging_params.graphite_params, now);
|
||||
sort_description, max_block_size, /*max_block_size_bytes=*/0, merging_params.graphite_params, now);
|
||||
}
|
||||
|
||||
UNREACHABLE();
|
||||
|
@ -176,6 +176,7 @@ void SortingStep::mergingSorted(QueryPipelineBuilder & pipeline, const SortDescr
|
||||
pipeline.getNumStreams(),
|
||||
result_sort_desc,
|
||||
sort_settings.max_block_size,
|
||||
/*max_block_size_bytes=*/0,
|
||||
SortingQueueStrategy::Batch,
|
||||
limit_,
|
||||
always_read_till_end);
|
||||
@ -269,6 +270,7 @@ void SortingStep::fullSort(
|
||||
pipeline.getNumStreams(),
|
||||
result_sort_desc,
|
||||
sort_settings.max_block_size,
|
||||
/*max_block_size_bytes=*/0,
|
||||
SortingQueueStrategy::Batch,
|
||||
limit_,
|
||||
always_read_till_end);
|
||||
|
@ -186,6 +186,7 @@ void MergeSortingTransform::consume(Chunk chunk)
|
||||
0,
|
||||
description,
|
||||
max_merged_block_size,
|
||||
/*max_merged_block_size_bytes*/0,
|
||||
SortingQueueStrategy::Batch,
|
||||
limit,
|
||||
/*always_read_till_end_=*/ false,
|
||||
|
@ -83,7 +83,7 @@ TEST(MergingSortedTest, SimpleBlockSizeTest)
|
||||
EXPECT_EQ(pipe.numOutputPorts(), 3);
|
||||
|
||||
auto transform = std::make_shared<MergingSortedTransform>(pipe.getHeader(), pipe.numOutputPorts(), sort_description,
|
||||
DEFAULT_MERGE_BLOCK_SIZE, SortingQueueStrategy::Batch, 0, false, nullptr, false, true);
|
||||
8192, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch, 0, false, nullptr, false, true);
|
||||
|
||||
pipe.addTransform(std::move(transform));
|
||||
|
||||
@ -125,7 +125,7 @@ TEST(MergingSortedTest, MoreInterestingBlockSizes)
|
||||
EXPECT_EQ(pipe.numOutputPorts(), 3);
|
||||
|
||||
auto transform = std::make_shared<MergingSortedTransform>(pipe.getHeader(), pipe.numOutputPorts(), sort_description,
|
||||
DEFAULT_MERGE_BLOCK_SIZE, SortingQueueStrategy::Batch, 0, false, nullptr, false, true);
|
||||
8192, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch, 0, false, nullptr, false, true);
|
||||
|
||||
pipe.addTransform(std::move(transform));
|
||||
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <Backups/BackupEntryFromSmallFile.h>
|
||||
#include <Backups/BackupEntryFromImmutableFile.h>
|
||||
#include <Disks/SingleDiskVolume.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -415,6 +416,7 @@ MutableDataPartStoragePtr DataPartStorageOnDiskBase::freeze(
|
||||
|
||||
disk->removeFileIfExists(fs::path(to) / dir_path / "delete-on-destroy.txt");
|
||||
disk->removeFileIfExists(fs::path(to) / dir_path / "txn_version.txt");
|
||||
disk->removeFileIfExists(fs::path(to) / dir_path / IMergeTreeDataPart::METADATA_VERSION_FILE_NAME);
|
||||
|
||||
auto single_disk_volume = std::make_shared<SingleDiskVolume>(disk->getName(), disk, 0);
|
||||
|
||||
@ -461,6 +463,7 @@ void DataPartStorageOnDiskBase::rename(
|
||||
|
||||
if (volume->getDisk()->exists(to))
|
||||
{
|
||||
/// FIXME it should be logical error
|
||||
if (remove_new_dir_if_exists)
|
||||
{
|
||||
Names files;
|
||||
@ -471,7 +474,8 @@ void DataPartStorageOnDiskBase::rename(
|
||||
"Part directory {} already exists and contains {} files. Removing it.",
|
||||
fullPath(volume->getDisk(), to), files.size());
|
||||
|
||||
executeWriteOperation([&](auto & disk) { disk.removeRecursive(to); });
|
||||
/// Do not remove blobs if they exist
|
||||
executeWriteOperation([&](auto & disk) { disk.removeSharedRecursive(to, true, {}); });
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -24,7 +24,7 @@ template <typename T>
|
||||
std::optional<EphemeralLockInZooKeeper> createEphemeralLockInZooKeeper(
|
||||
const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const T & deduplication_path)
|
||||
{
|
||||
constexpr bool async_insert = std::is_same_v<T, std::vector<String>>;
|
||||
static constexpr bool async_insert = std::is_same_v<T, std::vector<String>>;
|
||||
|
||||
String path;
|
||||
|
||||
@ -42,16 +42,15 @@ std::optional<EphemeralLockInZooKeeper> createEphemeralLockInZooKeeper(
|
||||
if constexpr (async_insert)
|
||||
{
|
||||
for (const auto & single_dedup_path : deduplication_path)
|
||||
{
|
||||
ops.emplace_back(zkutil::makeCreateRequest(single_dedup_path, "", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(single_dedup_path, -1));
|
||||
}
|
||||
zkutil::addCheckNotExistsRequest(ops, *zookeeper_, single_dedup_path);
|
||||
}
|
||||
else
|
||||
{
|
||||
ops.emplace_back(zkutil::makeCreateRequest(deduplication_path, "", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(deduplication_path, -1));
|
||||
zkutil::addCheckNotExistsRequest(ops, *zookeeper_, deduplication_path);
|
||||
}
|
||||
|
||||
auto deduplication_path_ops_size = ops.size();
|
||||
|
||||
ops.emplace_back(zkutil::makeCreateRequest(path_prefix_, holder_path, zkutil::CreateMode::EphemeralSequential));
|
||||
Coordination::Responses responses;
|
||||
Coordination::Error e = zookeeper_->tryMulti(ops, responses);
|
||||
@ -60,9 +59,10 @@ std::optional<EphemeralLockInZooKeeper> createEphemeralLockInZooKeeper(
|
||||
if constexpr (async_insert)
|
||||
{
|
||||
auto failed_idx = zkutil::getFailedOpIndex(Coordination::Error::ZNODEEXISTS, responses);
|
||||
if (failed_idx < deduplication_path.size() * 2)
|
||||
|
||||
if (failed_idx < deduplication_path_ops_size)
|
||||
{
|
||||
const String & failed_op_path = deduplication_path[failed_idx / 2];
|
||||
const String & failed_op_path = ops[failed_idx]->getPath();
|
||||
LOG_DEBUG(
|
||||
&Poco::Logger::get("createEphemeralLockInZooKeeper"),
|
||||
"Deduplication path already exists: deduplication_path={}",
|
||||
|
@ -1357,14 +1357,6 @@ void IMergeTreeDataPart::loadColumns(bool require)
|
||||
else
|
||||
{
|
||||
loaded_metadata_version = metadata_snapshot->getMetadataVersion();
|
||||
|
||||
if (!is_readonly_storage)
|
||||
{
|
||||
writeMetadata(METADATA_VERSION_FILE_NAME, {}, [loaded_metadata_version](auto & buffer)
|
||||
{
|
||||
writeIntText(loaded_metadata_version, buffer);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
setColumns(loaded_columns, infos, loaded_metadata_version);
|
||||
|
@ -1999,9 +1999,9 @@ static BoolMask forAnyHyperrectangle(
|
||||
if (left_bounded && right_bounded)
|
||||
hyperrectangle[prefix_size] = Range(left_keys[prefix_size], true, right_keys[prefix_size], true);
|
||||
else if (left_bounded)
|
||||
hyperrectangle[prefix_size] = Range::createLeftBounded(left_keys[prefix_size], true);
|
||||
hyperrectangle[prefix_size] = Range::createLeftBounded(left_keys[prefix_size], true, data_types[prefix_size]->isNullable());
|
||||
else if (right_bounded)
|
||||
hyperrectangle[prefix_size] = Range::createRightBounded(right_keys[prefix_size], true);
|
||||
hyperrectangle[prefix_size] = Range::createRightBounded(right_keys[prefix_size], true, data_types[prefix_size]->isNullable());
|
||||
|
||||
return callback(hyperrectangle);
|
||||
}
|
||||
|
@ -31,6 +31,7 @@ MergeListElement::MergeListElement(
|
||||
source_part_paths.emplace_back(source_part->getDataPartStorage().getFullPath());
|
||||
|
||||
total_size_bytes_compressed += source_part->getBytesOnDisk();
|
||||
total_size_bytes_uncompressed += source_part->getTotalColumnsSize().data_uncompressed;
|
||||
total_size_marks += source_part->getMarksCount();
|
||||
total_rows_count += source_part->index_granularity.getTotalRows();
|
||||
}
|
||||
@ -57,6 +58,7 @@ MergeInfo MergeListElement::getInfo() const
|
||||
res.progress = progress.load(std::memory_order_relaxed);
|
||||
res.num_parts = num_parts;
|
||||
res.total_size_bytes_compressed = total_size_bytes_compressed;
|
||||
res.total_size_bytes_uncompressed = total_size_bytes_uncompressed;
|
||||
res.total_size_marks = total_size_marks;
|
||||
res.total_rows_count = total_rows_count;
|
||||
res.bytes_read_uncompressed = bytes_read_uncompressed.load(std::memory_order_relaxed);
|
||||
|
@ -40,6 +40,7 @@ struct MergeInfo
|
||||
Float64 progress;
|
||||
UInt64 num_parts;
|
||||
UInt64 total_size_bytes_compressed;
|
||||
UInt64 total_size_bytes_uncompressed;
|
||||
UInt64 total_size_marks;
|
||||
UInt64 total_rows_count;
|
||||
UInt64 bytes_read_uncompressed;
|
||||
@ -82,6 +83,7 @@ struct MergeListElement : boost::noncopyable
|
||||
std::atomic<bool> is_cancelled{};
|
||||
|
||||
UInt64 total_size_bytes_compressed{};
|
||||
UInt64 total_size_bytes_uncompressed{};
|
||||
UInt64 total_size_marks{};
|
||||
UInt64 total_rows_count{};
|
||||
std::atomic<UInt64> bytes_read_uncompressed{};
|
||||
|
@ -921,7 +921,9 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream()
|
||||
/// If merge is vertical we cannot calculate it
|
||||
ctx->blocks_are_granules_size = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical);
|
||||
|
||||
UInt64 merge_block_size = data_settings->merge_max_block_size;
|
||||
/// There is no sense to have the block size bigger than one granule for merge operations.
|
||||
const UInt64 merge_block_size_rows = data_settings->merge_max_block_size;
|
||||
const UInt64 merge_block_size_bytes = data_settings->merge_max_block_size_bytes;
|
||||
|
||||
switch (ctx->merging_params.mode)
|
||||
{
|
||||
@ -930,7 +932,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream()
|
||||
header,
|
||||
pipes.size(),
|
||||
sort_description,
|
||||
merge_block_size,
|
||||
merge_block_size_rows,
|
||||
merge_block_size_bytes,
|
||||
SortingQueueStrategy::Default,
|
||||
/* limit_= */0,
|
||||
/* always_read_till_end_= */false,
|
||||
@ -942,35 +945,35 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream()
|
||||
case MergeTreeData::MergingParams::Collapsing:
|
||||
merged_transform = std::make_shared<CollapsingSortedTransform>(
|
||||
header, pipes.size(), sort_description, ctx->merging_params.sign_column, false,
|
||||
merge_block_size, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size);
|
||||
merge_block_size_rows, merge_block_size_bytes, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Summing:
|
||||
merged_transform = std::make_shared<SummingSortedTransform>(
|
||||
header, pipes.size(), sort_description, ctx->merging_params.columns_to_sum, partition_key_columns, merge_block_size);
|
||||
header, pipes.size(), sort_description, ctx->merging_params.columns_to_sum, partition_key_columns, merge_block_size_rows, merge_block_size_bytes);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Aggregating:
|
||||
merged_transform = std::make_shared<AggregatingSortedTransform>(header, pipes.size(), sort_description, merge_block_size);
|
||||
merged_transform = std::make_shared<AggregatingSortedTransform>(header, pipes.size(), sort_description, merge_block_size_rows, merge_block_size_bytes);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Replacing:
|
||||
merged_transform = std::make_shared<ReplacingSortedTransform>(
|
||||
header, pipes.size(), sort_description, ctx->merging_params.is_deleted_column, ctx->merging_params.version_column,
|
||||
merge_block_size, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size,
|
||||
merge_block_size_rows, merge_block_size_bytes, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size,
|
||||
(data_settings->clean_deleted_rows != CleanDeletedRows::Never) || global_ctx->cleanup);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Graphite:
|
||||
merged_transform = std::make_shared<GraphiteRollupSortedTransform>(
|
||||
header, pipes.size(), sort_description, merge_block_size,
|
||||
header, pipes.size(), sort_description, merge_block_size_rows, merge_block_size_bytes,
|
||||
ctx->merging_params.graphite_params, global_ctx->time_of_merge);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::VersionedCollapsing:
|
||||
merged_transform = std::make_shared<VersionedCollapsingTransform>(
|
||||
header, pipes.size(), sort_description, ctx->merging_params.sign_column,
|
||||
merge_block_size, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size);
|
||||
merge_block_size_rows, merge_block_size_bytes, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size);
|
||||
break;
|
||||
}
|
||||
|
||||
@ -1011,7 +1014,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream()
|
||||
|
||||
MergeAlgorithm MergeTask::ExecuteAndFinalizeHorizontalPart::chooseMergeAlgorithm() const
|
||||
{
|
||||
const size_t sum_rows_upper_bound = global_ctx->merge_list_element_ptr->total_rows_count;
|
||||
const size_t total_rows_count = global_ctx->merge_list_element_ptr->total_rows_count;
|
||||
const size_t total_size_bytes_uncompressed = global_ctx->merge_list_element_ptr->total_size_bytes_uncompressed;
|
||||
const auto data_settings = global_ctx->data->getSettings();
|
||||
|
||||
if (global_ctx->deduplicate)
|
||||
@ -1042,11 +1046,13 @@ MergeAlgorithm MergeTask::ExecuteAndFinalizeHorizontalPart::chooseMergeAlgorithm
|
||||
|
||||
bool enough_ordinary_cols = global_ctx->gathering_columns.size() >= data_settings->vertical_merge_algorithm_min_columns_to_activate;
|
||||
|
||||
bool enough_total_rows = sum_rows_upper_bound >= data_settings->vertical_merge_algorithm_min_rows_to_activate;
|
||||
bool enough_total_rows = total_rows_count >= data_settings->vertical_merge_algorithm_min_rows_to_activate;
|
||||
|
||||
bool enough_total_bytes = total_size_bytes_uncompressed >= data_settings->vertical_merge_algorithm_min_bytes_to_activate;
|
||||
|
||||
bool no_parts_overflow = global_ctx->future_part->parts.size() <= RowSourcePart::MAX_PARTS;
|
||||
|
||||
auto merge_alg = (is_supported_storage && enough_total_rows && enough_ordinary_cols && no_parts_overflow) ?
|
||||
auto merge_alg = (is_supported_storage && enough_total_rows && enough_total_bytes && enough_ordinary_cols && no_parts_overflow) ?
|
||||
MergeAlgorithm::Vertical : MergeAlgorithm::Horizontal;
|
||||
|
||||
return merge_alg;
|
||||
|
@ -280,23 +280,23 @@ Block MergeTreeDataWriter::mergeBlock(
|
||||
return nullptr;
|
||||
case MergeTreeData::MergingParams::Replacing:
|
||||
return std::make_shared<ReplacingSortedAlgorithm>(
|
||||
block, 1, sort_description, merging_params.is_deleted_column, merging_params.version_column, block_size + 1);
|
||||
block, 1, sort_description, merging_params.is_deleted_column, merging_params.version_column, block_size + 1, /*block_size_bytes=*/0);
|
||||
case MergeTreeData::MergingParams::Collapsing:
|
||||
return std::make_shared<CollapsingSortedAlgorithm>(
|
||||
block, 1, sort_description, merging_params.sign_column,
|
||||
false, block_size + 1, &Poco::Logger::get("MergeTreeDataWriter"));
|
||||
false, block_size + 1, /*block_size_bytes=*/0, &Poco::Logger::get("MergeTreeDataWriter"));
|
||||
case MergeTreeData::MergingParams::Summing:
|
||||
return std::make_shared<SummingSortedAlgorithm>(
|
||||
block, 1, sort_description, merging_params.columns_to_sum,
|
||||
partition_key_columns, block_size + 1);
|
||||
partition_key_columns, block_size + 1, /*block_size_bytes=*/0);
|
||||
case MergeTreeData::MergingParams::Aggregating:
|
||||
return std::make_shared<AggregatingSortedAlgorithm>(block, 1, sort_description, block_size + 1);
|
||||
return std::make_shared<AggregatingSortedAlgorithm>(block, 1, sort_description, block_size + 1, /*block_size_bytes=*/0);
|
||||
case MergeTreeData::MergingParams::VersionedCollapsing:
|
||||
return std::make_shared<VersionedCollapsingAlgorithm>(
|
||||
block, 1, sort_description, merging_params.sign_column, block_size + 1);
|
||||
block, 1, sort_description, merging_params.sign_column, block_size + 1, /*block_size_bytes=*/0);
|
||||
case MergeTreeData::MergingParams::Graphite:
|
||||
return std::make_shared<GraphiteRollupSortedAlgorithm>(
|
||||
block, 1, sort_description, block_size + 1, merging_params.graphite_params, time(nullptr));
|
||||
block, 1, sort_description, block_size + 1, /*block_size_bytes=*/0, merging_params.graphite_params, time(nullptr));
|
||||
}
|
||||
|
||||
UNREACHABLE();
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Storages/MergeTree/MergeTreeIndexReader.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
|
||||
|
||||
namespace
|
||||
{
|
||||
@ -20,7 +21,7 @@ std::unique_ptr<MergeTreeReaderStream> makeIndexReader(
|
||||
auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr;
|
||||
|
||||
return std::make_unique<MergeTreeReaderStream>(
|
||||
part->getDataPartStoragePtr(),
|
||||
std::make_shared<LoadedMergeTreeDataPartInfoForReader>(part),
|
||||
index->getFileName(), extension, marks_count,
|
||||
all_mark_ranges,
|
||||
std::move(settings), mark_cache, uncompressed_cache,
|
||||
|
@ -30,7 +30,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
MergeTreeMarksLoader::MergeTreeMarksLoader(
|
||||
DataPartStoragePtr data_part_storage_,
|
||||
MergeTreeDataPartInfoForReaderPtr data_part_reader_,
|
||||
MarkCache * mark_cache_,
|
||||
const String & mrk_path_,
|
||||
size_t marks_count_,
|
||||
@ -39,7 +39,7 @@ MergeTreeMarksLoader::MergeTreeMarksLoader(
|
||||
const ReadSettings & read_settings_,
|
||||
ThreadPool * load_marks_threadpool_,
|
||||
size_t columns_in_mark_)
|
||||
: data_part_storage(std::move(data_part_storage_))
|
||||
: data_part_reader(data_part_reader_)
|
||||
, mark_cache(mark_cache_)
|
||||
, mrk_path(mrk_path_)
|
||||
, marks_count(marks_count_)
|
||||
@ -98,6 +98,8 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl()
|
||||
/// Memory for marks must not be accounted as memory usage for query, because they are stored in shared cache.
|
||||
MemoryTrackerBlockerInThread temporarily_disable_memory_tracker;
|
||||
|
||||
auto data_part_storage = data_part_reader->getDataPartStorage();
|
||||
|
||||
size_t file_size = data_part_storage->getFileSize(mrk_path);
|
||||
size_t mark_size = index_granularity_info.getMarkSizeInBytes(columns_in_mark);
|
||||
size_t expected_uncompressed_size = mark_size * marks_count;
|
||||
@ -177,6 +179,8 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarks()
|
||||
{
|
||||
MarkCache::MappedPtr loaded_marks;
|
||||
|
||||
auto data_part_storage = data_part_reader->getDataPartStorage();
|
||||
|
||||
if (mark_cache)
|
||||
{
|
||||
auto key = mark_cache->hash(fs::path(data_part_storage->getFullPath()) / mrk_path);
|
||||
|
@ -1,9 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/MergeTree/IDataPartStorage.h>
|
||||
#include <Storages/MarkCache.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <Common/ThreadPool_fwd.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPartInfoForReader.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -18,7 +18,7 @@ public:
|
||||
using MarksPtr = MarkCache::MappedPtr;
|
||||
|
||||
MergeTreeMarksLoader(
|
||||
DataPartStoragePtr data_part_storage_,
|
||||
MergeTreeDataPartInfoForReaderPtr data_part_reader_,
|
||||
MarkCache * mark_cache_,
|
||||
const String & mrk_path,
|
||||
size_t marks_count_,
|
||||
@ -33,7 +33,7 @@ public:
|
||||
MarkInCompressedFile getMark(size_t row_index, size_t column_index = 0);
|
||||
|
||||
private:
|
||||
DataPartStoragePtr data_part_storage;
|
||||
MergeTreeDataPartInfoForReaderPtr data_part_reader;
|
||||
MarkCache * mark_cache = nullptr;
|
||||
String mrk_path;
|
||||
size_t marks_count;
|
||||
|
@ -36,7 +36,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact(
|
||||
settings_,
|
||||
avg_value_size_hints_)
|
||||
, marks_loader(
|
||||
data_part_info_for_read_->getDataPartStorage(),
|
||||
data_part_info_for_read_,
|
||||
mark_cache,
|
||||
data_part_info_for_read_->getIndexGranularityInfo().getMarksFilePath(MergeTreeDataPartCompact::DATA_FILE_NAME),
|
||||
data_part_info_for_read_->getMarksCount(),
|
||||
|
@ -15,7 +15,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
MergeTreeReaderStream::MergeTreeReaderStream(
|
||||
DataPartStoragePtr data_part_storage_,
|
||||
MergeTreeDataPartInfoForReaderPtr data_part_reader_,
|
||||
const String & path_prefix_,
|
||||
const String & data_file_extension_,
|
||||
size_t marks_count_,
|
||||
@ -35,7 +35,7 @@ MergeTreeReaderStream::MergeTreeReaderStream(
|
||||
, all_mark_ranges(all_mark_ranges_)
|
||||
, file_size(file_size_)
|
||||
, uncompressed_cache(uncompressed_cache_)
|
||||
, data_part_storage(std::move(data_part_storage_))
|
||||
, data_part_storage(data_part_reader_->getDataPartStorage())
|
||||
, path_prefix(path_prefix_)
|
||||
, data_file_extension(data_file_extension_)
|
||||
, is_low_cardinality_dictionary(is_low_cardinality_dictionary_)
|
||||
@ -44,7 +44,7 @@ MergeTreeReaderStream::MergeTreeReaderStream(
|
||||
, save_marks_in_cache(settings.save_marks_in_cache)
|
||||
, index_granularity_info(index_granularity_info_)
|
||||
, marks_loader(
|
||||
data_part_storage,
|
||||
data_part_reader_,
|
||||
mark_cache,
|
||||
index_granularity_info->getMarksFilePath(path_prefix),
|
||||
marks_count,
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Compression/CompressedReadBufferFromFile.h>
|
||||
#include <Storages/MergeTree/MergeTreeIOSettings.h>
|
||||
#include <Storages/MergeTree/MergeTreeMarksLoader.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPartInfoForReader.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -19,7 +20,7 @@ class MergeTreeReaderStream
|
||||
{
|
||||
public:
|
||||
MergeTreeReaderStream(
|
||||
DataPartStoragePtr data_part_storage_,
|
||||
MergeTreeDataPartInfoForReaderPtr data_part_reader_,
|
||||
const String & path_prefix_,
|
||||
const String & data_file_extension_,
|
||||
size_t marks_count_,
|
||||
|
@ -242,7 +242,7 @@ void MergeTreeReaderWide::addStreams(
|
||||
auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr;
|
||||
|
||||
streams.emplace(stream_name, std::make_unique<MergeTreeReaderStream>(
|
||||
data_part_info_for_read->getDataPartStorage(), stream_name, DATA_FILE_EXTENSION,
|
||||
data_part_info_for_read, stream_name, DATA_FILE_EXTENSION,
|
||||
data_part_info_for_read->getMarksCount(), all_mark_ranges, settings, mark_cache,
|
||||
uncompressed_cache, data_part_info_for_read->getFileSizeOrZero(stream_name + DATA_FILE_EXTENSION),
|
||||
&data_part_info_for_read->getIndexGranularityInfo(),
|
||||
|
@ -40,7 +40,8 @@ struct Settings;
|
||||
M(Float, ratio_of_defaults_for_sparse_serialization, 1.0, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \
|
||||
\
|
||||
/** Merge settings. */ \
|
||||
M(UInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \
|
||||
M(UInt64, merge_max_block_size, 8192, "How many rows in blocks should be formed for merge operations. By default has the same value as `index_granularity`.", 0) \
|
||||
M(UInt64, merge_max_block_size_bytes, 10 * 1024 * 1024, "How many bytes in blocks should be formed for merge operations. By default has the same value as `index_granularity_bytes`.", 0) \
|
||||
M(UInt64, max_bytes_to_merge_at_max_space_in_pool, 150ULL * 1024 * 1024 * 1024, "Maximum in total size of parts to merge, when there are maximum free threads in background pool (or entries in replication queue).", 0) \
|
||||
M(UInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).", 0) \
|
||||
M(UInt64, max_replicated_merges_in_queue, 1000, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \
|
||||
@ -126,7 +127,8 @@ struct Settings;
|
||||
M(UInt64, min_relative_delay_to_close, 300, "Minimal delay from other replicas to close, stop serving requests and not return Ok during status check.", 0) \
|
||||
M(UInt64, min_absolute_delay_to_close, 0, "Minimal absolute delay to close, stop serving requests and not return Ok during status check.", 0) \
|
||||
M(UInt64, enable_vertical_merge_algorithm, 1, "Enable usage of Vertical merge algorithm.", 0) \
|
||||
M(UInt64, vertical_merge_algorithm_min_rows_to_activate, 16 * DEFAULT_MERGE_BLOCK_SIZE, "Minimal (approximate) sum of rows in merging parts to activate Vertical merge algorithm.", 0) \
|
||||
M(UInt64, vertical_merge_algorithm_min_rows_to_activate, 16 * 8192, "Minimal (approximate) sum of rows in merging parts to activate Vertical merge algorithm.", 0) \
|
||||
M(UInt64, vertical_merge_algorithm_min_bytes_to_activate, 0, "Minimal (approximate) uncompressed size in bytes in merging parts to activate Vertical merge algorithm.", 0) \
|
||||
M(UInt64, vertical_merge_algorithm_min_columns_to_activate, 11, "Minimal amount of non-PK columns to activate Vertical merge algorithm.", 0) \
|
||||
\
|
||||
/** Compatibility settings */ \
|
||||
|
@ -220,12 +220,13 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr<TableJoin> analyzed_join,
|
||||
Names left_key_names_resorted;
|
||||
for (const auto & key_name : key_names)
|
||||
{
|
||||
const auto & renamed_key = analyzed_join->renamedRightColumnName(key_name);
|
||||
const auto & renamed_key = analyzed_join->renamedRightColumnNameWithAlias(key_name);
|
||||
/// find position of renamed_key in key_names_right
|
||||
auto it = std::find(key_names_right.begin(), key_names_right.end(), renamed_key);
|
||||
if (it == key_names_right.end())
|
||||
throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN,
|
||||
"Key '{}' not found in JOIN ON section. All Join engine keys '{}' have to be used", key_name, fmt::join(key_names, ", "));
|
||||
"Key '{}' not found in JOIN ON section. Join engine key{} '{}' have to be used",
|
||||
key_name, key_names.size() > 1 ? "s" : "", fmt::join(key_names, ", "));
|
||||
const size_t key_position = std::distance(key_names_right.begin(), it);
|
||||
left_key_names_resorted.push_back(key_names_left[key_position]);
|
||||
}
|
||||
|
@ -1330,6 +1330,11 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
|
||||
uncovered_unexpected_parts.size(), uncovered_unexpected_parts_rows, unexpected_parts_nonnew, unexpected_parts_nonnew_rows,
|
||||
parts_to_fetch.size(), parts_to_fetch_blocks, covered_unexpected_parts.size(), unexpected_parts_rows - uncovered_unexpected_parts_rows);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!parts_to_fetch.empty())
|
||||
LOG_DEBUG(log, "Found parts to fetch (exist in zookeeper, but not locally): [{}]", fmt::join(parts_to_fetch, ", "));
|
||||
}
|
||||
|
||||
/// Add to the queue jobs to pick up the missing parts from other replicas and remove from ZK the information that we have them.
|
||||
queue.setBrokenPartsToEnqueueFetchesOnLoading(std::move(parts_to_fetch));
|
||||
@ -2470,8 +2475,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo
|
||||
{
|
||||
/// We check that it was not suddenly upgraded to new version.
|
||||
/// Otherwise it can be upgraded and instantly become lost, but we cannot notice that.
|
||||
ops.push_back(zkutil::makeCreateRequest(fs::path(source_path) / "is_lost", "0", zkutil::CreateMode::Persistent));
|
||||
ops.push_back(zkutil::makeRemoveRequest(fs::path(source_path) / "is_lost", -1));
|
||||
zkutil::addCheckNotExistsRequest(ops, *zookeeper, fs::path(source_path) / "is_lost");
|
||||
}
|
||||
else /// The replica we clone should not suddenly become lost.
|
||||
ops.push_back(zkutil::makeCheckRequest(fs::path(source_path) / "is_lost", source_is_lost_stat.version));
|
||||
@ -8874,8 +8878,7 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP
|
||||
/// We must be sure that this part doesn't exist on other replicas
|
||||
if (!zookeeper->exists(current_part_path))
|
||||
{
|
||||
ops.emplace_back(zkutil::makeCreateRequest(current_part_path, "", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(current_part_path, -1));
|
||||
zkutil::addCheckNotExistsRequest(ops, *zookeeper, current_part_path);
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -22,6 +22,7 @@ NamesAndTypesList StorageSystemMerges::getNamesAndTypes()
|
||||
{"partition_id", std::make_shared<DataTypeString>()},
|
||||
{"is_mutation", std::make_shared<DataTypeUInt8>()},
|
||||
{"total_size_bytes_compressed", std::make_shared<DataTypeUInt64>()},
|
||||
{"total_size_bytes_uncompressed", std::make_shared<DataTypeUInt64>()},
|
||||
{"total_size_marks", std::make_shared<DataTypeUInt64>()},
|
||||
{"bytes_read_uncompressed", std::make_shared<DataTypeUInt64>()},
|
||||
{"rows_read", std::make_shared<DataTypeUInt64>()},
|
||||
@ -59,6 +60,7 @@ void StorageSystemMerges::fillData(MutableColumns & res_columns, ContextPtr cont
|
||||
res_columns[i++]->insert(merge.partition_id);
|
||||
res_columns[i++]->insert(merge.is_mutation);
|
||||
res_columns[i++]->insert(merge.total_size_bytes_compressed);
|
||||
res_columns[i++]->insert(merge.total_size_bytes_uncompressed);
|
||||
res_columns[i++]->insert(merge.total_size_marks);
|
||||
res_columns[i++]->insert(merge.bytes_read_uncompressed);
|
||||
res_columns[i++]->insert(merge.rows_read);
|
||||
|
@ -470,7 +470,7 @@ CHECK_DESCRIPTIONS = [
|
||||
),
|
||||
CheckDescription(
|
||||
"Flaky tests",
|
||||
"Runs a flaky tests from master multiple times to identify if they are stable.",
|
||||
"Checks if new added or modified tests are flaky by running them repeatedly, in parallel, with more randomization. Functional tests are run 100 times with address sanitizer, and additional randomization of thread scheduling. Integrational tests are run up to 10 times. If at least once a new test has failed, or was too long, this check will be red. We don't allow flaky tests, read https://clickhouse.com/blog/decorating-a-christmas-tree-with-the-help-of-flaky-tests/",
|
||||
lambda x: "tests flaky check" in x,
|
||||
),
|
||||
CheckDescription(
|
||||
|
@ -224,14 +224,22 @@ def test_attach_detach_partition(cluster):
|
||||
wait_for_delete_empty_parts(node, "hdfs_test")
|
||||
wait_for_delete_inactive_parts(node, "hdfs_test")
|
||||
wait_for_delete_hdfs_objects(
|
||||
cluster, FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
cluster,
|
||||
FILES_OVERHEAD
|
||||
+ FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
- FILES_OVERHEAD_METADATA_VERSION,
|
||||
)
|
||||
|
||||
node.query("ALTER TABLE hdfs_test ATTACH PARTITION '2020-01-03'")
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(8192)"
|
||||
|
||||
hdfs_objects = fs.listdir("/clickhouse")
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
assert (
|
||||
len(hdfs_objects)
|
||||
== FILES_OVERHEAD
|
||||
+ FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
- FILES_OVERHEAD_METADATA_VERSION
|
||||
)
|
||||
|
||||
node.query("ALTER TABLE hdfs_test DROP PARTITION '2020-01-03'")
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(4096)"
|
||||
@ -355,7 +363,14 @@ def test_move_replace_partition_to_another_table(cluster):
|
||||
|
||||
# Number of objects in HDFS should be unchanged.
|
||||
hdfs_objects = fs.listdir("/clickhouse")
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD * 2 + FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
for obj in hdfs_objects:
|
||||
print("Object in HDFS after move", obj)
|
||||
wait_for_delete_hdfs_objects(
|
||||
cluster,
|
||||
FILES_OVERHEAD * 2
|
||||
+ FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
- FILES_OVERHEAD_METADATA_VERSION * 2,
|
||||
)
|
||||
|
||||
# Add new partitions to source table, but with different values and replace them from copied table.
|
||||
node.query(
|
||||
@ -370,7 +385,15 @@ def test_move_replace_partition_to_another_table(cluster):
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(16384)"
|
||||
|
||||
hdfs_objects = fs.listdir("/clickhouse")
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD * 2 + FILES_OVERHEAD_PER_PART_WIDE * 6
|
||||
for obj in hdfs_objects:
|
||||
print("Object in HDFS after insert", obj)
|
||||
|
||||
wait_for_delete_hdfs_objects(
|
||||
cluster,
|
||||
FILES_OVERHEAD * 2
|
||||
+ FILES_OVERHEAD_PER_PART_WIDE * 6
|
||||
- FILES_OVERHEAD_METADATA_VERSION * 2,
|
||||
)
|
||||
|
||||
node.query("ALTER TABLE hdfs_test REPLACE PARTITION '2020-01-03' FROM hdfs_clone")
|
||||
node.query("ALTER TABLE hdfs_test REPLACE PARTITION '2020-01-05' FROM hdfs_clone")
|
||||
@ -381,7 +404,10 @@ def test_move_replace_partition_to_another_table(cluster):
|
||||
|
||||
# Wait for outdated partitions deletion.
|
||||
wait_for_delete_hdfs_objects(
|
||||
cluster, FILES_OVERHEAD * 2 + FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
cluster,
|
||||
FILES_OVERHEAD * 2
|
||||
+ FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
- FILES_OVERHEAD_METADATA_VERSION * 2,
|
||||
)
|
||||
|
||||
node.query("DROP TABLE hdfs_clone NO DELAY")
|
||||
@ -390,4 +416,13 @@ def test_move_replace_partition_to_another_table(cluster):
|
||||
|
||||
# Data should remain in hdfs
|
||||
hdfs_objects = fs.listdir("/clickhouse")
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
|
||||
for obj in hdfs_objects:
|
||||
print("Object in HDFS after drop", obj)
|
||||
|
||||
wait_for_delete_hdfs_objects(
|
||||
cluster,
|
||||
FILES_OVERHEAD
|
||||
+ FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
- FILES_OVERHEAD_METADATA_VERSION * 2,
|
||||
)
|
||||
|
@ -101,44 +101,45 @@ def run_s3_mocks(cluster):
|
||||
)
|
||||
|
||||
|
||||
def list_objects(cluster, path="data/"):
|
||||
def list_objects(cluster, path="data/", hint="list_objects"):
|
||||
minio = cluster.minio_client
|
||||
objects = list(minio.list_objects(cluster.minio_bucket, path, recursive=True))
|
||||
logging.info(f"list_objects ({len(objects)}): {[x.object_name for x in objects]}")
|
||||
logging.info(f"{hint} ({len(objects)}): {[x.object_name for x in objects]}")
|
||||
return objects
|
||||
|
||||
|
||||
def wait_for_delete_s3_objects(cluster, expected, timeout=30):
|
||||
minio = cluster.minio_client
|
||||
while timeout > 0:
|
||||
if (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
== expected
|
||||
):
|
||||
if len(list_objects(cluster, "data/")) == expected:
|
||||
return
|
||||
timeout -= 1
|
||||
time.sleep(1)
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
== expected
|
||||
)
|
||||
assert len(list_objects(cluster, "data/")) == expected
|
||||
|
||||
|
||||
@pytest.fixture(autouse=True)
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
def drop_table(cluster, node_name):
|
||||
yield
|
||||
node = cluster.instances[node_name]
|
||||
def remove_all_s3_objects(cluster):
|
||||
minio = cluster.minio_client
|
||||
for obj in list_objects(cluster, "data/"):
|
||||
minio.remove_object(cluster.minio_bucket, obj.object_name)
|
||||
|
||||
node.query("DROP TABLE IF EXISTS s3_test NO DELAY")
|
||||
|
||||
@pytest.fixture(autouse=True, scope="function")
|
||||
def clear_minio(cluster):
|
||||
try:
|
||||
wait_for_delete_s3_objects(cluster, 0)
|
||||
finally:
|
||||
# CH do some writes to the S3 at start. For example, file data/clickhouse_access_check_{server_uuid}.
|
||||
# Set the timeout there as 10 sec in order to resolve the race with that file exists.
|
||||
wait_for_delete_s3_objects(cluster, 0, timeout=10)
|
||||
except:
|
||||
# Remove extra objects to prevent tests cascade failing
|
||||
for obj in list_objects(cluster, "data/"):
|
||||
minio.remove_object(cluster.minio_bucket, obj.object_name)
|
||||
remove_all_s3_objects(cluster)
|
||||
|
||||
yield
|
||||
|
||||
|
||||
def check_no_objects_after_drop(cluster, table_name="s3_test", node_name="node"):
|
||||
node = cluster.instances[node_name]
|
||||
node.query(f"DROP TABLE IF EXISTS {table_name} NO DELAY")
|
||||
wait_for_delete_s3_objects(cluster, 0, timeout=0)
|
||||
|
||||
|
||||
@pytest.mark.parametrize(
|
||||
@ -158,10 +159,7 @@ def test_simple_insert_select(
|
||||
values1 = generate_values("2020-01-03", 4096)
|
||||
node.query("INSERT INTO s3_test VALUES {}".format(values1))
|
||||
assert node.query("SELECT * FROM s3_test order by dt, id FORMAT Values") == values1
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
== FILES_OVERHEAD + files_per_part
|
||||
)
|
||||
assert len(list_objects(cluster, "data/")) == FILES_OVERHEAD + files_per_part
|
||||
|
||||
values2 = generate_values("2020-01-04", 4096)
|
||||
node.query("INSERT INTO s3_test VALUES {}".format(values2))
|
||||
@ -169,15 +167,14 @@ def test_simple_insert_select(
|
||||
node.query("SELECT * FROM s3_test ORDER BY dt, id FORMAT Values")
|
||||
== values1 + "," + values2
|
||||
)
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
== FILES_OVERHEAD + files_per_part * 2
|
||||
)
|
||||
assert len(list_objects(cluster, "data/")) == FILES_OVERHEAD + files_per_part * 2
|
||||
|
||||
assert (
|
||||
node.query("SELECT count(*) FROM s3_test where id = 1 FORMAT Values") == "(2)"
|
||||
)
|
||||
|
||||
check_no_objects_after_drop(cluster)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("merge_vertical,node_name", [(True, "node"), (False, "node")])
|
||||
def test_insert_same_partition_and_merge(cluster, merge_vertical, node_name):
|
||||
@ -188,7 +185,6 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical, node_name):
|
||||
|
||||
node = cluster.instances[node_name]
|
||||
create_table(node, "s3_test", **settings)
|
||||
minio = cluster.minio_client
|
||||
|
||||
node.query("SYSTEM STOP MERGES s3_test")
|
||||
node.query(
|
||||
@ -214,7 +210,7 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical, node_name):
|
||||
node.query("SELECT count(distinct(id)) FROM s3_test FORMAT Values") == "(8192)"
|
||||
)
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
len(list_objects(cluster, "data/"))
|
||||
== FILES_OVERHEAD_PER_PART_WIDE * 6 + FILES_OVERHEAD
|
||||
)
|
||||
|
||||
@ -242,6 +238,8 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical, node_name):
|
||||
cluster, FILES_OVERHEAD_PER_PART_WIDE + FILES_OVERHEAD, timeout=45
|
||||
)
|
||||
|
||||
check_no_objects_after_drop(cluster)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
def test_alter_table_columns(cluster, node_name):
|
||||
@ -287,12 +285,13 @@ def test_alter_table_columns(cluster, node_name):
|
||||
cluster, FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE + 2
|
||||
)
|
||||
|
||||
check_no_objects_after_drop(cluster)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
def test_attach_detach_partition(cluster, node_name):
|
||||
node = cluster.instances[node_name]
|
||||
create_table(node, "s3_test")
|
||||
minio = cluster.minio_client
|
||||
|
||||
node.query(
|
||||
"INSERT INTO s3_test VALUES {}".format(generate_values("2020-01-03", 4096))
|
||||
@ -312,14 +311,18 @@ def test_attach_detach_partition(cluster, node_name):
|
||||
assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(4096)"
|
||||
assert (
|
||||
len(list_objects(cluster, "data/"))
|
||||
== FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
== FILES_OVERHEAD
|
||||
+ FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
- FILES_OVERHEAD_METADATA_VERSION
|
||||
)
|
||||
|
||||
node.query("ALTER TABLE s3_test ATTACH PARTITION '2020-01-03'")
|
||||
assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(8192)"
|
||||
assert (
|
||||
len(list_objects(cluster, "data/"))
|
||||
== FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
== FILES_OVERHEAD
|
||||
+ FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
- FILES_OVERHEAD_METADATA_VERSION
|
||||
)
|
||||
|
||||
node.query("ALTER TABLE s3_test DROP PARTITION '2020-01-03'")
|
||||
@ -337,7 +340,9 @@ def test_attach_detach_partition(cluster, node_name):
|
||||
assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(0)"
|
||||
assert (
|
||||
len(list_objects(cluster, "data/"))
|
||||
== FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 1
|
||||
== FILES_OVERHEAD
|
||||
+ FILES_OVERHEAD_PER_PART_WIDE * 1
|
||||
- FILES_OVERHEAD_METADATA_VERSION
|
||||
)
|
||||
node.query(
|
||||
"ALTER TABLE s3_test DROP DETACHED PARTITION '2020-01-04'",
|
||||
@ -349,12 +354,13 @@ def test_attach_detach_partition(cluster, node_name):
|
||||
== FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 0
|
||||
)
|
||||
|
||||
check_no_objects_after_drop(cluster)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
def test_move_partition_to_another_disk(cluster, node_name):
|
||||
node = cluster.instances[node_name]
|
||||
create_table(node, "s3_test")
|
||||
minio = cluster.minio_client
|
||||
|
||||
node.query(
|
||||
"INSERT INTO s3_test VALUES {}".format(generate_values("2020-01-03", 4096))
|
||||
@ -364,30 +370,31 @@ def test_move_partition_to_another_disk(cluster, node_name):
|
||||
)
|
||||
assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(8192)"
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
len(list_objects(cluster, "data/"))
|
||||
== FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
)
|
||||
|
||||
node.query("ALTER TABLE s3_test MOVE PARTITION '2020-01-04' TO DISK 'hdd'")
|
||||
assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(8192)"
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
len(list_objects(cluster, "data/"))
|
||||
== FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE
|
||||
)
|
||||
|
||||
node.query("ALTER TABLE s3_test MOVE PARTITION '2020-01-04' TO DISK 's3'")
|
||||
assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(8192)"
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
len(list_objects(cluster, "data/"))
|
||||
== FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
)
|
||||
|
||||
check_no_objects_after_drop(cluster)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
def test_table_manipulations(cluster, node_name):
|
||||
node = cluster.instances[node_name]
|
||||
create_table(node, "s3_test")
|
||||
minio = cluster.minio_client
|
||||
|
||||
node.query(
|
||||
"INSERT INTO s3_test VALUES {}".format(generate_values("2020-01-03", 4096))
|
||||
@ -399,9 +406,10 @@ def test_table_manipulations(cluster, node_name):
|
||||
node.query("RENAME TABLE s3_test TO s3_renamed")
|
||||
assert node.query("SELECT count(*) FROM s3_renamed FORMAT Values") == "(8192)"
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
len(list_objects(cluster, "data/"))
|
||||
== FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
)
|
||||
|
||||
node.query("RENAME TABLE s3_renamed TO s3_test")
|
||||
|
||||
assert node.query("CHECK TABLE s3_test FORMAT Values") == "(1)"
|
||||
@ -410,7 +418,7 @@ def test_table_manipulations(cluster, node_name):
|
||||
node.query("ATTACH TABLE s3_test")
|
||||
assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(8192)"
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
len(list_objects(cluster, "data/"))
|
||||
== FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
)
|
||||
|
||||
@ -418,17 +426,15 @@ def test_table_manipulations(cluster, node_name):
|
||||
wait_for_delete_empty_parts(node, "s3_test")
|
||||
wait_for_delete_inactive_parts(node, "s3_test")
|
||||
assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(0)"
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
== FILES_OVERHEAD
|
||||
)
|
||||
assert len(list_objects(cluster, "data/")) == FILES_OVERHEAD
|
||||
|
||||
check_no_objects_after_drop(cluster)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
def test_move_replace_partition_to_another_table(cluster, node_name):
|
||||
node = cluster.instances[node_name]
|
||||
create_table(node, "s3_test")
|
||||
minio = cluster.minio_client
|
||||
|
||||
node.query(
|
||||
"INSERT INTO s3_test VALUES {}".format(generate_values("2020-01-03", 4096))
|
||||
@ -444,11 +450,11 @@ def test_move_replace_partition_to_another_table(cluster, node_name):
|
||||
)
|
||||
assert node.query("SELECT sum(id) FROM s3_test FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(16384)"
|
||||
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
len(list_objects(cluster, "data/", "Objects at start"))
|
||||
== FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
)
|
||||
|
||||
create_table(node, "s3_clone")
|
||||
|
||||
node.query("ALTER TABLE s3_test MOVE PARTITION '2020-01-03' TO TABLE s3_clone")
|
||||
@ -457,10 +463,14 @@ def test_move_replace_partition_to_another_table(cluster, node_name):
|
||||
assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(8192)"
|
||||
assert node.query("SELECT sum(id) FROM s3_clone FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM s3_clone FORMAT Values") == "(8192)"
|
||||
|
||||
list_objects(cluster, "data/", "Object after move partition")
|
||||
# Number of objects in S3 should be unchanged.
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
== FILES_OVERHEAD * 2 + FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
wait_for_delete_s3_objects(
|
||||
cluster,
|
||||
FILES_OVERHEAD * 2
|
||||
+ FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
- FILES_OVERHEAD_METADATA_VERSION * 2,
|
||||
)
|
||||
|
||||
# Add new partitions to source table, but with different values and replace them from copied table.
|
||||
@ -472,9 +482,13 @@ def test_move_replace_partition_to_another_table(cluster, node_name):
|
||||
)
|
||||
assert node.query("SELECT sum(id) FROM s3_test FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(16384)"
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
== FILES_OVERHEAD * 2 + FILES_OVERHEAD_PER_PART_WIDE * 6
|
||||
|
||||
list_objects(cluster, "data/", "Object after insert")
|
||||
wait_for_delete_s3_objects(
|
||||
cluster,
|
||||
FILES_OVERHEAD * 2
|
||||
+ FILES_OVERHEAD_PER_PART_WIDE * 6
|
||||
- FILES_OVERHEAD_METADATA_VERSION * 2,
|
||||
)
|
||||
|
||||
node.query("ALTER TABLE s3_test REPLACE PARTITION '2020-01-03' FROM s3_clone")
|
||||
@ -486,39 +500,48 @@ def test_move_replace_partition_to_another_table(cluster, node_name):
|
||||
|
||||
# Wait for outdated partitions deletion.
|
||||
wait_for_delete_s3_objects(
|
||||
cluster, FILES_OVERHEAD * 2 + FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
cluster,
|
||||
FILES_OVERHEAD * 2
|
||||
+ FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
- FILES_OVERHEAD_METADATA_VERSION * 2,
|
||||
)
|
||||
|
||||
node.query("DROP TABLE s3_clone NO DELAY")
|
||||
assert node.query("SELECT sum(id) FROM s3_test FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(16384)"
|
||||
# Data should remain in S3
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
== FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
|
||||
list_objects(cluster, "data/", "Object after drop")
|
||||
wait_for_delete_s3_objects(
|
||||
cluster,
|
||||
FILES_OVERHEAD
|
||||
+ FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
- FILES_OVERHEAD_METADATA_VERSION * 2,
|
||||
)
|
||||
|
||||
node.query("ALTER TABLE s3_test FREEZE")
|
||||
# Number S3 objects should be unchanged.
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
== FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
list_objects(cluster, "data/", "Object after freeze")
|
||||
wait_for_delete_s3_objects(
|
||||
cluster,
|
||||
FILES_OVERHEAD
|
||||
+ FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
- FILES_OVERHEAD_METADATA_VERSION * 2,
|
||||
)
|
||||
|
||||
node.query("DROP TABLE s3_test NO DELAY")
|
||||
# Backup data should remain in S3.
|
||||
|
||||
wait_for_delete_s3_objects(cluster, FILES_OVERHEAD_PER_PART_WIDE * 4)
|
||||
wait_for_delete_s3_objects(
|
||||
cluster, FILES_OVERHEAD_PER_PART_WIDE * 4 - FILES_OVERHEAD_METADATA_VERSION * 4
|
||||
)
|
||||
|
||||
for obj in list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)):
|
||||
minio.remove_object(cluster.minio_bucket, obj.object_name)
|
||||
remove_all_s3_objects(cluster)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
def test_freeze_unfreeze(cluster, node_name):
|
||||
node = cluster.instances[node_name]
|
||||
create_table(node, "s3_test")
|
||||
minio = cluster.minio_client
|
||||
|
||||
node.query(
|
||||
"INSERT INTO s3_test VALUES {}".format(generate_values("2020-01-03", 4096))
|
||||
@ -533,8 +556,9 @@ def test_freeze_unfreeze(cluster, node_name):
|
||||
wait_for_delete_empty_parts(node, "s3_test")
|
||||
wait_for_delete_inactive_parts(node, "s3_test")
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
== FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
len(list_objects(cluster, "data/"))
|
||||
== FILES_OVERHEAD
|
||||
+ (FILES_OVERHEAD_PER_PART_WIDE - FILES_OVERHEAD_METADATA_VERSION) * 2
|
||||
)
|
||||
|
||||
# Unfreeze single partition from backup1.
|
||||
@ -544,13 +568,10 @@ def test_freeze_unfreeze(cluster, node_name):
|
||||
# Unfreeze all partitions from backup2.
|
||||
node.query("ALTER TABLE s3_test UNFREEZE WITH NAME 'backup2'")
|
||||
|
||||
# Data should be removed from S3.
|
||||
wait_for_delete_s3_objects(cluster, FILES_OVERHEAD)
|
||||
|
||||
# Data should be removed from S3.
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
== FILES_OVERHEAD
|
||||
)
|
||||
check_no_objects_after_drop(cluster)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
@ -558,7 +579,6 @@ def test_freeze_system_unfreeze(cluster, node_name):
|
||||
node = cluster.instances[node_name]
|
||||
create_table(node, "s3_test")
|
||||
create_table(node, "s3_test_removed")
|
||||
minio = cluster.minio_client
|
||||
|
||||
node.query(
|
||||
"INSERT INTO s3_test VALUES {}".format(generate_values("2020-01-04", 4096))
|
||||
@ -574,20 +594,18 @@ def test_freeze_system_unfreeze(cluster, node_name):
|
||||
wait_for_delete_inactive_parts(node, "s3_test")
|
||||
node.query("DROP TABLE s3_test_removed NO DELAY")
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
== FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
len(list_objects(cluster, "data/"))
|
||||
== FILES_OVERHEAD
|
||||
+ (FILES_OVERHEAD_PER_PART_WIDE - FILES_OVERHEAD_METADATA_VERSION) * 2
|
||||
)
|
||||
|
||||
# Unfreeze all data from backup3.
|
||||
node.query("SYSTEM UNFREEZE WITH NAME 'backup3'")
|
||||
|
||||
# Data should be removed from S3.
|
||||
wait_for_delete_s3_objects(cluster, FILES_OVERHEAD)
|
||||
|
||||
# Data should be removed from S3.
|
||||
assert (
|
||||
len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)))
|
||||
== FILES_OVERHEAD
|
||||
)
|
||||
check_no_objects_after_drop(cluster)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
@ -633,6 +651,8 @@ def test_s3_disk_apply_new_settings(cluster, node_name):
|
||||
# There should be 3 times more S3 requests because multi-part upload mode uses 3 requests to upload object.
|
||||
assert get_s3_requests() - s3_requests_before == s3_requests_to_write_partition * 3
|
||||
|
||||
check_no_objects_after_drop(cluster)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
def test_s3_no_delete_objects(cluster, node_name):
|
||||
@ -641,6 +661,7 @@ def test_s3_no_delete_objects(cluster, node_name):
|
||||
node, "s3_test_no_delete_objects", storage_policy="no_delete_objects_s3"
|
||||
)
|
||||
node.query("DROP TABLE s3_test_no_delete_objects SYNC")
|
||||
remove_all_s3_objects(cluster)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
@ -655,6 +676,7 @@ def test_s3_disk_reads_on_unstable_connection(cluster, node_name):
|
||||
assert node.query("SELECT sum(id) FROM s3_test").splitlines() == [
|
||||
"40499995500000"
|
||||
]
|
||||
check_no_objects_after_drop(cluster)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
@ -664,14 +686,13 @@ def test_lazy_seek_optimization_for_async_read(cluster, node_name):
|
||||
node.query(
|
||||
"CREATE TABLE s3_test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3';"
|
||||
)
|
||||
node.query("SYSTEM STOP MERGES s3_test")
|
||||
node.query(
|
||||
"INSERT INTO s3_test SELECT * FROM generateRandom('key UInt32, value String') LIMIT 10000000"
|
||||
)
|
||||
node.query("SELECT * FROM s3_test WHERE value LIKE '%abc%' ORDER BY value LIMIT 10")
|
||||
node.query("DROP TABLE IF EXISTS s3_test NO DELAY")
|
||||
minio = cluster.minio_client
|
||||
for obj in list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)):
|
||||
minio.remove_object(cluster.minio_bucket, obj.object_name)
|
||||
|
||||
check_no_objects_after_drop(cluster)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node_with_limited_disk"])
|
||||
@ -681,6 +702,7 @@ def test_cache_with_full_disk_space(cluster, node_name):
|
||||
node.query(
|
||||
"CREATE TABLE s3_test (key UInt32, value String) Engine=MergeTree() ORDER BY value SETTINGS storage_policy='s3_with_cache_and_jbod';"
|
||||
)
|
||||
node.query("SYSTEM STOP MERGES s3_test")
|
||||
node.query(
|
||||
"INSERT INTO s3_test SELECT number, toString(number) FROM numbers(100000000)"
|
||||
)
|
||||
@ -699,7 +721,7 @@ def test_cache_with_full_disk_space(cluster, node_name):
|
||||
assert node.contains_in_log(
|
||||
"Insert into cache is skipped due to insufficient disk space"
|
||||
)
|
||||
node.query("DROP TABLE IF EXISTS s3_test NO DELAY")
|
||||
check_no_objects_after_drop(cluster, node_name=node_name)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
@ -724,6 +746,7 @@ def test_store_cleanup_disk_s3(cluster, node_name):
|
||||
"CREATE TABLE s3_test UUID '00000000-1000-4000-8000-000000000001' (n UInt64) Engine=MergeTree() ORDER BY n SETTINGS storage_policy='s3';"
|
||||
)
|
||||
node.query("INSERT INTO s3_test SELECT 1")
|
||||
check_no_objects_after_drop(cluster)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
@ -800,3 +823,5 @@ def test_cache_setting_compatibility(cluster, node_name):
|
||||
node.query("SELECT * FROM s3_test FORMAT Null")
|
||||
|
||||
assert not node.contains_in_log("No such file or directory: Cache info:")
|
||||
|
||||
check_no_objects_after_drop(cluster)
|
||||
|
@ -70,7 +70,7 @@ def partition_complex_assert_columns_txt():
|
||||
)
|
||||
|
||||
|
||||
def partition_complex_assert_checksums():
|
||||
def partition_complex_assert_checksums(after_detach=False):
|
||||
# Do not check increment.txt - it can be changed by other tests with FREEZE
|
||||
cmd = [
|
||||
"bash",
|
||||
@ -80,36 +80,67 @@ def partition_complex_assert_checksums():
|
||||
" | sed 's shadow/[0-9]*/data/[a-z0-9_-]*/ shadow/1/data/test/ g' | sort | uniq",
|
||||
]
|
||||
|
||||
checksums = (
|
||||
"082814b5aa5109160d5c0c5aff10d4df\tshadow/1/data/test/partition_complex/19700102_2_2_0/k.bin\n"
|
||||
"082814b5aa5109160d5c0c5aff10d4df\tshadow/1/data/test/partition_complex/19700201_1_1_0/v1.bin\n"
|
||||
"13cae8e658e0ca4f75c56b1fc424e150\tshadow/1/data/test/partition_complex/19700102_2_2_0/minmax_p.idx\n"
|
||||
"25daad3d9e60b45043a70c4ab7d3b1c6\tshadow/1/data/test/partition_complex/19700102_2_2_0/partition.dat\n"
|
||||
"3726312af62aec86b64a7708d5751787\tshadow/1/data/test/partition_complex/19700201_1_1_0/partition.dat\n"
|
||||
"37855b06a39b79a67ea4e86e4a3299aa\tshadow/1/data/test/partition_complex/19700102_2_2_0/checksums.txt\n"
|
||||
"38e62ff37e1e5064e9a3f605dfe09d13\tshadow/1/data/test/partition_complex/19700102_2_2_0/v1.bin\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700102_2_2_0/k.mrk\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700102_2_2_0/p.mrk\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700102_2_2_0/v1.mrk\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700201_1_1_0/k.mrk\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700201_1_1_0/p.mrk\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700201_1_1_0/v1.mrk\n"
|
||||
"55a54008ad1ba589aa210d2629c1df41\tshadow/1/data/test/partition_complex/19700201_1_1_0/primary.idx\n"
|
||||
"5f087cb3e7071bf9407e095821e2af8f\tshadow/1/data/test/partition_complex/19700201_1_1_0/checksums.txt\n"
|
||||
"77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition_complex/19700102_2_2_0/columns.txt\n"
|
||||
"77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition_complex/19700201_1_1_0/columns.txt\n"
|
||||
"88cdc31ded355e7572d68d8cde525d3a\tshadow/1/data/test/partition_complex/19700201_1_1_0/p.bin\n"
|
||||
"9e688c58a5487b8eaf69c9e1005ad0bf\tshadow/1/data/test/partition_complex/19700102_2_2_0/primary.idx\n"
|
||||
"c0904274faa8f3f06f35666cc9c5bd2f\tshadow/1/data/test/partition_complex/19700102_2_2_0/default_compression_codec.txt\n"
|
||||
"c0904274faa8f3f06f35666cc9c5bd2f\tshadow/1/data/test/partition_complex/19700201_1_1_0/default_compression_codec.txt\n"
|
||||
"c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition_complex/19700102_2_2_0/count.txt\n"
|
||||
"c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition_complex/19700201_1_1_0/count.txt\n"
|
||||
"cfcb770c3ecd0990dcceb1bde129e6c6\tshadow/1/data/test/partition_complex/19700102_2_2_0/p.bin\n"
|
||||
"cfcd208495d565ef66e7dff9f98764da\tshadow/1/data/test/partition_complex/19700102_2_2_0/metadata_version.txt\n"
|
||||
"cfcd208495d565ef66e7dff9f98764da\tshadow/1/data/test/partition_complex/19700201_1_1_0/metadata_version.txt\n"
|
||||
"e2af3bef1fd129aea73a890ede1e7a30\tshadow/1/data/test/partition_complex/19700201_1_1_0/k.bin\n"
|
||||
"f2312862cc01adf34a93151377be2ddf\tshadow/1/data/test/partition_complex/19700201_1_1_0/minmax_p.idx\n"
|
||||
)
|
||||
# no metadata version
|
||||
if after_detach:
|
||||
checksums = (
|
||||
"082814b5aa5109160d5c0c5aff10d4df\tshadow/1/data/test/partition_complex/19700102_2_2_0/k.bin\n"
|
||||
"082814b5aa5109160d5c0c5aff10d4df\tshadow/1/data/test/partition_complex/19700201_1_1_0/v1.bin\n"
|
||||
"13cae8e658e0ca4f75c56b1fc424e150\tshadow/1/data/test/partition_complex/19700102_2_2_0/minmax_p.idx\n"
|
||||
"25daad3d9e60b45043a70c4ab7d3b1c6\tshadow/1/data/test/partition_complex/19700102_2_2_0/partition.dat\n"
|
||||
"3726312af62aec86b64a7708d5751787\tshadow/1/data/test/partition_complex/19700201_1_1_0/partition.dat\n"
|
||||
"37855b06a39b79a67ea4e86e4a3299aa\tshadow/1/data/test/partition_complex/19700102_2_2_0/checksums.txt\n"
|
||||
"38e62ff37e1e5064e9a3f605dfe09d13\tshadow/1/data/test/partition_complex/19700102_2_2_0/v1.bin\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700102_2_2_0/k.mrk\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700102_2_2_0/p.mrk\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700102_2_2_0/v1.mrk\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700201_1_1_0/k.mrk\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700201_1_1_0/p.mrk\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700201_1_1_0/v1.mrk\n"
|
||||
"55a54008ad1ba589aa210d2629c1df41\tshadow/1/data/test/partition_complex/19700201_1_1_0/primary.idx\n"
|
||||
"5f087cb3e7071bf9407e095821e2af8f\tshadow/1/data/test/partition_complex/19700201_1_1_0/checksums.txt\n"
|
||||
"77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition_complex/19700102_2_2_0/columns.txt\n"
|
||||
"77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition_complex/19700201_1_1_0/columns.txt\n"
|
||||
"88cdc31ded355e7572d68d8cde525d3a\tshadow/1/data/test/partition_complex/19700201_1_1_0/p.bin\n"
|
||||
"9e688c58a5487b8eaf69c9e1005ad0bf\tshadow/1/data/test/partition_complex/19700102_2_2_0/primary.idx\n"
|
||||
"c0904274faa8f3f06f35666cc9c5bd2f\tshadow/1/data/test/partition_complex/19700102_2_2_0/default_compression_codec.txt\n"
|
||||
"c0904274faa8f3f06f35666cc9c5bd2f\tshadow/1/data/test/partition_complex/19700201_1_1_0/default_compression_codec.txt\n"
|
||||
"c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition_complex/19700102_2_2_0/count.txt\n"
|
||||
"c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition_complex/19700201_1_1_0/count.txt\n"
|
||||
"cfcb770c3ecd0990dcceb1bde129e6c6\tshadow/1/data/test/partition_complex/19700102_2_2_0/p.bin\n"
|
||||
"e2af3bef1fd129aea73a890ede1e7a30\tshadow/1/data/test/partition_complex/19700201_1_1_0/k.bin\n"
|
||||
"f2312862cc01adf34a93151377be2ddf\tshadow/1/data/test/partition_complex/19700201_1_1_0/minmax_p.idx\n"
|
||||
)
|
||||
else:
|
||||
checksums = (
|
||||
"082814b5aa5109160d5c0c5aff10d4df\tshadow/1/data/test/partition_complex/19700102_2_2_0/k.bin\n"
|
||||
"082814b5aa5109160d5c0c5aff10d4df\tshadow/1/data/test/partition_complex/19700201_1_1_0/v1.bin\n"
|
||||
"13cae8e658e0ca4f75c56b1fc424e150\tshadow/1/data/test/partition_complex/19700102_2_2_0/minmax_p.idx\n"
|
||||
"25daad3d9e60b45043a70c4ab7d3b1c6\tshadow/1/data/test/partition_complex/19700102_2_2_0/partition.dat\n"
|
||||
"3726312af62aec86b64a7708d5751787\tshadow/1/data/test/partition_complex/19700201_1_1_0/partition.dat\n"
|
||||
"37855b06a39b79a67ea4e86e4a3299aa\tshadow/1/data/test/partition_complex/19700102_2_2_0/checksums.txt\n"
|
||||
"38e62ff37e1e5064e9a3f605dfe09d13\tshadow/1/data/test/partition_complex/19700102_2_2_0/v1.bin\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700102_2_2_0/k.mrk\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700102_2_2_0/p.mrk\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700102_2_2_0/v1.mrk\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700201_1_1_0/k.mrk\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700201_1_1_0/p.mrk\n"
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700201_1_1_0/v1.mrk\n"
|
||||
"55a54008ad1ba589aa210d2629c1df41\tshadow/1/data/test/partition_complex/19700201_1_1_0/primary.idx\n"
|
||||
"5f087cb3e7071bf9407e095821e2af8f\tshadow/1/data/test/partition_complex/19700201_1_1_0/checksums.txt\n"
|
||||
"77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition_complex/19700102_2_2_0/columns.txt\n"
|
||||
"77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition_complex/19700201_1_1_0/columns.txt\n"
|
||||
"88cdc31ded355e7572d68d8cde525d3a\tshadow/1/data/test/partition_complex/19700201_1_1_0/p.bin\n"
|
||||
"9e688c58a5487b8eaf69c9e1005ad0bf\tshadow/1/data/test/partition_complex/19700102_2_2_0/primary.idx\n"
|
||||
"c0904274faa8f3f06f35666cc9c5bd2f\tshadow/1/data/test/partition_complex/19700102_2_2_0/default_compression_codec.txt\n"
|
||||
"c0904274faa8f3f06f35666cc9c5bd2f\tshadow/1/data/test/partition_complex/19700201_1_1_0/default_compression_codec.txt\n"
|
||||
"c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition_complex/19700102_2_2_0/count.txt\n"
|
||||
"c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition_complex/19700201_1_1_0/count.txt\n"
|
||||
"cfcb770c3ecd0990dcceb1bde129e6c6\tshadow/1/data/test/partition_complex/19700102_2_2_0/p.bin\n"
|
||||
"cfcd208495d565ef66e7dff9f98764da\tshadow/1/data/test/partition_complex/19700102_2_2_0/metadata_version.txt\n"
|
||||
"cfcd208495d565ef66e7dff9f98764da\tshadow/1/data/test/partition_complex/19700201_1_1_0/metadata_version.txt\n"
|
||||
"e2af3bef1fd129aea73a890ede1e7a30\tshadow/1/data/test/partition_complex/19700201_1_1_0/k.bin\n"
|
||||
"f2312862cc01adf34a93151377be2ddf\tshadow/1/data/test/partition_complex/19700201_1_1_0/minmax_p.idx\n"
|
||||
)
|
||||
|
||||
assert TSV(instance.exec_in_container(cmd).replace(" ", "\t")) == TSV(checksums)
|
||||
|
||||
@ -134,7 +165,7 @@ def test_partition_complex(partition_table_complex):
|
||||
|
||||
q("ALTER TABLE test.partition_complex FREEZE")
|
||||
|
||||
partition_complex_assert_checksums()
|
||||
partition_complex_assert_checksums(True)
|
||||
|
||||
q("ALTER TABLE test.partition_complex DETACH PARTITION 197001")
|
||||
q("ALTER TABLE test.partition_complex ATTACH PARTITION 197001")
|
||||
@ -144,7 +175,7 @@ def test_partition_complex(partition_table_complex):
|
||||
q("ALTER TABLE test.partition_complex MODIFY COLUMN v1 Int8")
|
||||
|
||||
# Check the backup hasn't changed
|
||||
partition_complex_assert_checksums()
|
||||
partition_complex_assert_checksums(True)
|
||||
|
||||
q("OPTIMIZE TABLE test.partition_complex")
|
||||
|
||||
|
@ -5,7 +5,7 @@
|
||||
-1275 -424.99999983 -255 -1275 -424.99999983 -255
|
||||
101 101 101 101 101 101
|
||||
-101 -101 -101 -101 -101 -101
|
||||
(101,101,101) (101,101,101) (101,101,101) (101,101,101) (102,100,101)
|
||||
(101,101,101) (101,101,101) (101,101,101) (101,101,101) (1,1,1,1,1,1)
|
||||
5 5 5
|
||||
10 10 10
|
||||
-50 -50 -16.66666666 -16.66666666 -10 -10
|
||||
|
@ -24,7 +24,7 @@ SELECT (uniq(a), uniq(b), uniq(c)),
|
||||
(uniqCombined(a), uniqCombined(b), uniqCombined(c)),
|
||||
(uniqCombined(17)(a), uniqCombined(17)(b), uniqCombined(17)(c)),
|
||||
(uniqExact(a), uniqExact(b), uniqExact(c)),
|
||||
(uniqHLL12(a), uniqHLL12(b), uniqHLL12(c))
|
||||
(102 - uniqHLL12(a) >= 0, 102 - uniqHLL12(b) >= 0, 102 - uniqHLL12(c) >= 0, uniqHLL12(a) - 99 >= 0, uniqHLL12(b) - 99 >= 0, uniqHLL12(c) - 99 >= 0)
|
||||
FROM (SELECT * FROM decimal ORDER BY a);
|
||||
|
||||
SELECT uniqUpTo(10)(a), uniqUpTo(10)(b), uniqUpTo(10)(c) FROM decimal WHERE a >= 0 AND a < 5;
|
||||
|
@ -63,7 +63,6 @@ function thread6()
|
||||
done
|
||||
}
|
||||
|
||||
|
||||
# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout
|
||||
export -f thread1;
|
||||
export -f thread2;
|
||||
|
@ -18,22 +18,18 @@ Response 0 Create /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 \N 0 4
|
||||
Request 0 Exists /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 \N 0 0 \N \N \N 0 0 0 0
|
||||
Response 0 Exists /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 \N 0 0 ZOK \N \N 0 0 96 0
|
||||
blocks
|
||||
Request 0 Multi 0 0 \N 3 0 \N \N \N 0 0 0 0
|
||||
Request 0 Create /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 1 \N \N \N 0 0 0 0
|
||||
Request 0 Remove /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 -1 0 2 \N \N \N 0 0 0 0
|
||||
Request 0 Create /test/01158/default/rmt/block_numbers/all/block- 1 1 \N 0 3 \N \N \N 0 0 0 0
|
||||
Response 0 Multi 0 0 \N 3 0 ZOK \N \N 0 0 0 0
|
||||
Response 0 Create /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 1 ZOK \N \N /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 0 0
|
||||
Response 0 Remove /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 -1 0 2 ZOK \N \N 0 0 0 0
|
||||
Response 0 Create /test/01158/default/rmt/block_numbers/all/block- 1 1 \N 0 3 ZOK \N \N /test/01158/default/rmt/block_numbers/all/block-0000000000 0 0 0 0
|
||||
Request 0 Multi 0 0 \N 3 0 \N \N \N 0 0 0 0
|
||||
Request 0 Create /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 1 \N \N \N 0 0 0 0
|
||||
Request 0 Remove /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 -1 0 2 \N \N \N 0 0 0 0
|
||||
Request 0 Create /test/01158/default/rmt/block_numbers/all/block- 1 1 \N 0 3 \N \N \N 0 0 0 0
|
||||
Response 0 Multi 0 0 \N 3 0 ZNODEEXISTS \N \N 0 0 0 0
|
||||
Response 0 Error /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 1 ZNODEEXISTS \N \N 0 0 0 0
|
||||
Response 0 Error /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 -1 0 2 ZRUNTIMEINCONSISTENCY \N \N 0 0 0 0
|
||||
Response 0 Error /test/01158/default/rmt/block_numbers/all/block- 1 1 \N 0 3 ZRUNTIMEINCONSISTENCY \N \N 0 0 0 0
|
||||
Request 0 Multi 0 0 \N 2 0 \N \N \N 0 0 0 0
|
||||
Request 0 CheckNotExists /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 -1 0 1 \N \N \N 0 0 0 0
|
||||
Request 0 Create /test/01158/default/rmt/block_numbers/all/block- 1 1 \N 0 2 \N \N \N 0 0 0 0
|
||||
Response 0 Multi 0 0 \N 2 0 ZOK \N \N 0 0 0 0
|
||||
Response 0 CheckNotExists /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 -1 0 1 ZOK \N \N 0 0 0 0
|
||||
Response 0 Create /test/01158/default/rmt/block_numbers/all/block- 1 1 \N 0 2 ZOK \N \N /test/01158/default/rmt/block_numbers/all/block-0000000000 0 0 0 0
|
||||
Request 0 Multi 0 0 \N 2 0 \N \N \N 0 0 0 0
|
||||
Request 0 CheckNotExists /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 -1 0 1 \N \N \N 0 0 0 0
|
||||
Request 0 Create /test/01158/default/rmt/block_numbers/all/block- 1 1 \N 0 2 \N \N \N 0 0 0 0
|
||||
Response 0 Multi 0 0 \N 2 0 ZNODEEXISTS \N \N 0 0 0 0
|
||||
Response 0 Error /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 -1 0 1 ZNODEEXISTS \N \N 0 0 0 0
|
||||
Response 0 Error /test/01158/default/rmt/block_numbers/all/block- 1 1 \N 0 2 ZRUNTIMEINCONSISTENCY \N \N 0 0 0 0
|
||||
Request 0 Get /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 0 \N \N \N 0 0 0 0
|
||||
Response 0 Get /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 0 ZOK \N \N 0 0 9 0
|
||||
duration_ms
|
||||
|
@ -1,5 +1,4 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: long
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
@ -8,7 +7,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# NOTE: database = $CLICKHOUSE_DATABASE is unwanted
|
||||
verify_sql="SELECT
|
||||
(SELECT sumIf(value, metric = 'PartsActive'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics)
|
||||
= (SELECT sum(active), sum(NOT active) FROM system.parts)"
|
||||
= (SELECT sum(active), sum(NOT active) FROM
|
||||
(SELECT active FROM system.parts UNION ALL SELECT active FROM system.projection_parts))"
|
||||
|
||||
# The query is not atomic - it can compare states between system.parts and system.metrics from different points in time.
|
||||
# So, there is inherent race condition. But it should get expected result eventually.
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: long, no-s3-storage
|
||||
# Tags: no-s3-storage
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
@ -11,7 +11,8 @@ set -o pipefail
|
||||
# NOTE: database = $CLICKHOUSE_DATABASE is unwanted
|
||||
verify_sql="SELECT
|
||||
(SELECT sumIf(value, metric = 'PartsInMemory'), sumIf(value, metric = 'PartsCompact'), sumIf(value, metric = 'PartsWide') FROM system.metrics) =
|
||||
(SELECT countIf(part_type == 'InMemory'), countIf(part_type == 'Compact'), countIf(part_type == 'Wide') FROM system.parts)"
|
||||
(SELECT countIf(part_type == 'InMemory'), countIf(part_type == 'Compact'), countIf(part_type == 'Wide')
|
||||
FROM (SELECT part_type FROM system.parts UNION ALL SELECT part_type FROM system.projection_parts))"
|
||||
|
||||
# The query is not atomic - it can compare states between system.parts and system.metrics from different points in time.
|
||||
# So, there is inherent race condition (especially in fasttest that runs tests in parallel).
|
||||
|
@ -361,6 +361,7 @@ CREATE TABLE system.merges
|
||||
`partition_id` String,
|
||||
`is_mutation` UInt8,
|
||||
`total_size_bytes_compressed` UInt64,
|
||||
`total_size_bytes_uncompressed` UInt64,
|
||||
`total_size_marks` UInt64,
|
||||
`bytes_read_uncompressed` UInt64,
|
||||
`rows_read` UInt64,
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
SET send_logs_level = 'fatal';
|
||||
|
||||
drop table if exists rmt;
|
||||
drop table if exists rmt2;
|
||||
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user