Merge branch 'master' into system-sync-cache

This commit is contained in:
Kseniia Sumarokova 2023-08-10 13:18:35 +02:00 committed by GitHub
commit aed955a331
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
120 changed files with 997 additions and 6210 deletions
.github/workflows
docker/packager
docs/en/sql-reference
data-types
functions
statements
src
tests
analyzer_tech_debt.txt
ci
integration
test_cluster_copier
test_dictionaries_all_layouts_separate_sources
test_dictionaries_complex_key_cache_string
test_dictionary_allow_read_expired_keys
test_extreme_deduplication
test_hive_query
test_host_regexp_multiple_ptr_records
test_insert_into_distributed_through_materialized_view
test_keeper_reconfig_replace_leader_in_one_command
test_mutations_with_merge_tree
test_polymorphic_parts
test_profile_max_sessions_for_user
test_replicated_merge_tree_hdfs_zero_copy
test_reverse_dns_query
test_s3_zero_copy_replication
test_s3_zero_copy_ttl
test_session_log
test_storage_nats
test_storage_postgresql_replica

View File

@ -3,6 +3,9 @@ name: BackportPR
env:
# Force the stdout and stderr streams to be unbuffered
PYTHONUNBUFFERED: 1
# Export system tables to ClickHouse Cloud
CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }}
CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }}
on: # yamllint disable-line rule:truthy
push:

View File

@ -3,6 +3,9 @@ name: MasterCI
env:
# Force the stdout and stderr streams to be unbuffered
PYTHONUNBUFFERED: 1
# Export system tables to ClickHouse Cloud
CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }}
CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }}
on: # yamllint disable-line rule:truthy
push:

View File

@ -3,6 +3,9 @@ name: PullRequestCI
env:
# Force the stdout and stderr streams to be unbuffered
PYTHONUNBUFFERED: 1
# Export system tables to ClickHouse Cloud
CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }}
CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }}
on: # yamllint disable-line rule:truthy
pull_request:

View File

@ -3,6 +3,9 @@ name: ReleaseBranchCI
env:
# Force the stdout and stderr streams to be unbuffered
PYTHONUNBUFFERED: 1
# Export system tables to ClickHouse Cloud
CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }}
CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }}
on: # yamllint disable-line rule:truthy
push:

View File

@ -22,7 +22,7 @@ def check_image_exists_locally(image_name: str) -> bool:
output = subprocess.check_output(
f"docker images -q {image_name} 2> /dev/null", shell=True
)
return output != ""
return output != b""
except subprocess.CalledProcessError:
return False
@ -46,7 +46,7 @@ def build_image(image_name: str, filepath: Path) -> None:
)
def pre_build(repo_path: Path, env_variables: List[str]):
def pre_build(repo_path: Path, env_variables: List[str]) -> None:
if "WITH_PERFORMANCE=1" in env_variables:
current_branch = subprocess.check_output(
"git branch --show-current", shell=True, encoding="utf-8"
@ -81,8 +81,9 @@ def run_docker_image_with_env(
env_variables: List[str],
ch_root: Path,
ccache_dir: Optional[Path],
):
) -> None:
output_dir.mkdir(parents=True, exist_ok=True)
env_part = " -e ".join(env_variables)
if env_part:
env_part = " -e " + env_part
@ -129,9 +130,10 @@ def parse_env_variables(
version: str,
official: bool,
additional_pkgs: bool,
with_profiler: bool,
with_coverage: bool,
with_binaries: str,
):
) -> List[str]:
DARWIN_SUFFIX = "-darwin"
DARWIN_ARM_SUFFIX = "-darwin-aarch64"
ARM_SUFFIX = "-aarch64"
@ -322,6 +324,9 @@ def parse_env_variables(
# utils are not included into clickhouse-bundle, so build everything
build_target = "all"
if with_profiler:
cmake_flags.append("-DENABLE_BUILD_PROFILING=1")
if with_coverage:
cmake_flags.append("-DWITH_COVERAGE=1")
@ -416,6 +421,7 @@ def parse_args() -> argparse.Namespace:
parser.add_argument("--version")
parser.add_argument("--official", action="store_true")
parser.add_argument("--additional-pkgs", action="store_true")
parser.add_argument("--with-profiler", action="store_true")
parser.add_argument("--with-coverage", action="store_true")
parser.add_argument(
"--with-binaries", choices=("programs", "tests", ""), default=""
@ -451,7 +457,7 @@ def parse_args() -> argparse.Namespace:
return args
def main():
def main() -> None:
logging.basicConfig(level=logging.INFO, format="%(asctime)s %(message)s")
args = parse_args()
@ -479,6 +485,7 @@ def main():
args.version,
args.official,
args.additional_pkgs,
args.with_profiler,
args.with_coverage,
args.with_binaries,
)

View File

@ -6,42 +6,42 @@ sidebar_label: UUID
# UUID
A universally unique identifier (UUID) is a 16-byte number used to identify records. For detailed information about the UUID, see [Wikipedia](https://en.wikipedia.org/wiki/Universally_unique_identifier).
A Universally Unique Identifier (UUID) is a 16-byte value used to identify records. For detailed information about UUIDs, see [Wikipedia](https://en.wikipedia.org/wiki/Universally_unique_identifier).
The example of UUID type value is represented below:
While different UUID variants exist (see [here](https://datatracker.ietf.org/doc/html/draft-ietf-uuidrev-rfc4122bis)), ClickHouse does not validate that inserted UUIDs conform to a particular variant. UUIDs are internally treated as a sequence of 16 random bytes with [8-4-4-4-12 representation](https://en.wikipedia.org/wiki/Universally_unique_identifier#Textual_representation) at SQL level.
Example UUID value:
``` text
61f0c404-5cb3-11e7-907b-a6006ad3dba0
```
If you do not specify the UUID column value when inserting a new record, the UUID value is filled with zero:
The default UUID is all-zero. It is used, for example, when a new record is inserted but no value for a UUID column is specified:
``` text
00000000-0000-0000-0000-000000000000
```
## How to Generate
## Generating UUIDs
To generate the UUID value, ClickHouse provides the [generateUUIDv4](../../sql-reference/functions/uuid-functions.md) function.
ClickHouse provides the [generateUUIDv4](../../sql-reference/functions/uuid-functions.md) function to generate random UUID version 4 values.
## Usage Example
**Example 1**
This example demonstrates creating a table with the UUID type column and inserting a value into the table.
This example demonstrates the creation of a table with a UUID column and the insertion of a value into the table.
``` sql
CREATE TABLE t_uuid (x UUID, y String) ENGINE=TinyLog
```
``` sql
INSERT INTO t_uuid SELECT generateUUIDv4(), 'Example 1'
```
``` sql
SELECT * FROM t_uuid
```
Result:
``` text
┌────────────────────────────────────x─┬─y─────────┐
│ 417ddc5d-e556-4d27-95dd-a34d84e46a50 │ Example 1 │
@ -50,13 +50,11 @@ SELECT * FROM t_uuid
**Example 2**
In this example, the UUID column value is not specified when inserting a new record.
In this example, no UUID column value is specified when the record is inserted, i.e. the default UUID value is inserted:
``` sql
INSERT INTO t_uuid (y) VALUES ('Example 2')
```
``` sql
SELECT * FROM t_uuid
```

View File

@ -729,6 +729,30 @@ Returns whether string `str` ends with `suffix`.
endsWith(str, suffix)
```
## endsWithUTF8
Returns whether string `str` ends with `suffix`, the difference between `endsWithUTF8` and `endsWith` is that `endsWithUTF8` match `str` and `suffix` by UTF-8 characters.
**Syntax**
```sql
endsWithUTF8(str, suffix)
```
**Example**
``` sql
SELECT endsWithUTF8('中国', '\xbd'), endsWith('中国', '\xbd')
```
Result:
```result
┌─endsWithUTF8('中国', '½')─┬─endsWith('中国', '½')─┐
│ 0 │ 1 │
└──────────────────────────┴──────────────────────┘
```
## startsWith
Returns whether string `str` starts with `prefix`.
@ -745,6 +769,25 @@ startsWith(str, prefix)
SELECT startsWith('Spider-Man', 'Spi');
```
## startsWithUTF8
Returns whether string `str` starts with `prefix`, the difference between `startsWithUTF8` and `startsWith` is that `startsWithUTF8` match `str` and `suffix` by UTF-8 characters.
**Example**
``` sql
SELECT startsWithUTF8('中国', '\xe4'), startsWith('中国', '\xe4')
```
Result:
```result
┌─startsWithUTF8('中国', '⥩─┬─startsWith('中国', '⥩─┐
│ 0 │ 1 │
└────────────────────────────┴────────────────────────┘
```
## trim
Removes the specified characters from the start or end of a string. If not specified otherwise, the function removes whitespace (ASCII-character 32).

View File

@ -5,19 +5,38 @@ sidebar_label: CHECK TABLE
title: "CHECK TABLE Statement"
---
Checks if the data in the table is corrupted.
The `CHECK TABLE` query in ClickHouse is used to perform a validation check on a specific table or its partitions. It ensures the integrity of the data by verifying the checksums and other internal data structures.
``` sql
CHECK TABLE [db.]name [PARTITION partition_expr]
Particularly it compares actual file sizes with the expected values which are stored on the server. If the file sizes do not match the stored values, it means the data is corrupted. This can be caused, for example, by a system crash during query execution.
:::note
The `CHECK TABLE`` query may read all the data in the table and hold some resources, making it resource-intensive.
Consider the potential impact on performance and resource utilization before executing this query.
:::
## Syntax
The basic syntax of the query is as follows:
```sql
CHECK TABLE table_name [PARTITION partition_expression] [FORMAT format] [SETTINGS check_query_single_value_result = (0|1) [, other_settings] ]
```
The `CHECK TABLE` query compares actual file sizes with the expected values which are stored on the server. If the file sizes do not match the stored values, it means the data is corrupted. This can be caused, for example, by a system crash during query execution.
- `table_name`: Specifies the name of the table that you want to check.
- `partition_expression`: (Optional) If you want to check a specific partition of the table, you can use this expression to specify the partition.
- `FORMAT format`: (Optional) Allows you to specify the output format of the result.
- `SETTINGS`: (Optional) Allows additional settings.
- **`check_query_single_value_result`**: (Optional) This setting allows you to toggle between a detailed result (`0`) or a summarized result (`1`).
- Other settings (e.g. `max_threads` can be applied as well).
The query response contains the `result` column with a single row. The row has a value of
[Boolean](../../sql-reference/data-types/boolean.md) type:
- 0 - The data in the table is corrupted.
- 1 - The data maintains integrity.
The query response depends on the value of contains `check_query_single_value_result` setting.
In case of `check_query_single_value_result = 1` only `result` column with a single row is returned. Value inside this row is `1` if the integrity check is passed and `0` if data is corrupted.
With `check_query_single_value_result = 0` the query returns the following columns:
- `part_path`: Indicates the path to the data part or file name.
- `is_passed`: Returns 1 if the check for this part is successful, 0 otherwise.
- `message`: Any additional messages related to the check, such as errors or success messages.
The `CHECK TABLE` query supports the following table engines:
@ -26,30 +45,15 @@ The `CHECK TABLE` query supports the following table engines:
- [StripeLog](../../engines/table-engines/log-family/stripelog.md)
- [MergeTree family](../../engines/table-engines/mergetree-family/mergetree.md)
Performed over the tables with another table engines causes an exception.
Performed over the tables with another table engines causes an `NOT_IMPLEMETED` exception.
Engines from the `*Log` family do not provide automatic data recovery on failure. Use the `CHECK TABLE` query to track data loss in a timely manner.
## Checking the MergeTree Family Tables
## Examples
For `MergeTree` family engines, if [check_query_single_value_result](../../operations/settings/settings.md#check_query_single_value_result) = 0, the `CHECK TABLE` query shows a check status for every individual data part of a table on the local server.
By default `CHECK TABLE` query shows the general table check status:
```sql
SET check_query_single_value_result = 0;
CHECK TABLE test_table;
```
```text
┌─part_path─┬─is_passed─┬─message─┐
│ all_1_4_1 │ 1 │ │
│ all_1_4_2 │ 1 │ │
└───────────┴───────────┴─────────┘
```
If `check_query_single_value_result` = 1, the `CHECK TABLE` query shows the general table check status.
```sql
SET check_query_single_value_result = 1;
CHECK TABLE test_table;
```
@ -59,11 +63,60 @@ CHECK TABLE test_table;
└────────┘
```
If you want to see the check status for every individual data part you may use `check_query_single_value_result` setting.
Also, to check a specific partition of the table, you can use the `PARTITION` keyword.
```sql
CHECK TABLE t0 PARTITION ID '201003'
FORMAT PrettyCompactMonoBlock
SETTINGS check_query_single_value_result = 0
```
Output:
```text
┌─part_path────┬─is_passed─┬─message─┐
│ 201003_7_7_0 │ 1 │ │
│ 201003_3_3_0 │ 1 │ │
└──────────────┴───────────┴─────────┘
```
### Receiving a 'Corrupted' Result
:::warning
Disclaimer: The procedure described here, including the manual manipulating or removing files directly from the data directory, is for experimental or development environments only. Do **not** attempt this on a production server, as it may lead to data loss or other unintended consequences.
:::
Remove the existing checksum file:
```bash
rm /var/lib/clickhouse-server/data/default/t0/201003_3_3_0/checksums.txt
```
```sql
CHECK TABLE t0 PARTITION ID '201003'
FORMAT PrettyCompactMonoBlock
SETTINGS check_query_single_value_result = 0
Output:
```text
┌─part_path────┬─is_passed─┬─message──────────────────────────────────┐
│ 201003_7_7_0 │ 1 │ │
│ 201003_3_3_0 │ 1 │ Checksums recounted and written to disk. │
└──────────────┴───────────┴──────────────────────────────────────────┘
```
If the checksums.txt file is missing, it can be restored. It will be recalculated and rewritten during the execution of the CHECK TABLE command for the specific partition, and the status will still be reported as 'success.'"
## If the Data Is Corrupted
If the table is corrupted, you can copy the non-corrupted data to another table. To do this:
1. Create a new table with the same structure as damaged table. To do this execute the query `CREATE TABLE <new_table_name> AS <damaged_table_name>`.
2. Set the [max_threads](../../operations/settings/settings.md#settings-max_threads) value to 1 to process the next query in a single thread. To do this run the query `SET max_threads = 1`.
2. Set the `max_threads` value to 1 to process the next query in a single thread. To do this run the query `SET max_threads = 1`.
3. Execute the query `INSERT INTO <new_table_name> SELECT * FROM <damaged_table_name>`. This request copies the non-corrupted data from the damaged table to another table. Only the data before the corrupted part will be copied.
4. Restart the `clickhouse-client` to reset the `max_threads` value.

View File

@ -82,33 +82,35 @@ size_t FileChecker::getTotalSize() const
}
CheckResults FileChecker::check() const
FileChecker::DataValidationTasksPtr FileChecker::getDataValidationTasks()
{
if (map.empty())
return {};
return std::make_unique<DataValidationTasks>(map);
}
CheckResults results;
for (const auto & name_size : map)
CheckResult FileChecker::checkNextEntry(DataValidationTasksPtr & check_data_tasks, bool & has_nothing_to_do) const
{
String name;
size_t expected_size;
bool is_finished = check_data_tasks->next(name, expected_size);
if (is_finished)
{
const String & name = name_size.first;
String path = parentPath(files_info_path) + name;
bool exists = fileReallyExists(path);
auto real_size = exists ? getRealFileSize(path) : 0; /// No race condition assuming no one else is working with these files.
if (real_size != name_size.second)
{
String failure_message = exists
? ("Size of " + path + " is wrong. Size is " + toString(real_size) + " but should be " + toString(name_size.second))
: ("File " + path + " doesn't exist");
results.emplace_back(name, false, failure_message);
break;
}
results.emplace_back(name, true, "");
has_nothing_to_do = true;
return {};
}
return results;
String path = parentPath(files_info_path) + name;
bool exists = fileReallyExists(path);
auto real_size = exists ? getRealFileSize(path) : 0; /// No race condition assuming no one else is working with these files.
if (real_size != expected_size)
{
String failure_message = exists
? ("Size of " + path + " is wrong. Size is " + toString(real_size) + " but should be " + toString(expected_size))
: ("File " + path + " doesn't exist");
return CheckResult(name, false, failure_message);
}
return CheckResult(name, true, "");
}
void FileChecker::repair()

View File

@ -3,6 +3,7 @@
#include <Storages/CheckResults.h>
#include <map>
#include <base/types.h>
#include <mutex>
namespace Poco { class Logger; }
@ -28,7 +29,11 @@ public:
bool empty() const { return map.empty(); }
/// Check the files whose parameters are specified in sizes.json
CheckResults check() const;
/// See comment in IStorage::checkDataNext
struct DataValidationTasks;
using DataValidationTasksPtr = std::unique_ptr<DataValidationTasks>;
DataValidationTasksPtr getDataValidationTasks();
CheckResult checkNextEntry(DataValidationTasksPtr & check_data_tasks, bool & has_nothing_to_do) const;
/// Truncate files that have excessive size to the expected size.
/// Throw exception if the file size is less than expected.
@ -41,6 +46,36 @@ public:
/// Returns total size of all files.
size_t getTotalSize() const;
struct DataValidationTasks
{
DataValidationTasks(const std::map<String, size_t> & map_)
: map(map_), it(map.begin())
{}
bool next(String & out_name, size_t & out_size)
{
std::lock_guard lock(mutex);
if (it == map.end())
return true;
out_name = it->first;
out_size = it->second;
++it;
return false;
}
size_t size() const
{
std::lock_guard lock(mutex);
return std::distance(it, map.end());
}
const std::map<String, size_t> & map;
mutable std::mutex mutex;
using Iterator = std::map<String, size_t>::const_iterator;
Iterator it;
};
private:
void load();

View File

@ -102,6 +102,19 @@ bool ReadBufferFromAzureBlobStorage::nextImpl()
size_t bytes_read = 0;
size_t sleep_time_with_backoff_milliseconds = 100;
auto handle_exception = [&, this](const auto & e, size_t i)
{
LOG_INFO(log, "Exception caught during Azure Read for file {} at attempt {}/{}: {}", path, i + 1, max_single_read_retries, e.Message);
if (i + 1 == max_single_read_retries)
throw;
sleepForMilliseconds(sleep_time_with_backoff_milliseconds);
sleep_time_with_backoff_milliseconds *= 2;
initialized = false;
initialize();
};
for (size_t i = 0; i < max_single_read_retries; ++i)
{
try
@ -111,16 +124,13 @@ bool ReadBufferFromAzureBlobStorage::nextImpl()
read_settings.remote_throttler->add(bytes_read, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds);
break;
}
catch (const Azure::Core::Http::TransportException & e)
{
handle_exception(e, i);
}
catch (const Azure::Storage::StorageException & e)
{
LOG_INFO(log, "Exception caught during Azure Read for file {} at attempt {}: {}", path, i, e.Message);
if (i + 1 == max_single_read_retries)
throw;
sleepForMilliseconds(sleep_time_with_backoff_milliseconds);
sleep_time_with_backoff_milliseconds *= 2;
initialized = false;
initialize();
handle_exception(e, i);
}
}
@ -211,6 +221,17 @@ void ReadBufferFromAzureBlobStorage::initialize()
blob_client = std::make_unique<Azure::Storage::Blobs::BlobClient>(blob_container_client->GetBlobClient(path));
size_t sleep_time_with_backoff_milliseconds = 100;
auto handle_exception = [&, this](const auto & e, size_t i)
{
LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {}/{}: {}", path, offset, i + 1, max_single_download_retries, e.Message);
if (i + 1 == max_single_download_retries)
throw;
sleepForMilliseconds(sleep_time_with_backoff_milliseconds);
sleep_time_with_backoff_milliseconds *= 2;
};
for (size_t i = 0; i < max_single_download_retries; ++i)
{
try
@ -219,14 +240,13 @@ void ReadBufferFromAzureBlobStorage::initialize()
data_stream = std::move(download_response.Value.BodyStream);
break;
}
catch (const Azure::Core::Http::TransportException & e)
{
handle_exception(e, i);
}
catch (const Azure::Core::RequestFailedException & e)
{
LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {} : {}", path, offset, i + 1, e.Message);
if (i + 1 == max_single_download_retries)
throw;
sleepForMilliseconds(sleep_time_with_backoff_milliseconds);
sleep_time_with_backoff_milliseconds *= 2;
handle_exception(e,i);
}
}

View File

@ -1173,21 +1173,8 @@ class FunctionBinaryArithmetic : public IFunction
const auto * left_array_col = typeid_cast<const ColumnArray *>(arguments[0].column.get());
const auto * right_array_col = typeid_cast<const ColumnArray *>(arguments[1].column.get());
const auto & left_offsets = left_array_col->getOffsets();
const auto & right_offsets = right_array_col->getOffsets();
chassert(left_offsets.size() == right_offsets.size() && "Unexpected difference in number of offsets");
/// Unpacking non-const arrays and checking sizes of them.
for (auto offset_index = 0U; offset_index < left_offsets.size(); ++offset_index)
{
if (left_offsets[offset_index] != right_offsets[offset_index])
{
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH,
"Cannot apply operation for arrays of different sizes. Size of the first argument: {}, size of the second argument: {}",
*left_array_col->getOffsets().data(),
*right_array_col ->getOffsets().data());
}
}
if (!left_array_col->hasEqualOffsets(*right_array_col))
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Two arguments for function {} must have equal sizes", getName());
const auto & left_array_type = typeid_cast<const DataTypeArray *>(arguments[0].type.get())->getNestedType();
new_arguments[0] = {left_array_col->getDataPtr(), left_array_type, arguments[0].name};
@ -1198,6 +1185,7 @@ class FunctionBinaryArithmetic : public IFunction
result_array_type = typeid_cast<const DataTypeArray *>(result_type.get())->getNestedType();
size_t rows_count = 0;
const auto & left_offsets = left_array_col->getOffsets();
if (!left_offsets.empty())
rows_count = left_offsets.back();
auto res = executeImpl(new_arguments, result_array_type, rows_count);

View File

@ -28,10 +28,24 @@ namespace ErrorCodes
struct NameStartsWith
{
static constexpr auto name = "startsWith";
static constexpr auto is_utf8 = false;
};
struct NameEndsWith
{
static constexpr auto name = "endsWith";
static constexpr auto is_utf8 = false;
};
struct NameStartsWithUTF8
{
static constexpr auto name = "startsWithUTF8";
static constexpr auto is_utf8 = true;
};
struct NameEndsWithUTF8
{
static constexpr auto name = "endsWithUTF8";
static constexpr auto is_utf8 = true;
};
DECLARE_MULTITARGET_CODE(
@ -41,6 +55,7 @@ class FunctionStartsEndsWith : public IFunction
{
public:
static constexpr auto name = Name::name;
static constexpr auto is_utf8 = Name::is_utf8;
String getName() const override
{
@ -64,7 +79,8 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (isStringOrFixedString(arguments[0]) && isStringOrFixedString(arguments[1]))
if (!is_utf8 && isStringOrFixedString(arguments[0]) && isStringOrFixedString(arguments[1])
|| isString(arguments[0]) && isString(arguments[1]))
return std::make_shared<DataTypeUInt8>();
if (isArray(arguments[0]) && isArray(arguments[1]))
@ -78,8 +94,11 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
auto data_type = arguments[0].type;
if (isStringOrFixedString(*data_type))
if (!is_utf8 && isStringOrFixedString(*data_type))
return executeImplString(arguments, {}, input_rows_count);
if (is_utf8 && isString(*data_type))
return executeImplStringUTF8(arguments, {}, input_rows_count);
if (isArray(data_type))
return executeImplArray(arguments, {}, input_rows_count);
return {};
@ -131,7 +150,6 @@ private:
typename ColumnVector<UInt8>::Container & vec_res = col_res->getData();
vec_res.resize(input_rows_count);
if (const ColumnString * haystack = checkAndGetColumn<ColumnString>(haystack_column))
dispatch<StringSource>(StringSource(*haystack), needle_column, vec_res);
else if (const ColumnFixedString * haystack_fixed = checkAndGetColumn<ColumnFixedString>(haystack_column))
@ -146,6 +164,26 @@ private:
return col_res;
}
ColumnPtr executeImplStringUTF8(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const
{
const IColumn * haystack_column = arguments[0].column.get();
const IColumn * needle_column = arguments[1].column.get();
auto col_res = ColumnVector<UInt8>::create();
typename ColumnVector<UInt8>::Container & vec_res = col_res->getData();
vec_res.resize(input_rows_count);
if (const ColumnString * haystack = checkAndGetColumn<ColumnString>(haystack_column))
dispatchUTF8<UTF8StringSource>(UTF8StringSource(*haystack), needle_column, vec_res);
else if (const ColumnConst * haystack_const = checkAndGetColumnConst<ColumnString>(haystack_column))
dispatchUTF8<ConstSource<UTF8StringSource>>(ConstSource<UTF8StringSource>(*haystack_const), needle_column, vec_res);
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal combination of columns as arguments of function {}", getName());
return col_res;
}
template <typename HaystackSource>
void dispatch(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray<UInt8> & res_data) const
{
@ -161,6 +199,17 @@ private:
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal combination of columns as arguments of function {}", getName());
}
template <typename HaystackSource>
void dispatchUTF8(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray<UInt8> & res_data) const
{
if (const ColumnString * needle = checkAndGetColumn<ColumnString>(needle_column))
execute<HaystackSource, UTF8StringSource>(haystack_source, UTF8StringSource(*needle), res_data);
else if (const ColumnConst * needle_const = checkAndGetColumnConst<ColumnString>(needle_column))
execute<HaystackSource, ConstSource<UTF8StringSource>>(haystack_source, ConstSource<UTF8StringSource>(*needle_const), res_data);
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal combination of columns as arguments of function {}", getName());
}
template <typename HaystackSource, typename NeedleSource>
static void execute(HaystackSource haystack_source, NeedleSource needle_source, PaddedPODArray<UInt8> & res_data)
{
@ -172,18 +221,27 @@ private:
auto needle = needle_source.getWhole();
if (needle.size > haystack.size)
{
res_data[row_num] = false;
}
else
{
if constexpr (std::is_same_v<Name, NameStartsWith>)
{
if constexpr (std::is_same_v<Name, NameStartsWith>) /// startsWith
res_data[row_num] = StringRef(haystack.data, needle.size) == StringRef(needle.data, needle.size);
}
else /// endsWith
{
else if constexpr (std::is_same_v<Name, NameEndsWith>) /// endsWith
res_data[row_num] = StringRef(haystack.data + haystack.size - needle.size, needle.size) == StringRef(needle.data, needle.size);
else /// startsWithUTF8 or endsWithUTF8
{
auto length = UTF8::countCodePoints(needle.data, needle.size);
if constexpr (std::is_same_v<Name, NameStartsWithUTF8>)
{
auto slice = haystack_source.getSliceFromLeft(0, length);
res_data[row_num] = StringRef(slice.data, slice.size) == StringRef(needle.data, needle.size);
}
else
{
auto slice = haystack_source.getSliceFromRight(length);
res_data[row_num] = StringRef(slice.data, slice.size) == StringRef(needle.data, needle.size);
}
}
}

View File

@ -0,0 +1,21 @@
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionStartsEndsWith.h>
namespace DB
{
using FunctionEndsWithUTF8 = FunctionStartsEndsWith<NameEndsWithUTF8>;
REGISTER_FUNCTION(EndsWithUTF8)
{
factory.registerFunction<FunctionEndsWithUTF8>(FunctionDocumentation{
.description = R"(
Returns whether string `str` ends with `suffix`, the difference between `endsWithUTF8` and `endsWith` is that `endsWithUTF8` match `str` and `suffix` by UTF-8 characters.
)",
.examples{{"endsWithUTF8", "select endsWithUTF8('富强民主文明和谐', '富强');", ""}},
.categories{"String"}});
}
}

View File

@ -0,0 +1,21 @@
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionStartsEndsWith.h>
namespace DB
{
using FunctionStartsWithUTF8 = FunctionStartsEndsWith<NameStartsWithUTF8>;
REGISTER_FUNCTION(StartsWithUTF8)
{
factory.registerFunction<FunctionStartsWithUTF8>(FunctionDocumentation{
.description = R"(
Returns whether string `str` starts with `prefix`, the difference between `startsWithUTF8` and `startsWith` is that `startsWithUTF8` match `str` and `suffix` by UTF-8 characters.
)",
.examples{{"startsWithUTF8", "select startsWithUTF8('富强民主文明和谐', '富强');", ""}},
.categories{"String"}});
}
}

View File

@ -1101,17 +1101,14 @@ inline void readBinary(bool & x, ReadBuffer & buf)
}
inline void readBinary(String & x, ReadBuffer & buf) { readStringBinary(x, buf); }
inline void readBinary(Int32 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(Int128 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(Int256 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(UInt32 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(UInt128 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(UInt256 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(Decimal32 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(Decimal64 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(Decimal128 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(Decimal256 & x, ReadBuffer & buf) { readPODBinary(x.value, buf); }
inline void readBinary(LocalDate & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(UUID & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(IPv4 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(IPv6 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(StackTrace::FramePointers & x, ReadBuffer & buf) { readPODBinary(x, buf); }

View File

@ -8,72 +8,201 @@
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnsNumber.h>
#include <Common/typeid_cast.h>
#include <Interpreters/ProcessList.h>
#include <algorithm>
#include <Columns/IColumn.h>
#include <Processors/ResizeProcessor.h>
#include <Processors/IAccumulatingTransform.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{
NamesAndTypes getBlockStructure()
Block getSingleValueBlock(UInt8 value)
{
return {
return Block{{ColumnUInt8::create(1, value), std::make_shared<DataTypeUInt8>(), "result"}};
}
Block getHeaderForCheckResult()
{
auto names_and_types = NamesAndTypes{
{"part_path", std::make_shared<DataTypeString>()},
{"is_passed", std::make_shared<DataTypeUInt8>()},
{"message", std::make_shared<DataTypeString>()},
};
return Block({
{names_and_types[0].type->createColumn(), names_and_types[0].type, names_and_types[0].name},
{names_and_types[1].type->createColumn(), names_and_types[1].type, names_and_types[1].name},
{names_and_types[2].type->createColumn(), names_and_types[2].type, names_and_types[2].name},
});
}
Chunk getChunkFromCheckResult(const CheckResult & check_result)
{
MutableColumns columns = getHeaderForCheckResult().cloneEmptyColumns();
columns[0]->insert(check_result.fs_path);
columns[1]->insert(static_cast<UInt8>(check_result.success));
columns[2]->insert(check_result.failure_message);
return Chunk(std::move(columns), 1);
}
class TableCheckWorkerProcessor : public ISource
{
public:
TableCheckWorkerProcessor(IStorage::DataValidationTasksPtr check_data_tasks_, StoragePtr table_)
: ISource(getHeaderForCheckResult())
, table(table_)
, check_data_tasks(check_data_tasks_)
{
}
String getName() const override { return "TableCheckWorkerProcessor"; }
protected:
std::optional<Chunk> tryGenerate() override
{
bool has_nothing_to_do = false;
auto check_result = table->checkDataNext(check_data_tasks, has_nothing_to_do);
if (has_nothing_to_do)
return {};
/// We can omit manual `progess` call, ISource will may count it automatically by returned chunk
/// However, we want to report only rows in progress
progress(1, 0);
if (!check_result.success)
{
LOG_WARNING(&Poco::Logger::get("InterpreterCheckQuery"),
"Check query for table {} failed, path {}, reason: {}",
table->getStorageID().getNameForLogs(),
check_result.fs_path,
check_result.failure_message);
}
return getChunkFromCheckResult(check_result);
}
private:
StoragePtr table;
IStorage::DataValidationTasksPtr check_data_tasks;
};
class TableCheckResultEmitter : public IAccumulatingTransform
{
public:
TableCheckResultEmitter() : IAccumulatingTransform(getHeaderForCheckResult(), getSingleValueBlock(1).cloneEmpty()) {}
String getName() const override { return "TableCheckResultEmitter"; }
void consume(Chunk chunk) override
{
if (result_value == 0)
return;
auto columns = chunk.getColumns();
if (columns.size() != 3)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong number of columns: {}", columns.size());
const auto * col = checkAndGetColumn<ColumnUInt8>(columns[1].get());
for (size_t i = 0; i < col->size(); ++i)
{
if (col->getElement(i) == 0)
{
result_value = 0;
return;
}
}
}
Chunk generate() override
{
if (is_value_emitted.exchange(true))
return {};
auto block = getSingleValueBlock(result_value);
return Chunk(block.getColumns(), block.rows());
}
private:
std::atomic<UInt8> result_value{1};
std::atomic_bool is_value_emitted{false};
};
}
InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_)
InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextPtr context_)
: WithContext(context_)
, query_ptr(query_ptr_)
{
}
BlockIO InterpreterCheckQuery::execute()
{
const auto & check = query_ptr->as<ASTCheckQuery &>();
auto table_id = getContext()->resolveStorageID(check, Context::ResolveOrdinary);
const auto & context = getContext();
auto table_id = context->resolveStorageID(check, Context::ResolveOrdinary);
getContext()->checkAccess(AccessType::SHOW_TABLES, table_id);
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext());
auto check_results = table->checkData(query_ptr, getContext());
context->checkAccess(AccessType::SHOW_TABLES, table_id);
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context);
Block block;
if (getContext()->getSettingsRef().check_query_single_value_result)
{
bool result = std::all_of(check_results.begin(), check_results.end(), [] (const CheckResult & res) { return res.success; });
auto column = ColumnUInt8::create();
column->insertValue(static_cast<UInt64>(result));
block = Block{{std::move(column), std::make_shared<DataTypeUInt8>(), "result"}};
}
else
{
auto block_structure = getBlockStructure();
auto path_column = block_structure[0].type->createColumn();
auto is_passed_column = block_structure[1].type->createColumn();
auto message_column = block_structure[2].type->createColumn();
auto check_data_tasks = table->getCheckTaskList(query_ptr, context);
for (const auto & check_result : check_results)
{
path_column->insert(check_result.fs_path);
is_passed_column->insert(static_cast<UInt8>(check_result.success));
message_column->insert(check_result.failure_message);
}
block = Block({
{std::move(path_column), block_structure[0].type, block_structure[0].name},
{std::move(is_passed_column), block_structure[1].type, block_structure[1].name},
{std::move(message_column), block_structure[2].type, block_structure[2].name}});
}
const auto & settings = context->getSettingsRef();
BlockIO res;
res.pipeline = QueryPipeline(std::make_shared<SourceFromSingleChunk>(std::move(block)));
{
auto processors = std::make_shared<Processors>();
std::vector<OutputPort *> worker_ports;
size_t num_streams = std::max<size_t>(1, settings.max_threads);
for (size_t i = 0; i < num_streams; ++i)
{
auto worker_processor = std::make_shared<TableCheckWorkerProcessor>(check_data_tasks, table);
if (i == 0)
worker_processor->addTotalRowsApprox(check_data_tasks->size());
worker_ports.emplace_back(&worker_processor->getPort());
processors->emplace_back(worker_processor);
}
OutputPort * resize_outport;
{
auto resize_processor = std::make_shared<ResizeProcessor>(getHeaderForCheckResult(), worker_ports.size(), 1);
auto & resize_inputs = resize_processor->getInputs();
auto resize_inport_it = resize_inputs.begin();
for (size_t i = 0; i < worker_ports.size(); ++i, ++resize_inport_it)
connect(*worker_ports[i], *resize_inport_it);
resize_outport = &resize_processor->getOutputs().front();
processors->emplace_back(resize_processor);
}
if (settings.check_query_single_value_result)
{
auto emitter_processor = std::make_shared<TableCheckResultEmitter>();
auto * input_port = &emitter_processor->getInputPort();
processors->emplace_back(emitter_processor);
connect(*resize_outport, *input_port);
}
res.pipeline = QueryPipeline(Pipe(std::move(processors)));
res.pipeline.setNumThreads(num_streams);
}
return res;
}

View File

@ -520,8 +520,6 @@ ContextMutablePtr Session::makeSessionContext()
{},
session_context->getSettingsRef().max_sessions_for_user);
recordLoginSucess(session_context);
return session_context;
}
@ -584,8 +582,6 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std:
{ session_name_ },
max_sessions_for_user);
recordLoginSucess(session_context);
return session_context;
}
@ -659,38 +655,24 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t
if (user_id)
user = query_context->getUser();
/// Interserver does not create session context
recordLoginSucess(query_context);
if (!notified_session_log_about_login)
{
if (auto session_log = getSessionLog())
{
session_log->addLoginSuccess(
auth_id,
named_session ? std::optional<std::string>(named_session->key.second) : std::nullopt,
*query_context,
user);
notified_session_log_about_login = true;
}
}
return query_context;
}
void Session::recordLoginSucess(ContextPtr login_context) const
{
if (notified_session_log_about_login)
return;
if (!login_context)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Session or query context must be created");
if (auto session_log = getSessionLog())
{
const auto & settings = login_context->getSettingsRef();
const auto access = login_context->getAccess();
session_log->addLoginSuccess(auth_id,
named_session ? named_session->key.second : "",
settings,
access,
getClientInfo(),
user);
notified_session_log_about_login = true;
}
}
void Session::releaseSessionID()
{
if (!named_session)

View File

@ -97,8 +97,6 @@ public:
private:
std::shared_ptr<SessionLog> getSessionLog() const;
ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const;
void recordLoginSucess(ContextPtr login_context) const;
mutable bool notified_session_log_about_login = false;
const UUID auth_id;

View File

@ -199,13 +199,12 @@ void SessionLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insertData(auth_failure_reason.data(), auth_failure_reason.length());
}
void SessionLog::addLoginSuccess(const UUID & auth_id,
const String & session_id,
const Settings & settings,
const ContextAccessPtr & access,
const ClientInfo & client_info,
const UserPtr & login_user)
void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional<String> session_id, const Context & login_context, const UserPtr & login_user)
{
const auto access = login_context.getAccess();
const auto & settings = login_context.getSettingsRef();
const auto & client_info = login_context.getClientInfo();
DB::SessionLogElement log_entry(auth_id, SESSION_LOGIN_SUCCESS);
log_entry.client_info = client_info;
@ -216,7 +215,8 @@ void SessionLog::addLoginSuccess(const UUID & auth_id,
}
log_entry.external_auth_server = login_user ? login_user->auth_data.getLDAPServerName() : "";
log_entry.session_id = session_id;
if (session_id)
log_entry.session_id = *session_id;
if (const auto roles_info = access->getRolesInfo())
log_entry.roles = roles_info->getCurrentRolesNames();

View File

@ -20,7 +20,6 @@ enum SessionLogElementType : int8_t
class ContextAccess;
struct User;
using UserPtr = std::shared_ptr<const User>;
using ContextAccessPtr = std::shared_ptr<const ContextAccess>;
/** A struct which will be inserted as row into session_log table.
*
@ -73,13 +72,7 @@ class SessionLog : public SystemLog<SessionLogElement>
using SystemLog<SessionLogElement>::SystemLog;
public:
void addLoginSuccess(const UUID & auth_id,
const String & session_id,
const Settings & settings,
const ContextAccessPtr & access,
const ClientInfo & client_info,
const UserPtr & login_user);
void addLoginSuccess(const UUID & auth_id, std::optional<String> session_id, const Context & login_context, const UserPtr & login_user);
void addLoginFailure(const UUID & auth_id, const ClientInfo & info, const std::optional<String> & user, const Exception & reason);
void addLogOut(const UUID & auth_id, const UserPtr & login_user, const ClientInfo & client_info);
};

View File

@ -561,7 +561,8 @@ void HTTPHandler::processQuery(
session->makeSessionContext();
}
auto context = session->makeQueryContext();
auto client_info = session->getClientInfo();
auto context = session->makeQueryContext(std::move(client_info));
/// This parameter is used to tune the behavior of output formats (such as Native) for compatibility.
if (params.has("client_protocol_version"))

View File

@ -22,6 +22,4 @@ struct CheckResult
{}
};
using CheckResults = std::vector<CheckResult>;
}

View File

@ -273,6 +273,17 @@ bool IStorage::isStaticStorage() const
return false;
}
IStorage::DataValidationTasksPtr IStorage::getCheckTaskList(const ASTPtr & /* query */, ContextPtr /* context */)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Check query is not supported for {} storage", getName());
}
CheckResult IStorage::checkDataNext(DataValidationTasksPtr & /* check_task_list */, bool & has_nothing_to_do)
{
has_nothing_to_do = true;
return {};
}
void IStorage::adjustCreateQueryForBackup(ASTPtr &) const
{
}

View File

@ -595,8 +595,45 @@ public:
/// Provides a hint that the storage engine may evaluate the IN-condition by using an index.
virtual bool mayBenefitFromIndexForIn(const ASTPtr & /* left_in_operand */, ContextPtr /* query_context */, const StorageMetadataPtr & /* metadata_snapshot */) const { return false; }
/// Checks validity of the data
virtual CheckResults checkData(const ASTPtr & /* query */, ContextPtr /* context */) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Check query is not supported for {} storage", getName()); }
/** A list of tasks to check a validity of data.
* Each IStorage implementation may interpret this task in its own way.
* E.g. for some storages it to check data it need to check a list of files in filesystem, for others it can be a list of parts.
* Also it may hold resources (e.g. locks) required during check.
*/
struct DataValidationTasksBase
{
/// Number of entries left to check.
/// It decreases after each call to checkDataNext().
virtual size_t size() const = 0;
virtual ~DataValidationTasksBase() = default;
};
using DataValidationTasksPtr = std::shared_ptr<DataValidationTasksBase>;
virtual DataValidationTasksPtr getCheckTaskList(const ASTPtr & /* query */, ContextPtr /* context */);
/** Executes one task from the list.
* If no tasks left, sets has_nothing_to_do to true.
* Note: Function `checkDataNext` is accessing `check_task_list` thread-safely,
* and can be called simultaneously for the same `getCheckTaskList` result
* to process different tasks in parallel.
* Usage:
*
* auto check_task_list = storage.getCheckTaskList(query, context);
* size_t total_tasks = check_task_list->size();
* while (true)
* {
* size_t tasks_left = check_task_list->size();
* std::cout << "Checking data: " << (total_tasks - tasks_left) << " / " << total_tasks << " tasks done." << std::endl;
* bool has_nothing_to_do = false;
* auto result = storage.checkDataNext(check_task_list, has_nothing_to_do);
* if (has_nothing_to_do)
* break;
* doSomething(result);
* }
*/
virtual CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do);
/// Checks that table could be dropped right now
/// Otherwise - throws an exception with detailed information.

View File

@ -256,10 +256,6 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet(
if (!key_columns.contains(name))
key_columns.insert(name);
ASTPtr ast_filter_node = buildFilterNode(query_info.query);
if (!ast_filter_node)
return;
if (context->getSettingsRef().allow_experimental_analyzer)
{
if (!query_info.filter_actions_dag)
@ -280,6 +276,10 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet(
}
else
{
ASTPtr ast_filter_node = buildFilterNode(query_info.query);
if (!ast_filter_node)
return;
if (checkASTUseless(ast_filter_node))
return;
@ -457,11 +457,10 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const Actio
if (arguments_size != 1)
return nullptr;
auto bit_wrapper_function = FunctionFactory::instance().get("__bitWrapperFunc", context);
const auto & bit_wrapper_func_node = result_dag->addFunction(bit_wrapper_function, {arguments[0]}, {});
const ActionsDAG::Node * argument = &traverseDAG(*arguments[0], result_dag, context, node_to_result_node);
auto bit_swap_last_two_function = FunctionFactory::instance().get("__bitSwapLastTwo", context);
return &result_dag->addFunction(bit_swap_last_two_function, {&bit_wrapper_func_node}, {});
return &result_dag->addFunction(bit_swap_last_two_function, {argument}, {});
}
else if (function_name == "and" || function_name == "indexHint" || function_name == "or")
{

View File

@ -396,9 +396,9 @@ std::unique_ptr<ReadBuffer> createReadBuffer(
throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP,
"Cannot compile regex from glob ({}): {}", current_path, matcher->error());
return reader->readFile([matcher = std::move(matcher)](const std::string & path)
return reader->readFile([my_matcher = std::move(matcher)](const std::string & path)
{
return re2::RE2::FullMatch(path, *matcher);
return re2::RE2::FullMatch(path, *my_matcher);
});
}
else

View File

@ -866,15 +866,18 @@ SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetada
return std::make_shared<LogSink>(*this, metadata_snapshot, std::move(lock));
}
CheckResults StorageLog::checkData(const ASTPtr & /* query */, ContextPtr local_context)
IStorage::DataValidationTasksPtr StorageLog::getCheckTaskList(const ASTPtr & /* query */, ContextPtr local_context)
{
ReadLock lock{rwlock, getLockTimeout(local_context)};
if (!lock)
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded");
return file_checker.check();
return std::make_unique<DataValidationTasks>(file_checker.getDataValidationTasks(), std::move(lock));
}
CheckResult StorageLog::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do)
{
return file_checker.checkNextEntry(assert_cast<DataValidationTasks *>(check_task_list.get())->file_checker_tasks, has_nothing_to_do);
}
IStorage::ColumnSizeByName StorageLog::getColumnSizes() const
{

View File

@ -59,7 +59,8 @@ public:
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
CheckResults checkData(const ASTPtr & query, ContextPtr local_context) override;
DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override;
CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override;
void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override;
@ -142,6 +143,19 @@ private:
std::atomic<UInt64> total_rows = 0;
std::atomic<UInt64> total_bytes = 0;
struct DataValidationTasks : public IStorage::DataValidationTasksBase
{
DataValidationTasks(FileChecker::DataValidationTasksPtr file_checker_tasks_, ReadLock && lock_)
: file_checker_tasks(std::move(file_checker_tasks_)), lock(std::move(lock_))
{}
size_t size() const override { return file_checker_tasks->size(); }
FileChecker::DataValidationTasksPtr file_checker_tasks;
/// Lock to prevent table modification while checking
ReadLock lock;
};
FileChecker file_checker;
const size_t max_compress_block_size;

View File

@ -2197,9 +2197,8 @@ void StorageMergeTree::onActionLockRemove(StorageActionBlockType action_type)
background_moves_assignee.trigger();
}
CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_context)
IStorage::DataValidationTasksPtr StorageMergeTree::getCheckTaskList(const ASTPtr & query, ContextPtr local_context)
{
CheckResults results;
DataPartsVector data_parts;
if (const auto & check_query = query->as<ASTCheckQuery &>(); check_query.partition)
{
@ -2209,7 +2208,14 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_
else
data_parts = getVisibleDataPartsVector(local_context);
for (auto & part : data_parts)
return std::make_unique<DataValidationTasks>(std::move(data_parts), local_context);
}
CheckResult StorageMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do)
{
auto * data_validation_tasks = assert_cast<DataValidationTasks *>(check_task_list.get());
auto local_context = data_validation_tasks->context;
if (auto part = data_validation_tasks->next())
{
/// If the checksums file is not present, calculate the checksums and write them to disk.
static constexpr auto checksums_path = "checksums.txt";
@ -2224,12 +2230,12 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_
part_mutable.writeChecksums(part->checksums, local_context->getWriteSettings());
part->checkMetadata();
results.emplace_back(part->name, true, "Checksums recounted and written to disk.");
return CheckResult(part->name, true, "Checksums recounted and written to disk.");
}
catch (const Exception & ex)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);
results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'");
return CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'");
}
}
else
@ -2238,15 +2244,19 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_
{
checkDataPart(part, true);
part->checkMetadata();
results.emplace_back(part->name, true, "");
return CheckResult(part->name, true, "");
}
catch (const Exception & ex)
{
results.emplace_back(part->name, false, ex.message());
return CheckResult(part->name, false, ex.message());
}
}
}
return results;
else
{
has_nothing_to_do = true;
return {};
}
}

View File

@ -108,7 +108,8 @@ public:
void onActionLockRemove(StorageActionBlockType action_type) override;
CheckResults checkData(const ASTPtr & query, ContextPtr context) override;
DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override;
CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override;
bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override;
@ -278,6 +279,32 @@ private:
friend class MergePlainMergeTreeTask;
friend class MutatePlainMergeTreeTask;
struct DataValidationTasks : public IStorage::DataValidationTasksBase
{
DataValidationTasks(DataPartsVector && parts_, ContextPtr context_)
: parts(std::move(parts_)), it(parts.begin()), context(std::move(context_))
{}
DataPartPtr next()
{
std::lock_guard lock(mutex);
if (it == parts.end())
return nullptr;
return *(it++);
}
size_t size() const override
{
std::lock_guard lock(mutex);
return std::distance(it, parts.end());
}
mutable std::mutex mutex;
DataPartsVector parts;
DataPartsVector::const_iterator it;
ContextPtr context;
};
protected:
std::map<int64_t, MutationCommands> getAlterMutationCommandsForPart(const DataPartPtr & part) const override;

View File

@ -149,7 +149,9 @@ public:
return getNested()->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot);
}
CheckResults checkData(const ASTPtr & query, ContextPtr context) override { return getNested()->checkData(query, context); }
DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override { return getNested()->getCheckTaskList(query, context); }
CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override { return getNested()->checkDataNext(check_task_list, has_nothing_to_do); }
void checkTableCanBeDropped() const override { getNested()->checkTableCanBeDropped(); }
bool storesDataOnDisk() const override { return getNested()->storesDataOnDisk(); }
Strings getDataPaths() const override { return getNested()->getDataPaths(); }

View File

@ -8481,9 +8481,8 @@ void StorageReplicatedMergeTree::enqueuePartForCheck(const String & part_name, t
part_check_thread.enqueuePart(part_name, delay_to_check_seconds);
}
CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, ContextPtr local_context)
IStorage::DataValidationTasksPtr StorageReplicatedMergeTree::getCheckTaskList(const ASTPtr & query, ContextPtr local_context)
{
CheckResults results;
DataPartsVector data_parts;
if (const auto & check_query = query->as<ASTCheckQuery &>(); check_query.partition)
{
@ -8493,24 +8492,30 @@ CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, Context
else
data_parts = getVisibleDataPartsVector(local_context);
{
auto part_check_lock = part_check_thread.pausePartsCheck();
auto part_check_lock = part_check_thread.pausePartsCheck();
return std::make_unique<DataValidationTasks>(std::move(data_parts), std::move(part_check_lock));
}
for (auto & part : data_parts)
CheckResult StorageReplicatedMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do)
{
if (auto part = assert_cast<DataValidationTasks *>(check_task_list.get())->next())
{
try
{
try
{
results.push_back(part_check_thread.checkPartAndFix(part->name));
}
catch (const Exception & ex)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);
results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'");
}
return CheckResult(part_check_thread.checkPartAndFix(part->name));
}
catch (const Exception & ex)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);
return CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'");
}
}
return results;
else
{
has_nothing_to_do = true;
return {};
}
}

View File

@ -230,7 +230,8 @@ public:
/// Add a part to the queue of parts whose data you want to check in the background thread.
void enqueuePartForCheck(const String & part_name, time_t delay_to_check_seconds = 0);
CheckResults checkData(const ASTPtr & query, ContextPtr context) override;
DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override;
CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override;
/// Checks ability to use granularity
bool canUseAdaptiveGranularity() const override;
@ -990,6 +991,34 @@ private:
bool waitZeroCopyLockToDisappear(const ZeroCopyLock & lock, size_t milliseconds_to_wait) override;
void startupImpl(bool from_attach_thread);
struct DataValidationTasks : public IStorage::DataValidationTasksBase
{
explicit DataValidationTasks(DataPartsVector && parts_, std::unique_lock<std::mutex> && parts_check_lock_)
: parts_check_lock(std::move(parts_check_lock_)), parts(std::move(parts_)), it(parts.begin())
{}
DataPartPtr next()
{
std::lock_guard lock(mutex);
if (it == parts.end())
return nullptr;
return *(it++);
}
size_t size() const override
{
std::lock_guard lock(mutex);
return std::distance(it, parts.end());
}
std::unique_lock<std::mutex> parts_check_lock;
mutable std::mutex mutex;
DataPartsVector parts;
DataPartsVector::const_iterator it;
};
};
String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info);

View File

@ -403,16 +403,18 @@ SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const Storage
return std::make_shared<StripeLogSink>(*this, metadata_snapshot, std::move(lock));
}
CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, ContextPtr local_context)
IStorage::DataValidationTasksPtr StorageStripeLog::getCheckTaskList(const ASTPtr & /* query */, ContextPtr local_context)
{
ReadLock lock{rwlock, getLockTimeout(local_context)};
if (!lock)
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded");
return file_checker.check();
return std::make_unique<DataValidationTasks>(file_checker.getDataValidationTasks(), std::move(lock));
}
CheckResult StorageStripeLog::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do)
{
return file_checker.checkNextEntry(assert_cast<DataValidationTasks *>(check_task_list.get())->file_checker_tasks, has_nothing_to_do);
}
void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &)
{

View File

@ -53,7 +53,8 @@ public:
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
CheckResults checkData(const ASTPtr & query, ContextPtr ocal_context) override;
DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override;
CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override;
bool storesDataOnDisk() const override { return true; }
Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; }
@ -93,6 +94,20 @@ private:
const DiskPtr disk;
String table_path;
struct DataValidationTasks : public IStorage::DataValidationTasksBase
{
DataValidationTasks(FileChecker::DataValidationTasksPtr file_checker_tasks_, ReadLock && lock_)
: file_checker_tasks(std::move(file_checker_tasks_)), lock(std::move(lock_))
{}
size_t size() const override { return file_checker_tasks->size(); }
FileChecker::DataValidationTasksPtr file_checker_tasks;
/// Lock to prevent table modification while checking
ReadLock lock;
};
String data_file_path;
String index_file_path;
FileChecker file_checker;

View File

@ -6,7 +6,6 @@
00725_memory_tracking
00754_distributed_optimize_skip_select_on_unused_shards
00754_distributed_optimize_skip_select_on_unused_shards_with_prewhere
00838_unique_index
00927_asof_joins
00940_order_by_read_in_order_query_plan
00945_bloom_filter_index

View File

@ -1,12 +1,13 @@
#!/usr/bin/env python3
from pathlib import Path
from typing import List, Tuple
import subprocess
import logging
import json
import os
import sys
import time
from typing import List, Tuple
from ci_config import CI_CONFIG, BuildConfig
from docker_pull_helper import get_image_with_version
@ -18,18 +19,19 @@ from env_helper import (
S3_DOWNLOAD,
TEMP_PATH,
)
from git_helper import Git, git_runner
from pr_info import PRInfo
from s3_helper import S3Helper
from tee_popen import TeePopen
from version_helper import (
ClickHouseVersion,
Git,
get_version_from_repo,
update_version_local,
)
from clickhouse_helper import (
ClickHouseHelper,
prepare_tests_results_for_clickhouse,
get_instance_type,
)
from stopwatch import Stopwatch
@ -50,7 +52,7 @@ def _can_export_binaries(build_config: BuildConfig) -> bool:
def get_packager_cmd(
build_config: BuildConfig,
packager_path: str,
output_path: str,
output_path: Path,
build_version: str,
image_version: str,
official: bool,
@ -59,8 +61,8 @@ def get_packager_cmd(
comp = build_config.compiler
cmake_flags = "-DENABLE_CLICKHOUSE_SELF_EXTRACTING=1"
cmd = (
f"cd {packager_path} && CMAKE_FLAGS='{cmake_flags}' ./packager --output-dir={output_path} "
f"--package-type={package_type} --compiler={comp}"
f"cd {packager_path} && CMAKE_FLAGS='{cmake_flags}' ./packager "
f"--output-dir={output_path} --package-type={package_type} --compiler={comp}"
)
if build_config.debug_build:
@ -78,6 +80,7 @@ def get_packager_cmd(
cmd += " --additional-pkgs"
cmd += f" --docker-image-version={image_version}"
cmd += " --with-profiler"
cmd += f" --version={build_version}"
if _can_export_binaries(build_config):
@ -90,13 +93,13 @@ def get_packager_cmd(
def build_clickhouse(
packager_cmd: str, logs_path: str, build_output_path: str
) -> Tuple[str, bool]:
build_log_path = os.path.join(logs_path, BUILD_LOG_NAME)
packager_cmd: str, logs_path: Path, build_output_path: Path
) -> Tuple[Path, bool]:
build_log_path = logs_path / BUILD_LOG_NAME
success = False
with TeePopen(packager_cmd, build_log_path) as process:
retcode = process.wait()
if os.path.exists(build_output_path):
if build_output_path.exists():
build_results = os.listdir(build_output_path)
else:
build_results = []
@ -217,7 +220,7 @@ def upload_master_static_binaries(
pr_info: PRInfo,
build_config: BuildConfig,
s3_helper: S3Helper,
build_output_path: str,
build_output_path: Path,
) -> None:
"""Upload binary artifacts to a static S3 links"""
static_binary_name = build_config.static_binary_name
@ -229,7 +232,7 @@ def upload_master_static_binaries(
return
s3_path = "/".join((pr_info.base_ref, static_binary_name, "clickhouse"))
binary = os.path.join(build_output_path, "clickhouse")
binary = build_output_path / "clickhouse"
url = s3_helper.upload_build_file_to_s3(binary, s3_path)
print(f"::notice ::Binary static URL: {url}")
@ -242,8 +245,8 @@ def main():
build_config = CI_CONFIG.build_config[build_name]
if not os.path.exists(TEMP_PATH):
os.makedirs(TEMP_PATH)
temp_path = Path(TEMP_PATH)
os.makedirs(temp_path, exist_ok=True)
pr_info = PRInfo()
@ -282,9 +285,8 @@ def main():
logging.info("Build short name %s", build_name)
build_output_path = os.path.join(TEMP_PATH, build_name)
if not os.path.exists(build_output_path):
os.makedirs(build_output_path)
build_output_path = temp_path / build_name
os.makedirs(build_output_path, exist_ok=True)
packager_cmd = get_packager_cmd(
build_config,
@ -297,9 +299,8 @@ def main():
logging.info("Going to run packager with %s", packager_cmd)
logs_path = os.path.join(TEMP_PATH, "build_log")
if not os.path.exists(logs_path):
os.makedirs(logs_path)
logs_path = temp_path / "build_log"
os.makedirs(logs_path, exist_ok=True)
start = time.time()
log_path, success = build_clickhouse(packager_cmd, logs_path, build_output_path)
@ -320,8 +321,8 @@ def main():
# FIXME performance
performance_urls = []
performance_path = os.path.join(build_output_path, "performance.tar.zst")
if os.path.exists(performance_path):
performance_path = build_output_path / "performance.tar.zst"
if performance_path.exists():
performance_urls.append(
s3_helper.upload_build_file_to_s3(performance_path, s3_performance_path)
)
@ -344,9 +345,9 @@ def main():
print("::notice ::Build URLs: {}".format("\n".join(build_urls)))
if os.path.exists(log_path):
if log_path.exists():
log_url = s3_helper.upload_build_file_to_s3(
log_path, s3_path_prefix + "/" + os.path.basename(log_path)
log_path, s3_path_prefix + "/" + log_path.name
)
logging.info("Log url %s", log_url)
else:
@ -360,7 +361,79 @@ def main():
upload_master_static_binaries(pr_info, build_config, s3_helper, build_output_path)
# Upload profile data
ch_helper = ClickHouseHelper()
clickhouse_ci_logs_host = os.getenv("CLICKHOUSE_CI_LOGS_HOST", "")
if clickhouse_ci_logs_host:
instance_type = get_instance_type()
query = f"""INSERT INTO build_time_trace
(
pull_request_number,
commit_sha,
check_start_time,
check_name,
instance_type,
file,
library,
time,
pid,
tid,
ph,
ts,
dur,
cat,
name,
detail,
count,
avgMs,
args_name
)
SELECT {pr_info.number}, '{pr_info.sha}', '{stopwatch.start_time_str}', '{build_name}', '{instance_type}', *
FROM input('
file String,
library String,
time DateTime64(6),
pid UInt32,
tid UInt32,
ph String,
ts UInt64,
dur UInt64,
cat String,
name String,
detail String,
count UInt64,
avgMs UInt64,
args_name String')
FORMAT JSONCompactEachRow"""
auth = {
"X-ClickHouse-User": "ci",
"X-ClickHouse-Key": os.getenv("CLICKHOUSE_CI_LOGS_PASSWORD", ""),
}
url = f"https://{clickhouse_ci_logs_host}/"
profiles_dir = temp_path / "profiles_source"
os.makedirs(profiles_dir, exist_ok=True)
logging.info("Processing profile JSON files from {GIT_REPO_ROOT}/build_docker")
git_runner(
"./utils/prepare-time-trace/prepare-time-trace.sh "
f"build_docker {profiles_dir.absolute()}"
)
profile_data_file = temp_path / "profile.json"
with open(profile_data_file, "wb") as profile_fd:
for profile_sourse in os.listdir(profiles_dir):
with open(profiles_dir / profile_sourse, "rb") as ps_fd:
profile_fd.write(ps_fd.read())
logging.info(
"::notice ::Log Uploading profile data, path: %s, size: %s, query: %s",
profile_data_file,
profile_data_file.stat().st_size,
query,
)
ch_helper.insert_file(url, auth, query, profile_data_file)
# Upload statistics to CI database
prepared_events = prepare_tests_results_for_clickhouse(
pr_info,
[],

View File

@ -1,5 +1,6 @@
#!/usr/bin/env python3
from typing import List
from pathlib import Path
from typing import Dict, List, Optional
import json
import logging
import time
@ -16,30 +17,60 @@ class InsertException(Exception):
class ClickHouseHelper:
def __init__(self, url=None):
def __init__(
self, url: Optional[str] = None, auth: Optional[Dict[str, str]] = None
):
if url is None:
url = get_parameter_from_ssm("clickhouse-test-stat-url")
self.url = url
self.auth = {
self.auth = auth or {
"X-ClickHouse-User": get_parameter_from_ssm("clickhouse-test-stat-login"),
"X-ClickHouse-Key": get_parameter_from_ssm("clickhouse-test-stat-password"),
}
@staticmethod
def _insert_json_str_info_impl(url, auth, db, table, json_str):
def insert_file(
url: str,
auth: Optional[Dict[str, str]],
query: str,
file: Path,
additional_options: Optional[Dict[str, str]] = None,
) -> None:
params = {
"query": query,
"date_time_input_format": "best_effort",
"send_logs_level": "warning",
}
if additional_options:
for k, v in additional_options.items():
params[k] = v
with open(file, "rb") as data_fd:
ClickHouseHelper._insert_post(
url, params=params, data=data_fd, headers=auth
)
@staticmethod
def insert_json_str(url, auth, db, table, json_str):
params = {
"database": db,
"query": f"INSERT INTO {table} FORMAT JSONEachRow",
"date_time_input_format": "best_effort",
"send_logs_level": "warning",
}
ClickHouseHelper._insert_post(url, params=params, data=json_str, headers=auth)
@staticmethod
def _insert_post(*args, **kwargs):
url = ""
if args:
url = args[0]
url = kwargs.get("url", url)
for i in range(5):
try:
response = requests.post(
url, params=params, data=json_str, headers=auth
)
response = requests.post(*args, **kwargs)
except Exception as e:
error = f"Received exception while sending data to {url} on {i} attempt: {e}"
logging.warning(error)
@ -51,13 +82,8 @@ class ClickHouseHelper:
break
error = (
"Cannot insert data into clickhouse at try "
+ str(i)
+ ": HTTP code "
+ str(response.status_code)
+ ": '"
+ str(response.text)
+ "'"
f"Cannot insert data into clickhouse at try {i}: HTTP code "
f"{response.status_code}: '{response.text}'"
)
if response.status_code >= 500:
@ -76,7 +102,7 @@ class ClickHouseHelper:
raise InsertException(error)
def _insert_json_str_info(self, db, table, json_str):
self._insert_json_str_info_impl(self.url, self.auth, db, table, json_str)
self.insert_json_str(self.url, self.auth, db, table, json_str)
def insert_event_into(self, db, table, event, safe=True):
event_str = json.dumps(event)

View File

@ -595,24 +595,3 @@ def execute_task(started_cluster, task, cmd_options):
task.check()
finally:
zk.delete(task.zk_task_path, recursive=True)
# Tests
@pytest.mark.skip(reason="Too flaky :(")
def test_different_schema(started_cluster):
execute_task(started_cluster, TaskWithDifferentSchema(started_cluster), [])
@pytest.mark.skip(reason="Too flaky :(")
def test_ttl_columns(started_cluster):
execute_task(started_cluster, TaskTTL(started_cluster), [])
@pytest.mark.skip(reason="Too flaky :(")
def test_skip_index(started_cluster):
execute_task(started_cluster, TaskSkipIndex(started_cluster), [])
@pytest.mark.skip(reason="Too flaky :(")
def test_ttl_move_to_volume(started_cluster):
execute_task(started_cluster, TaskTTLMoveToVolume(started_cluster), [])

View File

@ -1,99 +0,0 @@
import os
import math
import pytest
# FIXME This test is too flaky
# https://github.com/ClickHouse/ClickHouse/issues/33006
pytestmark = pytest.mark.skip
from .common import *
from helpers.cluster import ClickHouseCluster
from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout
from helpers.external_sources import SourceCassandra
SOURCE = None
cluster = None
node = None
simple_tester = None
complex_tester = None
ranged_tester = None
test_name = "cassandra"
def setup_module(module):
global cluster
global node
global simple_tester
global complex_tester
global ranged_tester
cluster = ClickHouseCluster(__file__)
SOURCE = SourceCassandra(
"Cassandra",
None,
cluster.cassandra_port,
cluster.cassandra_host,
cluster.cassandra_port,
"",
"",
)
simple_tester = SimpleLayoutTester(test_name)
simple_tester.cleanup()
simple_tester.create_dictionaries(SOURCE)
complex_tester = ComplexLayoutTester(test_name)
complex_tester.create_dictionaries(SOURCE)
ranged_tester = RangedLayoutTester(test_name)
ranged_tester.create_dictionaries(SOURCE)
# Since that all .xml configs were created
main_configs = []
main_configs.append(os.path.join("configs", "disable_ssl_verification.xml"))
dictionaries = simple_tester.list_dictionaries()
node = cluster.add_instance(
"cass_node",
main_configs=main_configs,
dictionaries=dictionaries,
with_cassandra=True,
)
def teardown_module(module):
simple_tester.cleanup()
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
simple_tester.prepare(cluster)
complex_tester.prepare(cluster)
ranged_tester.prepare(cluster)
yield cluster
finally:
cluster.shutdown()
@pytest.mark.parametrize("layout_name", sorted(LAYOUTS_SIMPLE))
def test_simple(started_cluster, layout_name):
simple_tester.execute(layout_name, node)
@pytest.mark.parametrize("layout_name", sorted(LAYOUTS_COMPLEX))
def test_complex(started_cluster, layout_name):
complex_tester.execute(layout_name, node)
@pytest.mark.parametrize("layout_name", sorted(LAYOUTS_RANGED))
def test_ranged(started_cluster, layout_name):
ranged_tester.execute(layout_name, node)

View File

@ -27,71 +27,3 @@ def started_cluster():
yield cluster
finally:
cluster.shutdown()
@pytest.mark.skip(reason="SSD cache test can run on disk only")
@pytest.mark.parametrize("type", ["memory", "ssd"])
def test_memory_consumption(started_cluster, type):
node = started_cluster.instances[f"node_{type}"]
node.query(
"insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format(
"w" * 8
)
)
node.query(
"insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format(
"x" * 16
)
)
node.query(
"insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format(
"y" * 32
)
)
node.query(
"insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format(
"z" * 64
)
)
# Fill dictionary
node.query(
"select dictGetString('radars', 'client_id', tuple(toString(number))) from numbers(0, 5000)"
)
allocated_first = int(
node.query(
"select bytes_allocated from system.dictionaries where name = 'radars'"
).strip()
)
alloc_array = []
for i in range(5):
node.query(
"select dictGetString('radars', 'client_id', tuple(toString(number))) from numbers(0, 5000)"
)
allocated = int(
node.query(
"select bytes_allocated from system.dictionaries where name = 'radars'"
).strip()
)
alloc_array.append(allocated)
# size doesn't grow
assert all(allocated_first >= a for a in alloc_array)
for i in range(5):
node.query(
"select dictGetString('radars', 'client_id', tuple(toString(number))) from numbers(0, 5000)"
)
allocated = int(
node.query(
"select bytes_allocated from system.dictionaries where name = 'radars'"
).strip()
)
alloc_array.append(allocated)
# size doesn't grow
assert all(allocated_first >= a for a in alloc_array)

View File

@ -38,7 +38,6 @@ def started_cluster():
cluster.shutdown()
# @pytest.mark.skip(reason="debugging")
def test_default_reading(started_cluster):
assert None != dictionary_node.get_process_pid(
"clickhouse"

View File

@ -53,7 +53,6 @@ def started_cluster():
cluster.shutdown()
# @pytest.mark.skip(reason="debugging")
def test_return_real_values(started_cluster):
assert None != dictionary_node.get_process_pid(
"clickhouse"

View File

@ -38,7 +38,6 @@ def started_cluster():
cluster.shutdown()
# @pytest.mark.skip(reason="debugging")
def test_simple_dict_get(started_cluster):
assert None != dictionary_node.get_process_pid(
"clickhouse"

View File

@ -38,7 +38,6 @@ def started_cluster():
cluster.shutdown()
# @pytest.mark.skip(reason="debugging")
def test_simple_dict_get_or_default(started_cluster):
assert None != dictionary_node.get_process_pid(
"clickhouse"

View File

@ -71,73 +71,3 @@ def test_deduplication_window_in_seconds(started_cluster):
assert TSV.toMat(node.query("SELECT count() FROM simple"))[0][0] == "3"
node1.query("""DROP TABLE simple ON CLUSTER test_cluster""")
# Currently this test just reproduce incorrect behavior that sould be fixed
@pytest.mark.skip(reason="Flapping test")
def test_deduplication_works_in_case_of_intensive_inserts(started_cluster):
inserters = []
fetchers = []
node1.query(
"""
CREATE TABLE simple ON CLUSTER test_cluster (date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id"""
)
node1.query("INSERT INTO simple VALUES (0, 0)")
for node in nodes:
host = node.ip_address
inserters.append(
CommandRequest(
["/bin/bash"],
timeout=10,
stdin="""
set -e
for i in `seq 1000`; do
{} --host {} -q "INSERT INTO simple VALUES (0, 0)"
done
""".format(
cluster.get_client_cmd(), host
),
)
)
fetchers.append(
CommandRequest(
["/bin/bash"],
timeout=10,
stdin="""
set -e
for i in `seq 1000`; do
res=`{} --host {} -q "SELECT count() FROM simple"`
if [[ $? -ne 0 || $res -ne 1 ]]; then
echo "Selected $res elements! Host: {}" 1>&2
exit -1
fi;
done
""".format(
cluster.get_client_cmd(), host, node.name
),
)
)
# There were not errors during INSERTs
for inserter in inserters:
try:
inserter.get_answer()
except QueryTimeoutExceedException:
# Only timeout is accepted
pass
# There were not errors during SELECTs
for fetcher in fetchers:
try:
fetcher.get_answer()
except QueryTimeoutExceedException:
# Only timeout is accepted
pass
node1.query("""DROP TABLE simple ON CLUSTER test_cluster""")

View File

@ -1,32 +0,0 @@
<clickhouse>
<remote_servers >
<!-- Test only shard config for testing distributed storage -->
<simple>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
</simple>
</remote_servers>
<local_cache_for_remote_fs>
<enable>true</enable>
<root_dir>/tmp/clickhouse_local_cache,/tmp/clickhouse_local_cache1</root_dir>
<limit_size>207374182400</limit_size>
<bytes_read_befor_flush>1048576</bytes_read_befor_flush>
</local_cache_for_remote_fs>
<hdfs>
<libhdfs3_conf>/etc/clickhouse-server/extra_conf.d/hdfs-site.xml</libhdfs3_conf>
</hdfs>
<metric_log>
<database>system</database>
<table>metric_log</table>
<flush_interval_milliseconds>1000</flush_interval_milliseconds>
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
</metric_log>
</clickhouse>

View File

@ -1,6 +0,0 @@
<configuration>
<property>
<name>dfs.replication</name>
<value>1</value>
</property>
</configuration>

View File

@ -1,15 +0,0 @@
#!/bin/bash
hive -e "create database test"
hive -e "drop table if exists test.demo; create table test.demo(id string, score int) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'; create table test.demo_orc(id string, score int) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'; "
hive -e "drop table if exists test.parquet_demo; create table test.parquet_demo(id string, score int) PARTITIONED BY(day string, hour string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'"
hive -e "drop table if exists test.demo_text; create table test.demo_text(id string, score int, day string)row format delimited fields terminated by ','; load data local inpath '/demo_data.txt' into table test.demo_text "
hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.demo partition(day) select * from test.demo_text; insert into test.demo_orc partition(day) select * from test.demo_text"
hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '00' as hour from test.demo;"
hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '01' as hour from test.demo;"
hive -e "drop table if exists test.test_hive_types; CREATE TABLE test.test_hive_types( f_tinyint tinyint, f_smallint smallint, f_int int, f_integer int, f_bigint bigint, f_float float, f_double double, f_decimal decimal(10,0), f_timestamp timestamp, f_date date, f_string string, f_varchar varchar(100), f_char char(100), f_bool boolean, f_array_int array<int>, f_array_string array<string>, f_array_float array<float>, f_map_int map<string, int>, f_map_string map<string, string>, f_map_float map<string, float>, f_struct struct<a:string, b:int, c:float, d: struct<x:int, y:string>>) PARTITIONED BY( day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat';"
hive -e "insert into test.test_hive_types partition(day='2022-02-20') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, '2022-02-20 14:47:04', '2022-02-20', 'hello world', 'hello world', 'hello world', true, array(1,2,3), array('hello world', 'hello world'), array(float(1.1),float(1.2)), map('a', 100, 'b', 200, 'c', 300), map('a', 'aa', 'b', 'bb', 'c', 'cc'), map('a', float(111.1), 'b', float(222.2), 'c', float(333.3)), named_struct('a', 'aaa', 'b', 200, 'c', float(333.3), 'd', named_struct('x', 10, 'y', 'xyz')); insert into test.test_hive_types partition(day='2022-02-19') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, '2022-02-19 14:47:04', '2022-02-19', 'hello world', 'hello world', 'hello world', true, array(1,2,3), array('hello world', 'hello world'), array(float(1.1),float(1.2)), map('a', 100, 'b', 200, 'c', 300), map('a', 'aa', 'b', 'bb', 'c', 'cc'), map('a', float(111.1), 'b', float(222.2), 'c', float(333.3)), named_struct('a', 'aaa', 'b', 200, 'c', float(333.3), 'd', named_struct('x', 11, 'y', 'abc'));"

View File

@ -1,498 +0,0 @@
import pytest
# FIXME This test is too flaky
# https://github.com/ClickHouse/ClickHouse/issues/43541
pytestmark = pytest.mark.skip
import logging
import os
import time
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
logging.getLogger().setLevel(logging.INFO)
logging.getLogger().addHandler(logging.StreamHandler())
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster = ClickHouseCluster(__file__)
cluster.add_instance(
"h0_0_0",
main_configs=["configs/config.xml"],
extra_configs=["configs/hdfs-site.xml", "data/prepare_hive_data.sh"],
with_hive=True,
)
logging.info("Starting cluster ...")
cluster.start()
cluster.copy_file_to_container(
"roottesthivequery_hdfs1_1",
"/ClickHouse/tests/integration/test_hive_query/data/prepare_hive_data.sh",
"/prepare_hive_data.sh",
)
cluster.exec_in_container(
"roottesthivequery_hdfs1_1", ["bash", "-c", "bash /prepare_hive_data.sh"]
)
yield cluster
finally:
cluster.shutdown()
def test_create_parquet_table(started_cluster):
logging.info("Start testing creating hive table ...")
node = started_cluster.instances["h0_0_0"]
test_passed = False
for i in range(10):
node.query("set input_format_parquet_allow_missing_columns = true")
result = node.query(
"""
DROP TABLE IF EXISTS default.demo_parquet;
CREATE TABLE default.demo_parquet (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day)
"""
)
logging.info("create result {}".format(result))
if result.strip() == "":
test_passed = True
break
time.sleep(60)
assert test_passed
def test_create_parquet_table_1(started_cluster):
logging.info("Start testing creating hive table ...")
node = started_cluster.instances["h0_0_0"]
for i in range(10):
node.query("set input_format_parquet_allow_missing_columns = true")
result = node.query(
"""
DROP TABLE IF EXISTS default.demo_parquet_parts;
CREATE TABLE default.demo_parquet_parts (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String), `hour` String) ENGINE = Hive('thrift://hivetest:9083', 'test', 'parquet_demo') PARTITION BY(day, hour);
"""
)
logging.info("create result {}".format(result))
if result.strip() == "":
test_passed = True
break
time.sleep(60)
assert test_passed
def test_create_orc_table(started_cluster):
logging.info("Start testing creating hive table ...")
node = started_cluster.instances["h0_0_0"]
test_passed = False
for i in range(10):
result = node.query(
"""
DROP TABLE IF EXISTS default.demo_orc;
CREATE TABLE default.demo_orc (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day)
"""
)
logging.info("create result {}".format(result))
if result.strip() == "":
test_passed = True
break
time.sleep(60)
assert test_passed
def test_create_text_table(started_cluster):
logging.info("Start testing creating hive table ...")
node = started_cluster.instances["h0_0_0"]
result = node.query(
"""
DROP TABLE IF EXISTS default.demo_text;
CREATE TABLE default.demo_text (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_text') PARTITION BY (tuple())
"""
)
logging.info("create result {}".format(result))
assert result.strip() == ""
def test_parquet_groupby(started_cluster):
logging.info("Start testing groupby ...")
node = started_cluster.instances["h0_0_0"]
result = node.query(
"""
SELECT day, count(*) FROM default.demo_parquet group by day order by day
"""
)
expected_result = """2021-11-01 1
2021-11-05 2
2021-11-11 1
2021-11-16 2
"""
assert result == expected_result
def test_parquet_in_filter(started_cluster):
logging.info("Start testing groupby ...")
node = started_cluster.instances["h0_0_0"]
result = node.query(
"""
SELECT count(*) FROM default.demo_parquet_parts where day = '2021-11-05' and hour in ('00')
"""
)
expected_result = """2
"""
logging.info("query result:{}".format(result))
assert result == expected_result
def test_orc_groupby(started_cluster):
logging.info("Start testing groupby ...")
node = started_cluster.instances["h0_0_0"]
result = node.query(
"""
DROP TABLE IF EXISTS default.demo_orc;
CREATE TABLE default.demo_orc (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day);
SELECT day, count(*) FROM default.demo_orc group by day order by day
"""
)
expected_result = """2021-11-01 1
2021-11-05 2
2021-11-11 1
2021-11-16 2
"""
assert result == expected_result
@pytest.mark.parametrize(
"table,use_local_cache_for_remote_storage,enable_orc_file_minmax_index,enable_orc_stripe_minmax_index",
[
pytest.param(
"demo_orc_no_cache_no_index",
"false",
"false",
"false",
id="demo_orc_no_cache_no_index",
),
pytest.param(
"demo_orc_with_cache_no_index",
"true",
"false",
"false",
id="demo_orc_with_cache_no_index",
),
pytest.param(
"demo_orc_no_cache_file_index",
"false",
"true",
"false",
id="demo_orc_no_cache_file_index",
),
pytest.param(
"demo_orc_with_cache_file_index",
"true",
"true",
"false",
id="demo_orc_with_cache_file_index",
),
pytest.param(
"demo_orc_no_cache_stripe_index",
"false",
"true",
"true",
id="demo_orc_no_cache_stripe_index",
),
pytest.param(
"demo_orc_with_cache_stripe_index",
"true",
"true",
"true",
id="demo_orc_with_cache_stripe_index",
),
],
)
def test_orc_minmax_index(
started_cluster,
table,
use_local_cache_for_remote_storage,
enable_orc_file_minmax_index,
enable_orc_stripe_minmax_index,
):
node = started_cluster.instances["h0_0_0"]
result = node.query(
"""
DROP TABLE IF EXISTS default.{table};
CREATE TABLE default.{table} (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day)
SETTINGS enable_orc_file_minmax_index = {enable_orc_file_minmax_index}, enable_orc_stripe_minmax_index = {enable_orc_stripe_minmax_index};
""".format(
table=table,
enable_orc_file_minmax_index=enable_orc_file_minmax_index,
enable_orc_stripe_minmax_index=enable_orc_stripe_minmax_index,
)
)
assert result.strip() == ""
for i in range(2):
result = node.query(
"""
SELECT day, id, score FROM default.{table} where day >= '2021-11-05' and day <= '2021-11-16' and score >= 15 and score <= 30 order by day, id
SETTINGS use_local_cache_for_remote_storage = {use_local_cache_for_remote_storage}
""".format(
table=table,
use_local_cache_for_remote_storage=use_local_cache_for_remote_storage,
)
)
assert (
result
== """2021-11-05 abd 15
2021-11-16 aaa 22
"""
)
@pytest.mark.parametrize(
"table,use_local_cache_for_remote_storage,enable_parquet_rowgroup_minmax_index",
[
pytest.param(
"demo_parquet_no_cache_no_index",
"false",
"false",
id="demo_parquet_no_cache_no_index",
),
pytest.param(
"demo_parquet_with_cache_no_index",
"true",
"false",
id="demo_parquet_with_cache_no_index",
),
pytest.param(
"demo_parquet_no_cache_rowgroup_index",
"false",
"true",
id="demo_parquet_no_cache_rowgroup_index",
),
pytest.param(
"demo_parquet_with_cache_rowgroup_index",
"true",
"true",
id="demo_parquet_with_cache_rowgroup_index",
),
],
)
def test_parquet_minmax_index(
started_cluster,
table,
use_local_cache_for_remote_storage,
enable_parquet_rowgroup_minmax_index,
):
node = started_cluster.instances["h0_0_0"]
result = node.query(
"""
DROP TABLE IF EXISTS default.{table};
CREATE TABLE default.{table} (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day)
SETTINGS enable_parquet_rowgroup_minmax_index = {enable_parquet_rowgroup_minmax_index}
""".format(
table=table,
enable_parquet_rowgroup_minmax_index=enable_parquet_rowgroup_minmax_index,
)
)
assert result.strip() == ""
for i in range(2):
result = node.query(
"""
SELECT day, id, score FROM default.{table} where day >= '2021-11-05' and day <= '2021-11-16' and score >= 15 and score <= 30 order by day, id
SETTINGS use_local_cache_for_remote_storage = {use_local_cache_for_remote_storage}
""".format(
table=table,
use_local_cache_for_remote_storage=use_local_cache_for_remote_storage,
)
)
assert (
result
== """2021-11-05 abd 15
2021-11-16 aaa 22
"""
)
def test_hive_columns_prunning(started_cluster):
logging.info("Start testing groupby ...")
node = started_cluster.instances["h0_0_0"]
result = node.query(
"""
SELECT count(*) FROM default.demo_parquet_parts where day = '2021-11-05'
"""
)
expected_result = """4
"""
logging.info("query result:{}".format(result))
assert result == expected_result
def test_text_count(started_cluster):
node = started_cluster.instances["h0_0_0"]
result = node.query(
"""
DROP TABLE IF EXISTS default.demo_orc;
CREATE TABLE default.demo_orc (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day);
SELECT day, count(*) FROM default.demo_orc group by day order by day SETTINGS format_csv_delimiter = '\x01'
"""
)
expected_result = """2021-11-01 1
2021-11-05 2
2021-11-11 1
2021-11-16 2
"""
assert result == expected_result
def test_parquet_groupby_with_cache(started_cluster):
logging.info("Start testing groupby ...")
node = started_cluster.instances["h0_0_0"]
result = node.query(
"""
SELECT day, count(*) FROM default.demo_parquet group by day order by day
"""
)
expected_result = """2021-11-01 1
2021-11-05 2
2021-11-11 1
2021-11-16 2
"""
assert result == expected_result
def test_parquet_groupby_by_hive_function(started_cluster):
logging.info("Start testing groupby ...")
node = started_cluster.instances["h0_0_0"]
result = node.query(
"""
SELECT day, count(*) FROM hive('thrift://hivetest:9083', 'test', 'demo', '`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)', 'day') group by day order by day
"""
)
expected_result = """2021-11-01 1
2021-11-05 2
2021-11-11 1
2021-11-16 2
"""
assert result == expected_result
def test_cache_read_bytes(started_cluster):
node = started_cluster.instances["h0_0_0"]
result = node.query(
"""
CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day)
"""
)
test_passed = False
for i in range(10):
result = node.query(
"""
SELECT * FROM default.demo_parquet_1 settings input_format_parquet_allow_missing_columns = true
"""
)
node.query("system flush logs")
result = node.query(
"select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0"
)
if result.strip() == "0":
logging.info("ProfileEvent_ExternalDataSourceLocalCacheReadBytes == 0")
time.sleep(10)
continue
test_passed = True
break
assert test_passed
def test_cache_dir_use(started_cluster):
node = started_cluster.instances["h0_0_0"]
result0 = node.exec_in_container(
["bash", "-c", "ls /tmp/clickhouse_local_cache | wc -l"]
)
result1 = node.exec_in_container(
["bash", "-c", "ls /tmp/clickhouse_local_cache1 | wc -l"]
)
assert result0 != "0" and result1 != "0"
def test_hive_struct_type(started_cluster):
node = started_cluster.instances["h0_0_0"]
result = node.query(
"""
CREATE TABLE IF NOT EXISTS default.test_hive_types (`f_tinyint` Int8, `f_smallint` Int16, `f_int` Int32, `f_integer` Int32, `f_bigint` Int64, `f_float` Float32, `f_double` Float64, `f_decimal` Float64, `f_timestamp` DateTime, `f_date` Date, `f_string` String, `f_varchar` String, `f_char` String, `f_bool` Boolean, `f_array_int` Array(Int32), `f_array_string` Array(String), `f_array_float` Array(Float32), `f_map_int` Map(String, Int32), `f_map_string` Map(String, String), `f_map_float` Map(String, Float32), `f_struct` Tuple(a String, b Int32, c Float32, d Tuple(x Int32, y String)), `day` String) ENGINE = Hive('thrift://hivetest:9083', 'test', 'test_hive_types') PARTITION BY (day)
"""
)
result = node.query(
"""
SELECT * FROM default.test_hive_types WHERE day = '2022-02-20' SETTINGS input_format_parquet_import_nested=1
"""
)
expected_result = """1 2 3 4 5 6.11 7.22 8 2022-02-20 14:47:04 2022-02-20 hello world hello world hello world true [1,2,3] ['hello world','hello world'] [1.1,1.2] {'a':100,'b':200,'c':300} {'a':'aa','b':'bb','c':'cc'} {'a':111.1,'b':222.2,'c':333.3} ('aaa',200,333.3,(10,'xyz')) 2022-02-20"""
assert result.strip() == expected_result
result = node.query(
"""
SELECT day, f_struct.a, f_struct.d.x FROM default.test_hive_types WHERE day = '2022-02-20' SETTINGS input_format_parquet_import_nested=1
"""
)
expected_result = """2022-02-20 aaa 10"""
def test_table_alter_add(started_cluster):
node = started_cluster.instances["h0_0_0"]
result = node.query("DROP TABLE IF EXISTS default.demo_parquet_1")
result = node.query(
"""
CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day)
"""
)
result = node.query(
"""
ALTER TABLE default.demo_parquet_1 ADD COLUMN id Nullable(String) FIRST
"""
)
result = node.query("""DESC default.demo_parquet_1 FORMAT TSV""")
expected_result = "id\tNullable(String)\t\t\t\t\t\nscore\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)"
assert result.strip() == expected_result
def test_table_alter_drop(started_cluster):
node = started_cluster.instances["h0_0_0"]
result = node.query("DROP TABLE IF EXISTS default.demo_parquet_1")
result = node.query(
"""
CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day)
"""
)
result = node.query(
"""
ALTER TABLE default.demo_parquet_1 DROP COLUMN id
"""
)
result = node.query("""DESC default.demo_parquet_1 FORMAT TSV""")
expected_result = """score\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)"""
assert result.strip() == expected_result
def test_table_alter_comment(started_cluster):
node = started_cluster.instances["h0_0_0"]
result = node.query("DROP TABLE IF EXISTS default.demo_parquet_1")
result = node.query(
"""
CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day)
"""
)
result = node.query(
"""ALTER TABLE default.demo_parquet_1 COMMENT COLUMN id 'Text comment'"""
)
result = node.query("""DESC default.demo_parquet_1 FORMAT TSV""")
expected_result = """id\tNullable(String)\t\t\tText comment\t\t\nscore\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)"""
assert result.strip() == expected_result

View File

@ -1,11 +0,0 @@
<yandex>
<users>
<test_dns>
<password/>
<networks>
<host_regexp>test1\.example\.com$</host_regexp>
</networks>
<profile>default</profile>
</test_dns>
</users>
</yandex>

View File

@ -1,5 +0,0 @@
<yandex>
<listen_host>::</listen_host>
<listen_host>0.0.0.0</listen_host>
<listen_try>1</listen_try>
</yandex>

View File

@ -1,8 +0,0 @@
. {
hosts /example.com {
reload "20ms"
fallthrough
}
forward . 127.0.0.11
log
}

View File

@ -1 +0,0 @@
filled in runtime, but needs to exist in order to be volume mapped in docker

View File

@ -1,115 +0,0 @@
import pytest
# FIXME This test is too flaky
# https://github.com/ClickHouse/ClickHouse/issues/51471
pytestmark = pytest.mark.skip
import socket
from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check
from time import sleep
import os
DOCKER_COMPOSE_PATH = get_docker_compose_path()
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
cluster = ClickHouseCluster(__file__)
ch_server = cluster.add_instance(
"clickhouse-server",
with_coredns=True,
main_configs=["configs/listen_host.xml"],
user_configs=["configs/host_regexp.xml"],
ipv6_address="2001:3984:3989::1:1111",
)
client = cluster.add_instance(
"clickhouse-client",
ipv6_address="2001:3984:3989::1:1112",
)
@pytest.fixture(scope="module")
def started_cluster():
global cluster
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def check_ptr_record(ip, hostname):
try:
host, aliaslist, ipaddrlist = socket.gethostbyaddr(ip)
if hostname.lower() == host.lower():
return True
except socket.herror:
pass
return False
def setup_dns_server(ip):
domains_string = "test3.example.com test2.example.com test1.example.com"
example_file_path = f'{ch_server.env_variables["COREDNS_CONFIG_DIR"]}/example.com'
run_and_check(f"echo '{ip} {domains_string}' > {example_file_path}", shell=True)
# DNS server takes time to reload the configuration.
for try_num in range(10):
if all(check_ptr_record(ip, host) for host in domains_string.split()):
break
sleep(1)
def setup_ch_server(dns_server_ip):
ch_server.exec_in_container(
(["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"])
)
ch_server.exec_in_container(
(["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"])
)
ch_server.query("SYSTEM DROP DNS CACHE")
def build_endpoint_v4(ip):
return f"'http://{ip}:8123/?query=SELECT+1&user=test_dns'"
def build_endpoint_v6(ip):
return build_endpoint_v4(f"[{ip}]")
def test_host_regexp_multiple_ptr_v4_fails_with_wrong_resolution(started_cluster):
server_ip = cluster.get_instance_ip("clickhouse-server")
random_ip = "9.9.9.9"
dns_server_ip = cluster.get_instance_ip(cluster.coredns_host)
setup_dns_server(random_ip)
setup_ch_server(dns_server_ip)
endpoint = build_endpoint_v4(server_ip)
assert "1\n" != client.exec_in_container(["bash", "-c", f"curl {endpoint}"])
def test_host_regexp_multiple_ptr_v4(started_cluster):
server_ip = cluster.get_instance_ip("clickhouse-server")
client_ip = cluster.get_instance_ip("clickhouse-client")
dns_server_ip = cluster.get_instance_ip(cluster.coredns_host)
setup_dns_server(client_ip)
setup_ch_server(dns_server_ip)
endpoint = build_endpoint_v4(server_ip)
assert "1\n" == client.exec_in_container(["bash", "-c", f"curl {endpoint}"])
def test_host_regexp_multiple_ptr_v6(started_cluster):
setup_dns_server(client.ipv6_address)
setup_ch_server(cluster.get_instance_global_ipv6(cluster.coredns_host))
endpoint = build_endpoint_v6(ch_server.ipv6_address)
assert "1\n" == client.exec_in_container(["bash", "-c", f"curl -6 {endpoint}"])

View File

@ -108,74 +108,6 @@ def test_reconnect(started_cluster):
assert remote.query("SELECT count(*) FROM local1").strip() == "3"
@pytest.mark.skip(reason="Flapping test")
def test_inserts_batching(started_cluster):
instance = instance_test_inserts_batching
with PartitionManager() as pm:
pm.partition_instances(instance, remote)
instance.query("INSERT INTO local2_source(d, x) VALUES ('2000-01-01', 1)")
# Sleep a bit so that this INSERT forms a batch of its own.
time.sleep(0.2)
instance.query("INSERT INTO local2_source(x, d) VALUES (2, '2000-01-01')")
for i in range(3, 7):
instance.query(
"INSERT INTO local2_source(d, x) VALUES ('2000-01-01', {})".format(i)
)
for i in range(7, 9):
instance.query(
"INSERT INTO local2_source(x, d) VALUES ({}, '2000-01-01')".format(i)
)
instance.query("INSERT INTO local2_source(d, x) VALUES ('2000-01-01', 9)")
# After ALTER the structure of the saved blocks will be different
instance.query("DROP TABLE local2_view")
instance.query("ALTER TABLE distributed ADD COLUMN s String")
# Memory Engine doesn't support ALTER so we just DROP/CREATE everything
instance.query("DROP TABLE local2_source")
instance.query(
"CREATE TABLE local2_source (d Date, x UInt32, s String) ENGINE = Memory"
)
instance.query(
"CREATE MATERIALIZED VIEW local2_view to distributed AS SELECT d,x,s FROM local2_source"
)
for i in range(10, 13):
instance.query(
"INSERT INTO local2_source(d, x) VALUES ('2000-01-01', {})".format(i)
)
time.sleep(1.0)
result = remote.query(
"SELECT _part, groupArray(x) FROM local2 GROUP BY _part ORDER BY _part"
)
# Explanation: as merges are turned off on remote instance, active parts in local2 table correspond 1-to-1
# to inserted blocks.
# Batches of max 3 rows are formed as min_insert_block_size_rows = 3.
# Blocks:
# 1. Failed batch that is retried with the same contents.
# 2. Full batch of inserts regardless of the order of columns thanks to the view.
# 3. Full batch of inserts regardless order of columns thanks to the view.
# 4. Full batch of inserts after ALTER (that have different block structure).
# 5. What was left to insert before ALTER.
expected = """\
20000101_20000101_1_1_0 [1]
20000101_20000101_2_2_0 [2,3,4]
20000101_20000101_3_3_0 [5,6,7]
20000101_20000101_4_4_0 [10,11,12]
20000101_20000101_5_5_0 [8,9]
"""
assert TSV(result) == TSV(expected)
def test_inserts_local(started_cluster):
instance = instance_test_inserts_local_cluster
instance.query("INSERT INTO local_source VALUES ('2000-01-01', 1)")

View File

@ -41,81 +41,3 @@ def started_cluster():
def get_fake_zk(node):
return ku.get_fake_zk(cluster, node)
@pytest.mark.skip(reason="test is flaky because changes are not properly waited for")
def test_reconfig_replace_leader_in_one_command(started_cluster):
"""
Remove leader from a cluster of 3 and add a new node to this cluster in a single command
"""
zk1 = get_fake_zk(node1)
config = ku.get_config_str(zk1)
assert len(config.split("\n")) == 3
assert "node1" in config
assert "node2" in config
assert "node3" in config
assert "node4" not in config
for i in range(100):
zk1.create(f"/test_four_{i}", b"somedata")
zk2 = get_fake_zk(node2)
zk2.sync("/test_four_0")
ku.wait_configs_equal(config, zk2)
zk3 = get_fake_zk(node3)
zk3.sync("/test_four_0")
ku.wait_configs_equal(config, zk3)
for i in range(100):
assert zk2.exists(f"/test_four_{i}") is not None
assert zk3.exists(f"/test_four_{i}") is not None
assert ku.is_leader(cluster, node1)
node4.start_clickhouse()
config, _ = zk2.reconfig(
joining="server.4=node4:9234", leaving="1", new_members=None
)
config = config.decode("utf-8")
print("After removing 1 and adding 4", config)
assert len(config.split("\n")) == 3
assert "node1" not in config
assert "node2" in config
assert "node3" in config
assert "node4" in config
ku.wait_until_connected(cluster, node4)
time.sleep(1)
zk4 = get_fake_zk(node4)
zk4.sync("/test_four_0")
ku.wait_configs_equal(config, zk4)
for i in range(100):
assert zk4.exists(f"test_four_{i}") is not None
zk4.create(f"/test_four_{100 + i}", b"somedata")
with pytest.raises(Exception):
zk1.stop()
zk1.close()
zk1 = get_fake_zk(node1)
zk1.sync("/test_four_0")
zk2.stop()
zk2.close()
zk2 = get_fake_zk(node2)
zk2.sync("/test_four_0")
ku.wait_configs_equal(config, zk2)
zk3.stop()
zk3.close()
zk3 = get_fake_zk(node3)
zk3.sync("/test_four_0")
ku.wait_configs_equal(config, zk3)
for i in range(200):
assert zk2.exists(f"test_four_{i}") is not None
assert zk3.exists(f"test_four_{i}") is not None

View File

@ -27,111 +27,6 @@ def started_cluster():
cluster.shutdown()
@pytest.mark.skip(reason="Skipping mutations in partition does not work")
def test_mutations_in_partition_background(started_cluster):
try:
numbers = 100
name = "test_mutations_in_partition"
instance_test_mutations.query(
f"""CREATE TABLE {name} (date Date, a UInt64, b String) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY a"""
)
instance_test_mutations.query(
f"""INSERT INTO {name} SELECT '2019-07-29' AS date, number, toString(number) FROM numbers({numbers})"""
)
for i in range(0, numbers, 3):
instance_test_mutations.query(
f"""ALTER TABLE {name} DELETE IN PARTITION {i} WHERE a = {i}"""
)
for i in range(1, numbers, 3):
instance_test_mutations.query(
f"""ALTER TABLE {name} UPDATE b = 'changed' IN PARTITION {i} WHERE a = {i} """
)
def count_and_changed():
return instance_test_mutations.query(
f"SELECT count(), countIf(b == 'changed') FROM {name} SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV"
).splitlines()
all_done = False
for wait_times_for_mutation in range(
100
): # wait for replication 80 seconds max
time.sleep(0.8)
if count_and_changed() == ["66,33"]:
all_done = True
break
print(
instance_test_mutations.query(
f"SELECT mutation_id, command, parts_to_do, is_done, latest_failed_part, latest_fail_reason, parts_to_do_names FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames"
)
)
assert (count_and_changed(), all_done) == (["66,33"], True)
assert instance_test_mutations.query(
f"SELECT count(), sum(is_done) FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV"
).splitlines() == ["67,67"]
finally:
instance_test_mutations.query(f"""DROP TABLE {name}""")
@pytest.mark.skip(reason="Skipping mutations in partition does not work")
@pytest.mark.parametrize("sync", [("last",), ("all",)])
def test_mutations_in_partition_sync(started_cluster, sync):
try:
numbers = 10
name = "test_mutations_in_partition_sync"
instance_test_mutations.query(
f"""CREATE TABLE {name} (date Date, a UInt64, b String) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY a"""
)
instance_test_mutations.query(
f"""INSERT INTO {name} SELECT '2019-07-29' AS date, number, toString(number) FROM numbers({numbers})"""
)
for i in range(0, numbers, 3):
instance_test_mutations.query(
f"""ALTER TABLE {name} DELETE IN PARTITION {i} WHERE a = {i}"""
+ (" SETTINGS mutations_sync = 1" if sync == "all" else "")
)
for reverse_index, i in reversed(
list(enumerate(reversed(range(1, numbers, 3))))
):
instance_test_mutations.query(
f"""ALTER TABLE {name} UPDATE b = 'changed' IN PARTITION {i} WHERE a = {i}"""
+ (
" SETTINGS mutations_sync = 1"
if not reverse_index or sync == "all"
else ""
)
)
def count_and_changed():
return instance_test_mutations.query(
f"SELECT count(), countIf(b == 'changed') FROM {name} SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV"
).splitlines()
print(
instance_test_mutations.query(
f"SELECT mutation_id, command, parts_to_do, is_done, latest_failed_part, latest_fail_reason FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames"
)
)
assert count_and_changed() == ["6,3"]
assert instance_test_mutations.query(
f"SELECT count(), sum(is_done) FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV"
).splitlines() == ["7,7"]
finally:
instance_test_mutations.query(f"""DROP TABLE {name}""")
def test_mutations_with_merge_background_task(started_cluster):
instance_test_mutations.query(
"""SYSTEM STOP MERGES test_mutations_with_ast_elements"""
@ -190,92 +85,3 @@ def test_mutations_with_truncate_table(started_cluster):
).rstrip()
== "0"
)
@pytest.mark.skip(reason="Skipping mutations in partition does not work")
def test_mutations_will_not_hang_for_non_existing_parts_sync(started_cluster):
try:
numbers = 100
name = "test_mutations_will_not_hang_for_non_existing_parts_sync"
instance_test_mutations.query(
f"""CREATE TABLE {name} (date Date, a UInt64, b String) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY a"""
)
instance_test_mutations.query(
f"""INSERT INTO {name} SELECT '2019-07-29' AS date, number, toString(number) FROM numbers({numbers})"""
)
for i in range(0, numbers, 3):
instance_test_mutations.query(
f"""ALTER TABLE {name} DELETE IN PARTITION {i+1000} WHERE a = {i} SETTINGS mutations_sync = 1"""
)
def count():
return instance_test_mutations.query(
f"SELECT count() FROM {name} SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV"
).splitlines()
print(
instance_test_mutations.query(
f"SELECT mutation_id, command, parts_to_do, is_done, latest_failed_part, latest_fail_reason, parts_to_do_names FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames"
)
)
assert count() == [f"{numbers}"]
assert instance_test_mutations.query(
f"SELECT count(), sum(is_done) FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV"
).splitlines() == [f"34,34"]
finally:
instance_test_mutations.query(f"""DROP TABLE {name}""")
@pytest.mark.skip(reason="Skipping mutations in partition does not work")
def test_mutations_will_not_hang_for_non_existing_parts_async(started_cluster):
try:
numbers = 100
name = "test_mutations_will_not_hang_for_non_existing_parts_async"
instance_test_mutations.query(
f"""CREATE TABLE {name} (date Date, a UInt64, b String) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY a"""
)
instance_test_mutations.query(
f"""INSERT INTO {name} SELECT '2019-07-29' AS date, number, toString(number) FROM numbers({numbers})"""
)
for i in range(0, numbers, 3):
instance_test_mutations.query(
f"""ALTER TABLE {name} DELETE IN PARTITION {i+1000} WHERE a = {i}"""
)
def count():
return instance_test_mutations.query(
f"SELECT count() FROM {name} SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV"
).splitlines()
def count_and_sum_is_done():
return instance_test_mutations.query(
f"SELECT count(), sum(is_done) FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV"
).splitlines()
all_done = False
for wait_times_for_mutation in range(
100
): # wait for replication 80 seconds max
time.sleep(0.8)
if count_and_sum_is_done() == ["34,34"]:
all_done = True
break
print(
instance_test_mutations.query(
f"SELECT mutation_id, command, parts_to_do, is_done, latest_failed_part, latest_fail_reason, parts_to_do_names FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames"
)
)
assert count() == [f"{numbers}"]
assert count_and_sum_is_done() == ["34,34"]
finally:
instance_test_mutations.query(f"""DROP TABLE {name}""")

View File

@ -414,54 +414,6 @@ def start_cluster_diff_versions():
cluster.shutdown()
@pytest.mark.skip(reason="compatability is temporary broken")
def test_polymorphic_parts_diff_versions(start_cluster_diff_versions):
# Check that replication with Wide parts works between different versions.
node_old = node7
node_new = node8
insert_random_data("polymorphic_table", node7, 100)
node8.query("SYSTEM SYNC REPLICA polymorphic_table", timeout=20)
assert node8.query("SELECT count() FROM polymorphic_table") == "100\n"
assert (
node8.query(
"SELECT DISTINCT part_type FROM system.parts WHERE table = 'polymorphic_table' and active"
)
== "Wide\n"
)
@pytest.mark.skip(reason="compatability is temporary broken")
def test_polymorphic_parts_diff_versions_2(start_cluster_diff_versions):
# Replication doesn't work on old version if part is created in compact format, because
# this version doesn't know anything about it. It's considered to be ok.
node_old = node7
node_new = node8
insert_random_data("polymorphic_table_2", node_new, 100)
assert node_new.query("SELECT count() FROM polymorphic_table_2") == "100\n"
assert node_old.query("SELECT count() FROM polymorphic_table_2") == "0\n"
with pytest.raises(Exception):
node_old.query("SYSTEM SYNC REPLICA polymorphic_table_2", timeout=3)
node_old.restart_with_latest_version(fix_metadata=True)
node_old.query("SYSTEM SYNC REPLICA polymorphic_table_2", timeout=20)
# Works after update
assert node_old.query("SELECT count() FROM polymorphic_table_2") == "100\n"
assert (
node_old.query(
"SELECT DISTINCT part_type FROM system.parts WHERE table = 'polymorphic_table_2' and active"
)
== "Compact\n"
)
def test_polymorphic_parts_non_adaptive(start_cluster):
node1.query("SYSTEM STOP MERGES")
node2.query("SYSTEM STOP MERGES")

View File

@ -27,7 +27,10 @@ proto_dir = os.path.join(SCRIPT_DIR, "./protos")
gen_dir = os.path.join(SCRIPT_DIR, "./_gen")
os.makedirs(gen_dir, exist_ok=True)
run_and_check(
f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto",
"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \
{proto_dir}/clickhouse_grpc.proto".format(
proto_dir=proto_dir, gen_dir=gen_dir
),
shell=True,
)

View File

@ -1,93 +0,0 @@
<clickhouse>
<storage_configuration>
<disks>
<hdfs1>
<type>hdfs</type>
<endpoint>hdfs://hdfs1:9000/clickhouse1/</endpoint>
<!-- FIXME: chicken and egg problem with current cluster.py -->
<skip_access_check>true</skip_access_check>
</hdfs1>
<hdfs1_again>
<type>hdfs</type>
<endpoint>hdfs://hdfs1:9000/clickhouse1/</endpoint>
<!-- FIXME: chicken and egg problem with current cluster.py -->
<skip_access_check>true</skip_access_check>
</hdfs1_again>
<hdfs2>
<type>hdfs</type>
<endpoint>hdfs://hdfs1:9000/clickhouse2/</endpoint>
<!-- FIXME: chicken and egg problem with current cluster.py -->
<skip_access_check>true</skip_access_check>
</hdfs2>
</disks>
<policies>
<hdfs_only>
<volumes>
<main>
<disk>hdfs1</disk>
</main>
</volumes>
</hdfs_only>
<hybrid>
<volumes>
<main>
<disk>default</disk>
</main>
<external>
<disk>hdfs1</disk>
</external>
</volumes>
<move_factor>0.0</move_factor>
</hybrid>
<tiered>
<volumes>
<main>
<disk>hdfs2</disk>
</main>
<external>
<disk>hdfs1</disk>
</external>
</volumes>
</tiered>
<tiered_copy>
<volumes>
<main>
<disk>hdfs1_again</disk>
</main>
<external>
<disk>hdfs1</disk>
</external>
</volumes>
</tiered_copy>
</policies>
</storage_configuration>
<merge_tree>
<min_bytes_for_wide_part>1024000</min_bytes_for_wide_part>
<old_parts_lifetime>1</old_parts_lifetime>
<allow_remote_fs_zero_copy_replication>true</allow_remote_fs_zero_copy_replication>
</merge_tree>
<remote_servers>
<test_cluster>
<shard>
<replica>
<host>node1</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>node2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
<macros>
<cluster>test_cluster</cluster>
<shard>1</shard>
</macros>
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
</clickhouse>

View File

@ -1,322 +0,0 @@
import pytest
# FIXME This test is too flaky
# https://github.com/ClickHouse/ClickHouse/issues/42561
pytestmark = pytest.mark.skip
import logging
from string import Template
import time
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry
from pyhdfs import HdfsClient
SHARDS = 2
FILES_OVERHEAD_PER_TABLE = 1 # format_version.txt
FILES_OVERHEAD_PER_PART_COMPACT = 7
def wait_for_hdfs_objects(cluster, fp, expected, num_tries=30):
fs = HdfsClient(hosts=cluster.hdfs_ip)
while num_tries > 0:
num_hdfs_objects = len(fs.listdir(fp))
if num_hdfs_objects == expected:
break
num_tries -= 1
time.sleep(1)
assert len(fs.listdir(fp)) == expected
@pytest.fixture(scope="module")
def cluster():
try:
cluster = ClickHouseCluster(__file__)
cluster.add_instance(
"node1",
main_configs=["configs/config.d/storage_conf.xml"],
macros={"replica": "node1"},
with_zookeeper=True,
with_hdfs=True,
)
cluster.add_instance(
"node2",
main_configs=["configs/config.d/storage_conf.xml"],
macros={"replica": "node2"},
with_zookeeper=True,
with_hdfs=True,
)
logging.info("Starting cluster...")
cluster.start()
if cluster.instances["node1"].is_debug_build():
# https://github.com/ClickHouse/ClickHouse/issues/27814
pytest.skip(
"libhdfs3 calls rand function which does not pass harmful check in debug build"
)
logging.info("Cluster started")
fs = HdfsClient(hosts=cluster.hdfs_ip)
fs.mkdirs("/clickhouse1")
fs.mkdirs("/clickhouse2")
logging.info("Created HDFS directory")
yield cluster
finally:
cluster.shutdown()
def test_hdfs_zero_copy_replication_insert(cluster):
node1 = cluster.instances["node1"]
node2 = cluster.instances["node2"]
try:
node1.query(
"""
CREATE TABLE hdfs_test ON CLUSTER test_cluster (dt DateTime, id Int64)
ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/hdfs_test', '{replica}')
ORDER BY (dt, id)
SETTINGS storage_policy='hdfs_only'
"""
)
wait_for_hdfs_objects(
cluster, "/clickhouse1", SHARDS * FILES_OVERHEAD_PER_TABLE
)
node1.query("INSERT INTO hdfs_test VALUES (now() - INTERVAL 3 DAY, 10)")
node2.query("SYSTEM SYNC REPLICA hdfs_test", timeout=30)
assert node1.query("SELECT count() FROM hdfs_test FORMAT Values") == "(1)"
assert node2.query("SELECT count() FROM hdfs_test FORMAT Values") == "(1)"
assert (
node1.query("SELECT id FROM hdfs_test ORDER BY dt FORMAT Values") == "(10)"
)
assert (
node2.query("SELECT id FROM hdfs_test ORDER BY dt FORMAT Values") == "(10)"
)
assert (
node1.query(
"SELECT partition_id,disk_name FROM system.parts WHERE table='hdfs_test' FORMAT Values"
)
== "('all','hdfs1')"
)
assert (
node2.query(
"SELECT partition_id,disk_name FROM system.parts WHERE table='hdfs_test' FORMAT Values"
)
== "('all','hdfs1')"
)
wait_for_hdfs_objects(
cluster,
"/clickhouse1",
SHARDS * FILES_OVERHEAD_PER_TABLE + FILES_OVERHEAD_PER_PART_COMPACT,
)
finally:
node1.query("DROP TABLE IF EXISTS hdfs_test SYNC")
node2.query("DROP TABLE IF EXISTS hdfs_test SYNC")
@pytest.mark.parametrize(
("storage_policy", "init_objects"),
[("hybrid", 0), ("tiered", 0), ("tiered_copy", FILES_OVERHEAD_PER_TABLE)],
)
def test_hdfs_zero_copy_replication_single_move(cluster, storage_policy, init_objects):
node1 = cluster.instances["node1"]
try:
node1.query(
Template(
"""
CREATE TABLE single_node_move_test (dt DateTime, id Int64)
ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/single_node_move_test', '{replica}')
ORDER BY (dt, id)
SETTINGS storage_policy='$policy',temporary_directories_lifetime=1
"""
).substitute(policy=storage_policy)
)
wait_for_hdfs_objects(cluster, "/clickhouse1", init_objects)
node1.query(
"INSERT INTO single_node_move_test VALUES (now() - INTERVAL 3 DAY, 10), (now() - INTERVAL 1 DAY, 11)"
)
assert (
node1.query(
"SELECT id FROM single_node_move_test ORDER BY dt FORMAT Values"
)
== "(10),(11)"
)
node1.query(
"ALTER TABLE single_node_move_test MOVE PARTITION ID 'all' TO VOLUME 'external'"
)
assert (
node1.query(
"SELECT partition_id,disk_name FROM system.parts WHERE table='single_node_move_test' FORMAT Values"
)
== "('all','hdfs1')"
)
assert (
node1.query(
"SELECT id FROM single_node_move_test ORDER BY dt FORMAT Values"
)
== "(10),(11)"
)
wait_for_hdfs_objects(
cluster, "/clickhouse1", init_objects + FILES_OVERHEAD_PER_PART_COMPACT
)
node1.query(
"ALTER TABLE single_node_move_test MOVE PARTITION ID 'all' TO VOLUME 'main'"
)
assert (
node1.query(
"SELECT id FROM single_node_move_test ORDER BY dt FORMAT Values"
)
== "(10),(11)"
)
finally:
node1.query("DROP TABLE IF EXISTS single_node_move_test SYNC")
@pytest.mark.parametrize(
("storage_policy", "init_objects"),
[("hybrid", 0), ("tiered", 0), ("tiered_copy", SHARDS * FILES_OVERHEAD_PER_TABLE)],
)
def test_hdfs_zero_copy_replication_move(cluster, storage_policy, init_objects):
node1 = cluster.instances["node1"]
node2 = cluster.instances["node2"]
try:
node1.query(
Template(
"""
CREATE TABLE move_test ON CLUSTER test_cluster (dt DateTime, id Int64)
ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/move_test', '{replica}')
ORDER BY (dt, id)
SETTINGS storage_policy='$policy'
"""
).substitute(policy=storage_policy)
)
wait_for_hdfs_objects(cluster, "/clickhouse1", init_objects)
node1.query(
"INSERT INTO move_test VALUES (now() - INTERVAL 3 DAY, 10), (now() - INTERVAL 1 DAY, 11)"
)
node2.query("SYSTEM SYNC REPLICA move_test", timeout=30)
assert (
node1.query("SELECT id FROM move_test ORDER BY dt FORMAT Values")
== "(10),(11)"
)
assert (
node2.query("SELECT id FROM move_test ORDER BY dt FORMAT Values")
== "(10),(11)"
)
node1.query(
"ALTER TABLE move_test MOVE PARTITION ID 'all' TO VOLUME 'external'"
)
wait_for_hdfs_objects(
cluster, "/clickhouse1", init_objects + FILES_OVERHEAD_PER_PART_COMPACT
)
node2.query(
"ALTER TABLE move_test MOVE PARTITION ID 'all' TO VOLUME 'external'"
)
assert (
node1.query(
"SELECT partition_id,disk_name FROM system.parts WHERE table='move_test' FORMAT Values"
)
== "('all','hdfs1')"
)
assert (
node2.query(
"SELECT partition_id,disk_name FROM system.parts WHERE table='move_test' FORMAT Values"
)
== "('all','hdfs1')"
)
assert (
node1.query("SELECT id FROM move_test ORDER BY dt FORMAT Values")
== "(10),(11)"
)
assert (
node2.query("SELECT id FROM move_test ORDER BY dt FORMAT Values")
== "(10),(11)"
)
wait_for_hdfs_objects(
cluster, "/clickhouse1", init_objects + FILES_OVERHEAD_PER_PART_COMPACT
)
finally:
node1.query("DROP TABLE IF EXISTS move_test SYNC")
node2.query("DROP TABLE IF EXISTS move_test SYNC")
@pytest.mark.parametrize(("storage_policy"), ["hybrid", "tiered", "tiered_copy"])
def test_hdfs_zero_copy_with_ttl_move(cluster, storage_policy):
node1 = cluster.instances["node1"]
node2 = cluster.instances["node2"]
try:
node1.query(
Template(
"""
CREATE TABLE ttl_move_test ON CLUSTER test_cluster (dt DateTime, id Int64)
ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/ttl_move_test', '{replica}')
ORDER BY (dt, id)
TTL dt + INTERVAL 2 DAY TO VOLUME 'external'
SETTINGS storage_policy='$policy'
"""
).substitute(policy=storage_policy)
)
node1.query("INSERT INTO ttl_move_test VALUES (now() - INTERVAL 3 DAY, 10)")
node1.query("INSERT INTO ttl_move_test VALUES (now() - INTERVAL 1 DAY, 11)")
node1.query("OPTIMIZE TABLE ttl_move_test FINAL")
node2.query("SYSTEM SYNC REPLICA ttl_move_test", timeout=30)
assert_eq_with_retry(node1, "SELECT count() FROM ttl_move_test", "2")
assert_eq_with_retry(node2, "SELECT count() FROM ttl_move_test", "2")
assert (
node1.query("SELECT id FROM ttl_move_test ORDER BY id FORMAT Values")
== "(10),(11)"
)
assert (
node2.query("SELECT id FROM ttl_move_test ORDER BY id FORMAT Values")
== "(10),(11)"
)
finally:
node1.query("DROP TABLE IF EXISTS ttl_move_test SYNC")
node2.query("DROP TABLE IF EXISTS ttl_move_test SYNC")
def test_hdfs_zero_copy_with_ttl_delete(cluster):
node1 = cluster.instances["node1"]
node2 = cluster.instances["node2"]
try:
node1.query(
"""
CREATE TABLE ttl_delete_test ON CLUSTER test_cluster (dt DateTime, id Int64)
ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/ttl_delete_test', '{replica}')
ORDER BY (dt, id)
TTL dt + INTERVAL 2 DAY
SETTINGS storage_policy='tiered'
"""
)
node1.query("INSERT INTO ttl_delete_test VALUES (now() - INTERVAL 3 DAY, 10)")
node1.query("INSERT INTO ttl_delete_test VALUES (now() - INTERVAL 1 DAY, 11)")
node1.query("OPTIMIZE TABLE ttl_delete_test FINAL")
node2.query("SYSTEM SYNC REPLICA ttl_delete_test", timeout=30)
assert_eq_with_retry(node1, "SELECT count() FROM ttl_delete_test", "1")
assert_eq_with_retry(node2, "SELECT count() FROM ttl_delete_test", "1")
assert (
node1.query("SELECT id FROM ttl_delete_test ORDER BY id FORMAT Values")
== "(11)"
)
assert (
node2.query("SELECT id FROM ttl_delete_test ORDER BY id FORMAT Values")
== "(11)"
)
finally:
node1.query("DROP TABLE IF EXISTS ttl_delete_test SYNC")
node2.query("DROP TABLE IF EXISTS ttl_delete_test SYNC")

View File

@ -1,3 +0,0 @@
<clickhouse>
<disable_internal_dns_cache>1</disable_internal_dns_cache>
</clickhouse>

View File

@ -1,5 +0,0 @@
<yandex>
<listen_host>::</listen_host>
<listen_host>0.0.0.0</listen_host>
<listen_try>1</listen_try>
</yandex>

View File

@ -1,3 +0,0 @@
<clickhouse>
<allow_reverse_dns_query_function>1</allow_reverse_dns_query_function>
</clickhouse>

View File

@ -1,4 +0,0 @@
. {
forward . 127.0.0.11
log
}

View File

@ -1,56 +0,0 @@
import pytest
# FIXME This test is broken
# https://github.com/ClickHouse/ClickHouse/issues/53194
pytestmark = pytest.mark.skip
from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check
from time import sleep
import os
DOCKER_COMPOSE_PATH = get_docker_compose_path()
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
cluster = ClickHouseCluster(__file__)
ch_server = cluster.add_instance(
"clickhouse-server",
with_coredns=True,
main_configs=[
"configs/config.xml",
"configs/reverse_dns_function.xml",
"configs/listen_host.xml",
],
)
@pytest.fixture(scope="module")
def started_cluster():
global cluster
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def setup_ch_server(dns_server_ip):
ch_server.exec_in_container(
(["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"])
)
ch_server.exec_in_container(
(["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"])
)
ch_server.query("SYSTEM DROP DNS CACHE")
def test_reverse_dns_query(started_cluster):
dns_server_ip = cluster.get_instance_ip(cluster.coredns_host)
setup_ch_server(dns_server_ip)
for _ in range(0, 200):
response = ch_server.query("select reverseDNSQuery('2001:4860:4860::8888')")
assert response == "['dns.google']\n"

View File

@ -153,96 +153,6 @@ def test_s3_zero_copy_replication(started_cluster, policy):
node2.query("DROP TABLE IF EXISTS s3_test SYNC")
@pytest.mark.skip(reason="Test is flaky (and never was stable)")
def test_s3_zero_copy_on_hybrid_storage(started_cluster):
node1 = cluster.instances["node1"]
node2 = cluster.instances["node2"]
node1.query(
"""
CREATE TABLE hybrid_test ON CLUSTER test_cluster (id UInt32, value String)
ENGINE=ReplicatedMergeTree('/clickhouse/tables/hybrid_test', '{}')
ORDER BY id
SETTINGS storage_policy='hybrid',temporary_directories_lifetime=1
""".format(
"{replica}"
)
)
node1.query("INSERT INTO hybrid_test VALUES (0,'data'),(1,'data')")
node2.query("SYSTEM SYNC REPLICA hybrid_test", timeout=30)
assert (
node1.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values")
== "(0,'data'),(1,'data')"
)
assert (
node2.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values")
== "(0,'data'),(1,'data')"
)
assert (
node1.query(
"SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values"
)
== "('all','default')"
)
assert (
node2.query(
"SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values"
)
== "('all','default')"
)
node1.query("ALTER TABLE hybrid_test MOVE PARTITION ID 'all' TO DISK 's31'")
assert (
node1.query(
"SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values"
)
== "('all','s31')"
)
assert (
node2.query(
"SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values"
)
== "('all','default')"
)
# Total objects in S3
s3_objects = get_large_objects_count(cluster, size=0)
node2.query("ALTER TABLE hybrid_test MOVE PARTITION ID 'all' TO DISK 's31'")
assert (
node1.query(
"SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values"
)
== "('all','s31')"
)
assert (
node2.query(
"SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values"
)
== "('all','s31')"
)
# Check that after moving partition on node2 no new obects on s3
wait_for_large_objects_count(cluster, s3_objects, size=0)
assert (
node1.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values")
== "(0,'data'),(1,'data')"
)
assert (
node2.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values")
== "(0,'data'),(1,'data')"
)
node1.query("DROP TABLE IF EXISTS hybrid_test SYNC")
node2.query("DROP TABLE IF EXISTS hybrid_test SYNC")
def insert_data_time(node, table, number_of_mb, time, start=0):
values = ",".join(
f"({x},{time})"

View File

@ -1,9 +0,0 @@
<clickhouse>
<!-- Profiles of settings. -->
<profiles>
<!-- Default settings. -->
<default>
<max_insert_delayed_streams_for_parallel_write>10</max_insert_delayed_streams_for_parallel_write>
</default>
</profiles>
</clickhouse>

View File

@ -1,39 +0,0 @@
<clickhouse>
<storage_configuration>
<disks>
<s3_disk>
<type>s3</type>
<endpoint>http://minio1:9001/root/data/</endpoint>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</s3_disk>
</disks>
<policies>
<s3_and_default>
<volumes>
<main>
<disk>default</disk>
</main>
<external>
<disk>s3_disk</disk>
</external>
</volumes>
</s3_and_default>
<s3>
<volumes>
<main>
<disk>s3_disk</disk>
</main>
</volumes>
</s3>
</policies>
</storage_configuration>
<merge_tree>
<allow_remote_fs_zero_copy_replication>true</allow_remote_fs_zero_copy_replication>
<ratio_of_defaults_for_sparse_serialization>1.0</ratio_of_defaults_for_sparse_serialization>
</merge_tree>
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
</clickhouse>

View File

@ -1,94 +0,0 @@
#!/usr/bin/env python3
import time
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance(
"node1", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True
)
node2 = cluster.add_instance(
"node2", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True
)
node3 = cluster.add_instance(
"node3", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True
)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_ttl_move_and_s3(started_cluster):
for i, node in enumerate([node1, node2, node3]):
node.query(
"""
CREATE TABLE s3_test_with_ttl (date DateTime, id UInt32, value String)
ENGINE=ReplicatedMergeTree('/clickhouse/tables/s3_test', '{}')
ORDER BY id
PARTITION BY id
TTL date TO DISK 's3_disk'
SETTINGS storage_policy='s3_and_default', temporary_directories_lifetime=1
""".format(
i
)
)
node1.query("SYSTEM STOP MOVES s3_test_with_ttl")
node2.query("SYSTEM STOP MOVES s3_test_with_ttl")
for i in range(30):
if i % 2 == 0:
node = node1
else:
node = node2
node.query(
f"INSERT INTO s3_test_with_ttl SELECT now() + 5, {i}, randomPrintableASCII(1048570)"
)
node1.query("SYSTEM SYNC REPLICA s3_test_with_ttl")
node2.query("SYSTEM SYNC REPLICA s3_test_with_ttl")
node3.query("SYSTEM SYNC REPLICA s3_test_with_ttl")
assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n"
assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n"
node1.query("SYSTEM START MOVES s3_test_with_ttl")
node2.query("SYSTEM START MOVES s3_test_with_ttl")
assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n"
assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n"
for attempt in reversed(range(5)):
time.sleep(5)
print(
node1.query(
"SELECT * FROM system.parts WHERE table = 's3_test_with_ttl' FORMAT Vertical"
)
)
minio = cluster.minio_client
objects = minio.list_objects(cluster.minio_bucket, "data/", recursive=True)
counter = 0
for obj in objects:
print(f"Objectname: {obj.object_name}, metadata: {obj.metadata}")
counter += 1
print(f"Total objects: {counter}")
if counter == 330:
break
print(f"Attempts remaining: {attempt}")
assert counter == 330

View File

@ -1,81 +0,0 @@
#!/usr/bin/env python3
import time
import pytest
# FIXME This test is too flaky
# https://github.com/ClickHouse/ClickHouse/issues/45887
pytestmark = pytest.mark.skip
from helpers.cluster import ClickHouseCluster
single_node_cluster = ClickHouseCluster(__file__)
small_node = single_node_cluster.add_instance(
"small_node", main_configs=["configs/s3.xml"], with_minio=True
)
@pytest.fixture(scope="module")
def started_single_node_cluster():
try:
single_node_cluster.start()
yield single_node_cluster
finally:
single_node_cluster.shutdown()
def test_move_and_s3_memory_usage(started_single_node_cluster):
if small_node.is_built_with_sanitizer() or small_node.is_debug_build():
pytest.skip("Disabled for debug and sanitizers. Too slow.")
small_node.query(
"CREATE TABLE s3_test_with_ttl (x UInt32, a String codec(NONE), b String codec(NONE), c String codec(NONE), d String codec(NONE), e String codec(NONE)) engine = MergeTree order by x partition by x SETTINGS storage_policy='s3_and_default'"
)
for _ in range(10):
small_node.query(
"insert into s3_test_with_ttl select 0, repeat('a', 100), repeat('b', 100), repeat('c', 100), repeat('d', 100), repeat('e', 100) from zeros(400000) settings max_block_size = 8192, max_insert_block_size=10000000, min_insert_block_size_rows=10000000"
)
# After this, we should have 5 columns per 10 * 100 * 400000 ~ 400 MB; total ~2G data in partition
small_node.query(
"optimize table s3_test_with_ttl final",
settings={
"send_logs_level": "error",
"allow_prefetched_read_pool_for_remote_filesystem": 0,
},
)
small_node.query("system flush logs")
# Will take memory usage from metric_log.
# It is easier then specifying total memory limit (insert queries can hit this limit).
small_node.query("truncate table system.metric_log")
small_node.query(
"alter table s3_test_with_ttl move partition 0 to volume 'external'",
settings={
"send_logs_level": "error",
"allow_prefetched_read_pool_for_remote_filesystem": 0,
},
)
small_node.query("system flush logs")
max_usage = small_node.query(
"""
select max(m.val - am.val * 4096) from
(select toStartOfMinute(event_time) as time, max(CurrentMetric_MemoryTracking) as val from system.metric_log group by time) as m join
(select toStartOfMinute(event_time) as time, min(value) as val from system.asynchronous_metric_log where metric='jemalloc.arenas.all.pdirty' group by time) as am using time;"""
)
# 3G limit is a big one. However, we can hit it anyway with parallel s3 writes enabled.
# Also actual value can be bigger because of memory drift.
# Increase it a little bit if test fails.
assert int(max_usage) < 3e9
res = small_node.query(
"select * from system.errors where last_error_message like '%Memory limit%' limit 1",
settings={
"allow_prefetched_read_pool_for_remote_filesystem": 0,
},
)
assert res == ""

View File

@ -1,46 +0,0 @@
#!/usr/bin/env python3
import time
import pytest
from helpers.cluster import ClickHouseCluster
single_node_cluster = ClickHouseCluster(__file__)
small_node = single_node_cluster.add_instance(
"small_node",
main_configs=["configs/s3.xml"],
user_configs=["configs/max_delayed_streams.xml"],
with_minio=True,
)
@pytest.fixture(scope="module")
def started_single_node_cluster():
try:
single_node_cluster.start()
yield single_node_cluster
finally:
single_node_cluster.shutdown()
def test_vertical_merge_memory_usage(started_single_node_cluster):
if small_node.is_built_with_sanitizer() or small_node.is_debug_build():
pytest.skip("Disabled for debug and sanitizers. Too slow.")
small_node.query(
"create table tvm2 (c0 UInt64, c1 UInt64, c2 UInt64, c3 UInt64, c4 UInt64, c5 UInt64, c6 UInt64, c7 UInt64, c8 UInt64, c9 UInt64, c10 UInt64, c11 UInt64, c12 UInt64, c13 UInt64, c14 UInt64, c15 UInt64, c16 UInt64, c17 UInt64, c18 UInt64, c19 UInt64, c20 UInt64, c21 UInt64, c22 UInt64, c23 UInt64, c24 UInt64, c25 UInt64, c26 UInt64, c27 UInt64, c28 UInt64, c29 UInt64, c30 UInt64, c31 UInt64, c32 UInt64, c33 UInt64, c34 UInt64, c35 UInt64, c36 UInt64, c37 UInt64, c38 UInt64, c39 UInt64, c40 UInt64, c41 UInt64, c42 UInt64, c43 UInt64, c44 UInt64, c45 UInt64, c46 UInt64, c47 UInt64, c48 UInt64, c49 UInt64, c50 UInt64, c51 UInt64, c52 UInt64, c53 UInt64, c54 UInt64, c55 UInt64, c56 UInt64, c57 UInt64, c58 UInt64, c59 UInt64, c60 UInt64, c61 UInt64, c62 UInt64, c63 UInt64, c64 UInt64, c65 UInt64, c66 UInt64, c67 UInt64, c68 UInt64, c69 UInt64, c70 UInt64, c71 UInt64, c72 UInt64, c73 UInt64, c74 UInt64, c75 UInt64, c76 UInt64, c77 UInt64, c78 UInt64, c79 UInt64, c80 UInt64, c81 UInt64, c82 UInt64, c83 UInt64, c84 UInt64, c85 UInt64, c86 UInt64, c87 UInt64, c88 UInt64, c89 UInt64, c90 UInt64, c91 UInt64, c92 UInt64, c93 UInt64, c94 UInt64, c95 UInt64, c96 UInt64, c97 UInt64, c98 UInt64, c99 UInt64, c100 UInt64, c101 UInt64, c102 UInt64, c103 UInt64, c104 UInt64, c105 UInt64, c106 UInt64, c107 UInt64, c108 UInt64, c109 UInt64, c110 UInt64, c111 UInt64, c112 UInt64, c113 UInt64, c114 UInt64, c115 UInt64, c116 UInt64, c117 UInt64, c118 UInt64, c119 UInt64, c120 UInt64, c121 UInt64, c122 UInt64, c123 UInt64, c124 UInt64, c125 UInt64, c126 UInt64, c127 UInt64, c128 UInt64, c129 UInt64, c130 UInt64, c131 UInt64, c132 UInt64, c133 UInt64, c134 UInt64, c135 UInt64, c136 UInt64, c137 UInt64, c138 UInt64, c139 UInt64, c140 UInt64, c141 UInt64, c142 UInt64, c143 UInt64, c144 UInt64, c145 UInt64, c146 UInt64, c147 UInt64, c148 UInt64, c149 UInt64, c150 UInt64, c151 UInt64, c152 UInt64, c153 UInt64, c154 UInt64, c155 UInt64, c156 UInt64, c157 UInt64, c158 UInt64, c159 UInt64, c160 UInt64, c161 UInt64, c162 UInt64, c163 UInt64, c164 UInt64, c165 UInt64, c166 UInt64, c167 UInt64, c168 UInt64, c169 UInt64, c170 UInt64, c171 UInt64, c172 UInt64, c173 UInt64, c174 UInt64, c175 UInt64, c176 UInt64, c177 UInt64, c178 UInt64, c179 UInt64, c180 UInt64, c181 UInt64, c182 UInt64, c183 UInt64, c184 UInt64, c185 UInt64, c186 UInt64, c187 UInt64, c188 UInt64, c189 UInt64, c190 UInt64, c191 UInt64, c192 UInt64, c193 UInt64, c194 UInt64, c195 UInt64, c196 UInt64, c197 UInt64, c198 UInt64, c199 UInt64, c200 UInt64, c201 UInt64, c202 UInt64, c203 UInt64, c204 UInt64, c205 UInt64, c206 UInt64, c207 UInt64, c208 UInt64, c209 UInt64, c210 UInt64, c211 UInt64, c212 UInt64, c213 UInt64, c214 UInt64, c215 UInt64, c216 UInt64, c217 UInt64, c218 UInt64, c219 UInt64, c220 UInt64, c221 UInt64, c222 UInt64, c223 UInt64, c224 UInt64, c225 UInt64, c226 UInt64, c227 UInt64, c228 UInt64, c229 UInt64, c230 UInt64, c231 UInt64, c232 UInt64, c233 UInt64, c234 UInt64, c235 UInt64, c236 UInt64, c237 UInt64, c238 UInt64, c239 UInt64, c240 UInt64, c241 UInt64, c242 UInt64, c243 UInt64, c244 UInt64, c245 UInt64, c246 UInt64, c247 UInt64, c248 UInt64, c249 UInt64, c250 UInt64, c251 UInt64, c252 UInt64, c253 UInt64, c254 UInt64, c255 UInt64, c256 UInt64, c257 UInt64, c258 UInt64, c259 UInt64, c260 UInt64, c261 UInt64, c262 UInt64, c263 UInt64, c264 UInt64, c265 UInt64, c266 UInt64, c267 UInt64, c268 UInt64, c269 UInt64, c270 UInt64, c271 UInt64, c272 UInt64, c273 UInt64, c274 UInt64, c275 UInt64, c276 UInt64, c277 UInt64, c278 UInt64, c279 UInt64, c280 UInt64, c281 UInt64, c282 UInt64, c283 UInt64, c284 UInt64, c285 UInt64, c286 UInt64, c287 UInt64, c288 UInt64, c289 UInt64, c290 UInt64, c291 UInt64, c292 UInt64, c293 UInt64, c294 UInt64, c295 UInt64, c296 UInt64, c297 UInt64, c298 UInt64, c299 UInt64) engine = MergeTree order by tuple() settings min_rows_for_wide_part = 10, min_bytes_for_wide_part=0, storage_policy = 's3', vertical_merge_algorithm_min_rows_to_activate=1"
)
small_node.query(
"insert into tvm2 select number + 0, number + 1, number + 2, number + 3, number + 4, number + 5, number + 6, number + 7, number + 8, number + 9, number + 10, number + 11, number + 12, number + 13, number + 14, number + 15, number + 16, number + 17, number + 18, number + 19, number + 20, number + 21, number + 22, number + 23, number + 24, number + 25, number + 26, number + 27, number + 28, number + 29, number + 30, number + 31, number + 32, number + 33, number + 34, number + 35, number + 36, number + 37, number + 38, number + 39, number + 40, number + 41, number + 42, number + 43, number + 44, number + 45, number + 46, number + 47, number + 48, number + 49, number + 50, number + 51, number + 52, number + 53, number + 54, number + 55, number + 56, number + 57, number + 58, number + 59, number + 60, number + 61, number + 62, number + 63, number + 64, number + 65, number + 66, number + 67, number + 68, number + 69, number + 70, number + 71, number + 72, number + 73, number + 74, number + 75, number + 76, number + 77, number + 78, number + 79, number + 80, number + 81, number + 82, number + 83, number + 84, number + 85, number + 86, number + 87, number + 88, number + 89, number + 90, number + 91, number + 92, number + 93, number + 94, number + 95, number + 96, number + 97, number + 98, number + 99, number + 100, number + 101, number + 102, number + 103, number + 104, number + 105, number + 106, number + 107, number + 108, number + 109, number + 110, number + 111, number + 112, number + 113, number + 114, number + 115, number + 116, number + 117, number + 118, number + 119, number + 120, number + 121, number + 122, number + 123, number + 124, number + 125, number + 126, number + 127, number + 128, number + 129, number + 130, number + 131, number + 132, number + 133, number + 134, number + 135, number + 136, number + 137, number + 138, number + 139, number + 140, number + 141, number + 142, number + 143, number + 144, number + 145, number + 146, number + 147, number + 148, number + 149, number + 150, number + 151, number + 152, number + 153, number + 154, number + 155, number + 156, number + 157, number + 158, number + 159, number + 160, number + 161, number + 162, number + 163, number + 164, number + 165, number + 166, number + 167, number + 168, number + 169, number + 170, number + 171, number + 172, number + 173, number + 174, number + 175, number + 176, number + 177, number + 178, number + 179, number + 180, number + 181, number + 182, number + 183, number + 184, number + 185, number + 186, number + 187, number + 188, number + 189, number + 190, number + 191, number + 192, number + 193, number + 194, number + 195, number + 196, number + 197, number + 198, number + 199, number + 200, number + 201, number + 202, number + 203, number + 204, number + 205, number + 206, number + 207, number + 208, number + 209, number + 210, number + 211, number + 212, number + 213, number + 214, number + 215, number + 216, number + 217, number + 218, number + 219, number + 220, number + 221, number + 222, number + 223, number + 224, number + 225, number + 226, number + 227, number + 228, number + 229, number + 230, number + 231, number + 232, number + 233, number + 234, number + 235, number + 236, number + 237, number + 238, number + 239, number + 240, number + 241, number + 242, number + 243, number + 244, number + 245, number + 246, number + 247, number + 248, number + 249, number + 250, number + 251, number + 252, number + 253, number + 254, number + 255, number + 256, number + 257, number + 258, number + 259, number + 260, number + 261, number + 262, number + 263, number + 264, number + 265, number + 266, number + 267, number + 268, number + 269, number + 270, number + 271, number + 272, number + 273, number + 274, number + 275, number + 276, number + 277, number + 278, number + 279, number + 280, number + 281, number + 282, number + 283, number + 284, number + 285, number + 286, number + 287, number + 288, number + 289, number + 290, number + 291, number + 292, number + 293, number + 294, number + 295, number + 296, number + 297, number + 298, number + 299 from numbers(20)"
)
small_node.query("optimize table tvm2 final")
small_node.query("system flush logs")
# Should be about 25M
res = small_node.query(
"select formatReadableSize(peak_memory_usage), * from system.part_log where table = 'tvm2' and database = currentDatabase() and event_date >= today() - 1 and event_type = 'MergeParts' and peak_memory_usage > 100000000 format Vertical"
)
assert res == ""

View File

@ -1 +0,0 @@
_gen

View File

@ -1,9 +0,0 @@
<clickhouse>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
<size>1000M</size>
<count>10</count>
</logger>
</clickhouse>

View File

@ -1,9 +0,0 @@
<clickhouse>
<postgresql_port>5433</postgresql_port>
<mysql_port>9001</mysql_port>
<grpc_port>9100</grpc_port>
<grpc replace="replace">
<!-- Enable if you want very detailed logs -->
<verbose_logs>false</verbose_logs>
</grpc>
</clickhouse>

View File

@ -1,9 +0,0 @@
<clickhouse>
<session_log>
<database>system</database>
<table>session_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</session_log>
</clickhouse>

View File

@ -1,23 +0,0 @@
<clickhouse>
<profiles>
<default>
<function_sleep_max_microseconds_per_block>0</function_sleep_max_microseconds_per_block>
</default>
</profiles>
<users>
<default>
</default>
<mysql_user>
<password>pass</password>
</mysql_user>
<postgres_user>
<password>pass</password>
</postgres_user>
<grpc_user>
<password>pass</password>
</grpc_user>
<parallel_user>
<password>pass</password>
</parallel_user>
</users>
</clickhouse>

View File

@ -1 +0,0 @@
../../../../src/Server/grpc_protos/clickhouse_grpc.proto

View File

@ -1,289 +0,0 @@
import os
import grpc
import pymysql.connections
import psycopg2 as py_psql
import pytest
import random
import sys
import threading
from helpers.cluster import ClickHouseCluster, run_and_check
POSTGRES_SERVER_PORT = 5433
MYSQL_SERVER_PORT = 9001
GRPC_PORT = 9100
SESSION_LOG_MATCHING_FIELDS = "auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface"
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
DEFAULT_ENCODING = "utf-8"
# Use grpcio-tools to generate *pb2.py files from *.proto.
proto_dir = os.path.join(SCRIPT_DIR, "./protos")
gen_dir = os.path.join(SCRIPT_DIR, "./_gen")
os.makedirs(gen_dir, exist_ok=True)
run_and_check(
f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto",
shell=True,
)
sys.path.append(gen_dir)
import clickhouse_grpc_pb2
import clickhouse_grpc_pb2_grpc
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance(
"node",
main_configs=[
"configs/ports.xml",
"configs/log.xml",
"configs/session_log.xml",
],
user_configs=["configs/users.xml"],
# Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387
env_variables={
"TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS", default="")
},
)
def grpc_get_url():
return f"{instance.ip_address}:{GRPC_PORT}"
def grpc_create_insecure_channel():
channel = grpc.insecure_channel(grpc_get_url())
grpc.channel_ready_future(channel).result(timeout=2)
return channel
session_id_counter = 0
def next_session_id():
global session_id_counter
session_id = session_id_counter
session_id_counter += 1
return str(session_id)
def grpc_query(query, user_, pass_, raise_exception):
try:
query_info = clickhouse_grpc_pb2.QueryInfo(
query=query,
session_id=next_session_id(),
user_name=user_,
password=pass_,
)
channel = grpc_create_insecure_channel()
stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel)
result = stub.ExecuteQuery(query_info)
if result and result.HasField("exception"):
raise Exception(result.exception.display_text)
return result.output.decode(DEFAULT_ENCODING)
except Exception:
assert raise_exception
def postgres_query(query, user_, pass_, raise_exception):
try:
client = py_psql.connect(
host=instance.ip_address,
port=POSTGRES_SERVER_PORT,
user=user_,
password=pass_,
database="default",
)
cursor = client.cursor()
cursor.execute(query)
cursor.fetchall()
except Exception:
assert raise_exception
def mysql_query(query, user_, pass_, raise_exception):
try:
client = pymysql.connections.Connection(
host=instance.ip_address,
user=user_,
password=pass_,
database="default",
port=MYSQL_SERVER_PORT,
)
cursor = client.cursor(pymysql.cursors.DictCursor)
if raise_exception:
with pytest.raises(Exception):
cursor.execute(query)
else:
cursor.execute(query)
cursor.fetchall()
except Exception:
assert raise_exception
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_grpc_session(started_cluster):
grpc_query("SELECT 1", "grpc_user", "pass", False)
grpc_query("SELECT 2", "grpc_user", "wrong_pass", True)
grpc_query("SELECT 3", "wrong_grpc_user", "pass", True)
instance.query("SYSTEM FLUSH LOGS")
login_success_records = instance.query(
"SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginSuccess'"
)
assert login_success_records == "grpc_user\t1\t1\n"
logout_records = instance.query(
"SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'Logout'"
)
assert logout_records == "grpc_user\t1\t1\n"
login_failure_records = instance.query(
"SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginFailure'"
)
assert login_failure_records == "grpc_user\t1\t1\n"
logins_and_logouts = instance.query(
f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'Logout')"
)
assert logins_and_logouts == "1\n"
def test_mysql_session(started_cluster):
mysql_query("SELECT 1", "mysql_user", "pass", False)
mysql_query("SELECT 2", "mysql_user", "wrong_pass", True)
mysql_query("SELECT 3", "wrong_mysql_user", "pass", True)
instance.query("SYSTEM FLUSH LOGS")
login_success_records = instance.query(
"SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginSuccess'"
)
assert login_success_records == "mysql_user\t1\t1\n"
logout_records = instance.query(
"SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'Logout'"
)
assert logout_records == "mysql_user\t1\t1\n"
login_failure_records = instance.query(
"SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginFailure'"
)
assert login_failure_records == "mysql_user\t1\t1\n"
logins_and_logouts = instance.query(
f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'Logout')"
)
assert logins_and_logouts == "1\n"
def test_postgres_session(started_cluster):
postgres_query("SELECT 1", "postgres_user", "pass", False)
postgres_query("SELECT 2", "postgres_user", "wrong_pass", True)
postgres_query("SELECT 3", "wrong_postgres_user", "pass", True)
instance.query("SYSTEM FLUSH LOGS")
login_success_records = instance.query(
"SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginSuccess'"
)
assert login_success_records == "postgres_user\t1\t1\n"
logout_records = instance.query(
"SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'Logout'"
)
assert logout_records == "postgres_user\t1\t1\n"
login_failure_records = instance.query(
"SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginFailure'"
)
assert login_failure_records == "postgres_user\t1\t1\n"
logins_and_logouts = instance.query(
f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'Logout')"
)
assert logins_and_logouts == "1\n"
def test_parallel_sessions(started_cluster):
thread_list = []
for _ in range(10):
# Sleep time does not significantly matter here,
# test should pass even without sleeping.
for function in [postgres_query, grpc_query, mysql_query]:
thread = threading.Thread(
target=function,
args=(
f"SELECT sleep({random.uniform(0.03, 0.04)})",
"parallel_user",
"pass",
False,
),
)
thread.start()
thread_list.append(thread)
thread = threading.Thread(
target=function,
args=(
f"SELECT sleep({random.uniform(0.03, 0.04)})",
"parallel_user",
"wrong_pass",
True,
),
)
thread.start()
thread_list.append(thread)
thread = threading.Thread(
target=function,
args=(
f"SELECT sleep({random.uniform(0.03, 0.04)})",
"wrong_parallel_user",
"pass",
True,
),
)
thread.start()
thread_list.append(thread)
for thread in thread_list:
thread.join()
instance.query("SYSTEM FLUSH LOGS")
port_0_sessions = instance.query(
f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user'"
)
assert port_0_sessions == "90\n"
port_0_sessions = instance.query(
f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_port = 0"
)
assert port_0_sessions == "0\n"
address_0_sessions = instance.query(
f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_address = toIPv6('::')"
)
assert address_0_sessions == "0\n"
grpc_sessions = instance.query(
f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'gRPC'"
)
assert grpc_sessions == "30\n"
mysql_sessions = instance.query(
f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'MySQL'"
)
assert mysql_sessions == "30\n"
postgres_sessions = instance.query(
f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'PostgreSQL'"
)
assert postgres_sessions == "30\n"
logins_and_logouts = instance.query(
f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'Logout')"
)
assert logins_and_logouts == "30\n"
logout_failure_sessions = instance.query(
f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginFailure'"
)
assert logout_failure_sessions == "30\n"

View File

@ -1,6 +0,0 @@
syntax = "proto3";
message ProtoKeyValue {
uint64 key = 1;
string value = 2;
}

View File

@ -1,7 +0,0 @@
<clickhouse>
<macros>
<nats_url>nats1:4444</nats_url>
<nats_subjects>macro</nats_subjects>
<nats_format>JSONEachRow</nats_format>
</macros>
</clickhouse>

View File

@ -1,13 +0,0 @@
<clickhouse>
<named_collections>
<nats1>
<nats_url>nats1:4444</nats_url>
<nats_subjects>named</nats_subjects>
<nats_format>JSONEachRow</nats_format>
<nats_skip_broken_messages>111</nats_skip_broken_messages>
<nats_num_consumers>12</nats_num_consumers>
<nats_username>click</nats_username>
<nats_password>house</nats_password>
</nats1>
</named_collections>
</clickhouse>

View File

@ -1,6 +0,0 @@
<clickhouse>
<nats>
<user>click</user>
<password>house</password>
</nats>
</clickhouse>

View File

@ -1,7 +0,0 @@
<clickhouse>
<profiles>
<default>
<stream_like_engine_allow_direct_select>1</stream_like_engine_allow_direct_select>
</default>
</profiles>
</clickhouse>

View File

@ -1,13 +0,0 @@
#!/bin/bash
set -euxo pipefail
mkdir -p "${NATS_CERT_DIR}/ca"
mkdir -p "${NATS_CERT_DIR}/nats"
openssl req -newkey rsa:4096 -x509 -days 365 -nodes -batch -keyout "${NATS_CERT_DIR}/ca/ca-key.pem" -out "${NATS_CERT_DIR}/ca/ca-cert.pem" -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=ca"
openssl req -newkey rsa:4096 -nodes -batch -keyout "${NATS_CERT_DIR}/nats/server-key.pem" -out "${NATS_CERT_DIR}/nats/server-req.pem" -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=server"
openssl x509 -req -days 365 -in "${NATS_CERT_DIR}/nats/server-req.pem" -CA "${NATS_CERT_DIR}/ca/ca-cert.pem" -CAkey "${NATS_CERT_DIR}/ca/ca-key.pem" -CAcreateserial -out "${NATS_CERT_DIR}/nats/server-cert.pem" -extfile <(
cat <<-EOF
subjectAltName = DNS:localhost, DNS:nats1
EOF
)
rm -f "${NATS_CERT_DIR}/nats/server-req.pem"

View File

@ -1,37 +0,0 @@
# -*- coding: utf-8 -*-
# Generated by the protocol buffer compiler. DO NOT EDIT!
# source: clickhouse_path/format_schemas/nats.proto
"""Generated protocol buffer code."""
from google.protobuf import descriptor as _descriptor
from google.protobuf import descriptor_pool as _descriptor_pool
from google.protobuf import message as _message
from google.protobuf import reflection as _reflection
from google.protobuf import symbol_database as _symbol_database
# @@protoc_insertion_point(imports)
_sym_db = _symbol_database.Default()
DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(
b'\n)clickhouse_path/format_schemas/nats.proto"+\n\rProtoKeyValue\x12\x0b\n\x03key\x18\x01 \x01(\x04\x12\r\n\x05value\x18\x02 \x01(\tb\x06proto3'
)
_PROTOKEYVALUE = DESCRIPTOR.message_types_by_name["ProtoKeyValue"]
ProtoKeyValue = _reflection.GeneratedProtocolMessageType(
"ProtoKeyValue",
(_message.Message,),
{
"DESCRIPTOR": _PROTOKEYVALUE,
"__module__": "clickhouse_path.format_schemas.nats_pb2"
# @@protoc_insertion_point(class_scope:ProtoKeyValue)
},
)
_sym_db.RegisterMessage(ProtoKeyValue)
if _descriptor._USE_C_DESCRIPTORS == False:
DESCRIPTOR._options = None
_PROTOKEYVALUE._serialized_start = 45
_PROTOKEYVALUE._serialized_end = 88
# @@protoc_insertion_point(module_scope)

File diff suppressed because it is too large Load Diff

View File

@ -1,50 +0,0 @@
0 0
1 1
2 2
3 3
4 4
5 5
6 6
7 7
8 8
9 9
10 10
11 11
12 12
13 13
14 14
15 15
16 16
17 17
18 18
19 19
20 20
21 21
22 22
23 23
24 24
25 25
26 26
27 27
28 28
29 29
30 30
31 31
32 32
33 33
34 34
35 35
36 36
37 37
38 38
39 39
40 40
41 41
42 42
43 43
44 44
45 45
46 46
47 47
48 48
49 49

View File

@ -1,11 +0,0 @@
<clickhouse>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-server/log.log</log>
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
<size>1000M</size>
<count>10</count>
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
</logger>
</clickhouse>

View File

@ -1,780 +0,0 @@
import pytest
# FIXME Tests with MaterializedPostgresSQL are temporarily disabled
# https://github.com/ClickHouse/ClickHouse/issues/36898
# https://github.com/ClickHouse/ClickHouse/issues/38677
# https://github.com/ClickHouse/ClickHouse/pull/39272#issuecomment-1190087190
pytestmark = pytest.mark.skip
import time
import psycopg2
import os.path as p
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry
from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT
from helpers.test_tools import TSV
import threading
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance(
"instance",
main_configs=["configs/log_conf.xml"],
with_postgres=True,
stay_alive=True,
)
postgres_table_template = """
CREATE TABLE IF NOT EXISTS {} (
key Integer NOT NULL, value Integer, PRIMARY KEY(key))
"""
queries = [
"INSERT INTO {} select i, i from generate_series(0, 10000) as t(i);",
"DELETE FROM {} WHERE (value*value) % 3 = 0;",
"UPDATE {} SET value = value + 125 WHERE key % 2 = 0;",
"UPDATE {} SET key=key+20000 WHERE key%2=0",
"INSERT INTO {} select i, i from generate_series(40000, 50000) as t(i);",
"DELETE FROM {} WHERE key % 10 = 0;",
"UPDATE {} SET value = value + 101 WHERE key % 2 = 1;",
"UPDATE {} SET key=key+80000 WHERE key%2=1",
"DELETE FROM {} WHERE value % 2 = 0;",
"UPDATE {} SET value = value + 2000 WHERE key % 5 = 0;",
"INSERT INTO {} select i, i from generate_series(200000, 250000) as t(i);",
"DELETE FROM {} WHERE value % 3 = 0;",
"UPDATE {} SET value = value * 2 WHERE key % 3 = 0;",
"UPDATE {} SET key=key+500000 WHERE key%2=1",
"INSERT INTO {} select i, i from generate_series(1000000, 1050000) as t(i);",
"DELETE FROM {} WHERE value % 9 = 2;",
"UPDATE {} SET key=key+10000000",
"UPDATE {} SET value = value + 2 WHERE key % 3 = 1;",
"DELETE FROM {} WHERE value%5 = 0;",
]
@pytest.mark.timeout(30)
def check_tables_are_synchronized(
table_name, order_by="key", postgres_database="postgres_database"
):
while True:
expected = instance.query(
"select * from {}.{} order by {};".format(
postgres_database, table_name, order_by
)
)
result = instance.query(
"select * from test.{} order by {};".format(table_name, order_by)
)
if result == expected:
break
assert result == expected
def get_postgres_conn(
ip, port, database=False, auto_commit=True, database_name="postgres_database"
):
if database == True:
conn_string = "host={} port={} dbname='{}' user='postgres' password='mysecretpassword'".format(
ip, port, database_name
)
else:
conn_string = (
"host={} port={} user='postgres' password='mysecretpassword'".format(
ip, port
)
)
conn = psycopg2.connect(conn_string)
if auto_commit:
conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT)
conn.autocommit = True
return conn
def create_postgres_db(cursor, name):
cursor.execute("CREATE DATABASE {}".format(name))
def create_clickhouse_postgres_db(ip, port, name="postgres_database"):
instance.query(
"""
CREATE DATABASE {}
ENGINE = PostgreSQL('{}:{}', '{}', 'postgres', 'mysecretpassword')""".format(
name, ip, port, name
)
)
def create_materialized_table(ip, port, table_name="postgresql_replica"):
instance.query(
f"""
CREATE TABLE test.{table_name} (key Int64, value Int64)
ENGINE = MaterializedPostgreSQL(
'{ip}:{port}', 'postgres_database', '{table_name}', 'postgres', 'mysecretpassword')
PRIMARY KEY key; """
)
def create_postgres_table(cursor, table_name, replica_identity_full=False):
cursor.execute("DROP TABLE IF EXISTS {}".format(table_name))
cursor.execute(postgres_table_template.format(table_name))
if replica_identity_full:
cursor.execute("ALTER TABLE {} REPLICA IDENTITY FULL;".format(table_name))
def postgresql_replica_check_result(
result, check=False, ref_file="test_postgresql_replica.reference"
):
fpath = p.join(p.dirname(__file__), ref_file)
with open(fpath) as reference:
if check:
assert TSV(result) == TSV(reference)
else:
return TSV(result) == TSV(reference)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
conn = get_postgres_conn(ip=cluster.postgres_ip, port=cluster.postgres_port)
cursor = conn.cursor()
create_postgres_db(cursor, "postgres_database")
create_clickhouse_postgres_db(
ip=cluster.postgres_ip, port=cluster.postgres_port
)
instance.query("CREATE DATABASE test")
yield cluster
finally:
cluster.shutdown()
@pytest.mark.timeout(320)
def test_initial_load_from_snapshot(started_cluster):
conn = get_postgres_conn(
ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
database=True,
)
cursor = conn.cursor()
create_postgres_table(cursor, "postgresql_replica")
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)"
)
instance.query("DROP TABLE IF EXISTS test.postgresql_replica")
create_materialized_table(
ip=started_cluster.postgres_ip, port=started_cluster.postgres_port
)
result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;")
while postgresql_replica_check_result(result) == False:
time.sleep(0.2)
result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;")
cursor.execute("DROP TABLE postgresql_replica;")
postgresql_replica_check_result(result, True)
instance.query(f"DROP TABLE test.postgresql_replica SYNC")
@pytest.mark.timeout(320)
def test_no_connection_at_startup(started_cluster):
conn = get_postgres_conn(
ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
database=True,
)
cursor = conn.cursor()
create_postgres_table(cursor, "postgresql_replica")
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)"
)
instance.query("DROP TABLE IF EXISTS test.postgresql_replica")
create_materialized_table(
ip=started_cluster.postgres_ip, port=started_cluster.postgres_port
)
time.sleep(3)
instance.query("DETACH TABLE test.postgresql_replica")
started_cluster.pause_container("postgres1")
instance.query("ATTACH TABLE test.postgresql_replica")
time.sleep(3)
started_cluster.unpause_container("postgres1")
result = instance.query("SELECT count() FROM test.postgresql_replica;")
while int(result) == 0:
time.sleep(0.5)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;")
cursor.execute("DROP TABLE postgresql_replica;")
postgresql_replica_check_result(result, True)
instance.query(f"DROP TABLE test.postgresql_replica SYNC")
@pytest.mark.timeout(320)
def test_detach_attach_is_ok(started_cluster):
conn = get_postgres_conn(
ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
database=True,
)
cursor = conn.cursor()
create_postgres_table(cursor, "postgresql_replica")
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)"
)
instance.query("DROP TABLE IF EXISTS test.postgresql_replica")
create_materialized_table(
ip=started_cluster.postgres_ip, port=started_cluster.postgres_port
)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
while int(result) == 0:
time.sleep(0.2)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;")
postgresql_replica_check_result(result, True)
instance.query("DETACH TABLE test.postgresql_replica")
instance.query("ATTACH TABLE test.postgresql_replica")
result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;")
while postgresql_replica_check_result(result) == False:
time.sleep(0.5)
result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;")
cursor.execute("DROP TABLE postgresql_replica;")
postgresql_replica_check_result(result, True)
instance.query(f"DROP TABLE test.postgresql_replica SYNC")
@pytest.mark.timeout(320)
def test_replicating_insert_queries(started_cluster):
conn = get_postgres_conn(
ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
database=True,
)
cursor = conn.cursor()
create_postgres_table(cursor, "postgresql_replica")
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(10)"
)
instance.query("DROP TABLE IF EXISTS test.postgresql_replica")
create_materialized_table(
ip=started_cluster.postgres_ip, port=started_cluster.postgres_port
)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
while int(result) != 10:
time.sleep(0.2)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT 10 + number, 10 + number from numbers(10)"
)
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT 20 + number, 20 + number from numbers(10)"
)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
while int(result) != 30:
time.sleep(0.2)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT 30 + number, 30 + number from numbers(10)"
)
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT 40 + number, 40 + number from numbers(10)"
)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
while int(result) != 50:
time.sleep(0.2)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;")
cursor.execute("DROP TABLE postgresql_replica;")
postgresql_replica_check_result(result, True)
instance.query(f"DROP TABLE test.postgresql_replica SYNC")
@pytest.mark.timeout(320)
def test_replicating_delete_queries(started_cluster):
conn = get_postgres_conn(
ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
database=True,
)
cursor = conn.cursor()
create_postgres_table(cursor, "postgresql_replica")
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)"
)
instance.query("DROP TABLE IF EXISTS test.postgresql_replica")
create_materialized_table(
ip=started_cluster.postgres_ip, port=started_cluster.postgres_port
)
result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;")
while postgresql_replica_check_result(result) == False:
time.sleep(0.2)
result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;")
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, 50 + number from numbers(50)"
)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
while int(result) != 100:
time.sleep(0.5)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
cursor.execute("DELETE FROM postgresql_replica WHERE key > 49;")
result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;")
while postgresql_replica_check_result(result) == False:
time.sleep(0.5)
result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;")
cursor.execute("DROP TABLE postgresql_replica;")
postgresql_replica_check_result(result, True)
@pytest.mark.timeout(320)
def test_replicating_update_queries(started_cluster):
conn = get_postgres_conn(
ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
database=True,
)
cursor = conn.cursor()
create_postgres_table(cursor, "postgresql_replica")
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT number, number + 10 from numbers(50)"
)
instance.query("DROP TABLE IF EXISTS test.postgresql_replica")
create_materialized_table(
ip=started_cluster.postgres_ip, port=started_cluster.postgres_port
)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
while int(result) != 50:
time.sleep(0.2)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
cursor.execute("UPDATE postgresql_replica SET value = value - 10;")
result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;")
while postgresql_replica_check_result(result) == False:
time.sleep(0.5)
result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;")
cursor.execute("DROP TABLE postgresql_replica;")
postgresql_replica_check_result(result, True)
@pytest.mark.timeout(320)
def test_resume_from_written_version(started_cluster):
conn = get_postgres_conn(
ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
database=True,
)
cursor = conn.cursor()
create_postgres_table(cursor, "postgresql_replica")
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT number, number + 10 from numbers(50)"
)
instance.query("DROP TABLE IF EXISTS test.postgresql_replica")
create_materialized_table(
ip=started_cluster.postgres_ip, port=started_cluster.postgres_port
)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
while int(result) != 50:
time.sleep(0.2)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, 50 + number from numbers(50)"
)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
while int(result) != 100:
time.sleep(0.2)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
instance.query("DETACH TABLE test.postgresql_replica")
cursor.execute("DELETE FROM postgresql_replica WHERE key > 49;")
cursor.execute("UPDATE postgresql_replica SET value = value - 10;")
instance.query("ATTACH TABLE test.postgresql_replica")
result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;")
while postgresql_replica_check_result(result) == False:
time.sleep(0.5)
result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;")
cursor.execute("DROP TABLE postgresql_replica;")
postgresql_replica_check_result(result, True)
@pytest.mark.timeout(320)
def test_many_replication_messages(started_cluster):
conn = get_postgres_conn(
ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
database=True,
)
cursor = conn.cursor()
create_postgres_table(cursor, "postgresql_replica")
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000)"
)
instance.query("DROP TABLE IF EXISTS test.postgresql_replica")
create_materialized_table(
ip=started_cluster.postgres_ip, port=started_cluster.postgres_port
)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
while int(result) != 100000:
time.sleep(0.2)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
print("SYNC OK")
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000, 100000)"
)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
while int(result) != 200000:
time.sleep(1)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
print("INSERT OK")
result = instance.query("SELECT key FROM test.postgresql_replica ORDER BY key;")
expected = instance.query("SELECT number from numbers(200000)")
assert result == expected
cursor.execute("UPDATE postgresql_replica SET value = key + 1 WHERE key < 100000;")
result = instance.query(
"SELECT key FROM test.postgresql_replica WHERE value = key + 1 ORDER BY key;"
)
expected = instance.query("SELECT number from numbers(100000)")
while result != expected:
time.sleep(1)
result = instance.query(
"SELECT key FROM test.postgresql_replica WHERE value = key + 1 ORDER BY key;"
)
print("UPDATE OK")
cursor.execute("DELETE FROM postgresql_replica WHERE key % 2 = 1;")
cursor.execute("DELETE FROM postgresql_replica WHERE key != value;")
result = instance.query(
"SELECT count() FROM (SELECT * FROM test.postgresql_replica);"
)
while int(result) != 50000:
time.sleep(1)
result = instance.query(
"SELECT count() FROM (SELECT * FROM test.postgresql_replica);"
)
print("DELETE OK")
cursor.execute("DROP TABLE postgresql_replica;")
@pytest.mark.timeout(320)
def test_connection_loss(started_cluster):
conn = get_postgres_conn(
ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
database=True,
)
cursor = conn.cursor()
create_postgres_table(cursor, "postgresql_replica")
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)"
)
instance.query("DROP TABLE IF EXISTS test.postgresql_replica")
create_materialized_table(
ip=started_cluster.postgres_ip, port=started_cluster.postgres_port
)
i = 50
while i < 100000:
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT {} + number, number from numbers(10000)".format(
i
)
)
i += 10000
started_cluster.pause_container("postgres1")
result = instance.query("SELECT count() FROM test.postgresql_replica;")
print(int(result))
time.sleep(6)
started_cluster.unpause_container("postgres1")
result = instance.query("SELECT count() FROM test.postgresql_replica;")
while int(result) < 100050:
time.sleep(1)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
cursor.execute("DROP TABLE postgresql_replica;")
assert int(result) == 100050
@pytest.mark.timeout(320)
def test_clickhouse_restart(started_cluster):
pytest.skip("Temporary disabled (FIXME)")
conn = get_postgres_conn(
ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
database=True,
)
cursor = conn.cursor()
create_postgres_table(cursor, "postgresql_replica")
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)"
)
instance.query("DROP TABLE IF EXISTS test.postgresql_replica")
create_materialized_table(
ip=started_cluster.postgres_ip, port=started_cluster.postgres_port
)
i = 50
while i < 100000:
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT {} + number, number from numbers(10000)".format(
i
)
)
i += 10000
instance.restart_clickhouse()
result = instance.query("SELECT count() FROM test.postgresql_replica;")
while int(result) < 100050:
time.sleep(1)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
cursor.execute("DROP TABLE postgresql_replica;")
print(result)
assert int(result) == 100050
def test_rename_table(started_cluster):
conn = get_postgres_conn(
ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
database=True,
)
cursor = conn.cursor()
create_postgres_table(cursor, "postgresql_replica")
instance.query("DROP TABLE IF EXISTS test.postgresql_replica")
create_materialized_table(
ip=started_cluster.postgres_ip, port=started_cluster.postgres_port
)
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(25)"
)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
while int(result) != 25:
time.sleep(0.5)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
instance.query(
"RENAME TABLE test.postgresql_replica TO test.postgresql_replica_renamed"
)
assert (
int(instance.query("SELECT count() FROM test.postgresql_replica_renamed;"))
== 25
)
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(25, 25)"
)
result = instance.query("SELECT count() FROM test.postgresql_replica_renamed;")
while int(result) != 50:
time.sleep(0.5)
result = instance.query("SELECT count() FROM test.postgresql_replica_renamed;")
result = instance.query(
"SELECT * FROM test.postgresql_replica_renamed ORDER BY key;"
)
postgresql_replica_check_result(result, True)
cursor.execute("DROP TABLE postgresql_replica;")
instance.query("DROP TABLE IF EXISTS test.postgresql_replica_renamed")
def test_virtual_columns(started_cluster):
conn = get_postgres_conn(
ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
database=True,
)
cursor = conn.cursor()
create_postgres_table(cursor, "postgresql_replica")
instance.query("DROP TABLE IF EXISTS test.postgresql_replica")
create_materialized_table(
ip=started_cluster.postgres_ip, port=started_cluster.postgres_port
)
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(10)"
)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
while int(result) != 10:
time.sleep(0.5)
result = instance.query("SELECT count() FROM test.postgresql_replica;")
# just check that it works, no check with `expected` because _version is taken as LSN, which will be different each time.
result = instance.query(
"SELECT key, value, _sign, _version FROM test.postgresql_replica;"
)
print(result)
cursor.execute("DROP TABLE postgresql_replica;")
instance.query(f"DROP TABLE test.postgresql_replica SYNC")
def test_abrupt_connection_loss_while_heavy_replication(started_cluster):
instance.query("DROP DATABASE IF EXISTS test_database")
conn = get_postgres_conn(
ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
database=True,
)
cursor = conn.cursor()
table_name = "postgresql_replica"
create_postgres_table(cursor, table_name)
instance.query(f"DROP TABLE IF EXISTS test.{table_name}")
create_materialized_table(
ip=started_cluster.postgres_ip, port=started_cluster.postgres_port
)
for i in range(len(queries)):
query = queries[i].format(table_name)
cursor.execute(query)
print("query {}".format(query.format(table_name)))
started_cluster.pause_container("postgres1")
result = instance.query("SELECT count() FROM test.postgresql_replica")
print(result) # Just debug
started_cluster.unpause_container("postgres1")
check_tables_are_synchronized("postgresql_replica")
result = instance.query("SELECT count() FROM test.postgresql_replica")
print(result) # Just debug
instance.query(f"DROP TABLE test.postgresql_replica SYNC")
def test_abrupt_server_restart_while_heavy_replication(started_cluster):
# FIXME (kssenii) temporary disabled
if instance.is_built_with_sanitizer():
pytest.skip("Temporary disabled (FIXME)")
conn = get_postgres_conn(
ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
database=True,
)
cursor = conn.cursor()
table_name = "postgresql_replica_697"
create_postgres_table(cursor, table_name)
instance.query(f"INSERT INTO postgres_database.{table_name} SELECT -1, 1")
instance.query(f"DROP TABLE IF EXISTS test.{table_name} SYNC")
create_materialized_table(
ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
table_name=table_name,
)
n = 1
while int(instance.query(f"select count() from test.{table_name}")) != 1:
sleep(1)
n += 1
if n > 10:
break
for query in queries:
cursor.execute(query.format(table_name))
print("query {}".format(query.format(table_name)))
instance.restart_clickhouse()
result = instance.query(f"SELECT count() FROM test.{table_name}")
print(result) # Just debug
check_tables_are_synchronized(table_name)
result = instance.query(f"SELECT count() FROM test.{table_name}")
print(result) # Just debug
instance.query(f"DROP TABLE test.{table_name} SYNC")
def test_drop_table_immediately(started_cluster):
conn = get_postgres_conn(
ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
database=True,
)
cursor = conn.cursor()
create_postgres_table(cursor, "postgresql_replica")
instance.query(
"INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000)"
)
instance.query("DROP TABLE IF EXISTS test.postgresql_replica")
create_materialized_table(
ip=started_cluster.postgres_ip, port=started_cluster.postgres_port
)
instance.query("DROP TABLE test.postgresql_replica")
create_materialized_table(
ip=started_cluster.postgres_ip, port=started_cluster.postgres_port
)
check_tables_are_synchronized("postgresql_replica")
instance.query(f"DROP TABLE test.postgresql_replica SYNC")
if __name__ == "__main__":
cluster.start()
input("Cluster created, press any key to destroy...")
cluster.shutdown()

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