Merge remote-tracking branch 'upstream/master' into HEAD

This commit is contained in:
Anton Popov 2020-09-16 21:07:35 +03:00
commit 8e9828d2ff
231 changed files with 5043 additions and 3251 deletions

View File

@ -28,7 +28,7 @@ elseif (COMPILER_CLANG)
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fchar8_t")
endif ()
else ()
set (CLANG_MINIMUM_VERSION 8)
set (CLANG_MINIMUM_VERSION 9)
if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS ${CLANG_MINIMUM_VERSION})
message (FATAL_ERROR "Clang version must be at least ${CLANG_MINIMUM_VERSION}.")
endif ()

View File

@ -6,7 +6,7 @@ toc_title: ANSI Compatibility
# ANSI SQL Compatibility of ClickHouse SQL Dialect {#ansi-sql-compatibility-of-clickhouse-sql-dialect}
!!! note "Note"
This article relies on Table 38, “Feature taxonomy and definition for mandatory features”, Annex F of ISO/IEC CD 9075-2:2013.
This article relies on Table 38, “Feature taxonomy and definition for mandatory features”, Annex F of [ISO/IEC CD 9075-2:2011](https://www.iso.org/obp/ui/#iso:std:iso-iec:9075:-2:ed-4:v1:en:sec:8).
## Differences in Behaviour {#differences-in-behaviour}
@ -77,6 +77,16 @@ The following table lists cases when query feature works in ClickHouse, but beha
| E071-05 | Columns combined via table operators need not have exactly the same data type | Yes{.text-success} | |
| E071-06 | Table operators in subqueries | Yes{.text-success} | |
| **E081** | **Basic privileges** | **Partial**{.text-warning} | Work in progress |
| E081-01 | SELECT privilege at the table level | | |
| E081-02 | DELETE privilege | | |
| E081-03 | INSERT privilege at the table level | | |
| E081-04 | UPDATE privilege at the table level | | |
| E081-05 | UPDATE privilege at the column level | | |
| E081-06 | REFERENCES privilege at the table level | | |
| E081-07 | REFERENCES privilege at the column level | | |
| E081-08 | WITH GRANT OPTION | | |
| E081-09 | USAGE privilege | | |
| E081-10 | EXECUTE privilege | | |
| **E091** | **Set functions** | **Yes**{.text-success} | |
| E091-01 | AVG | Yes{.text-success} | |
| E091-02 | COUNT | Yes{.text-success} | |
@ -169,6 +179,7 @@ The following table lists cases when query feature works in ClickHouse, but beha
| **F471** | **Scalar subquery values** | **Yes**{.text-success} | |
| **F481** | **Expanded NULL predicate** | **Yes**{.text-success} | |
| **F812** | **Basic flagging** | **No**{.text-danger} | |
| **S011** | **Distinct data types** | | |
| **T321** | **Basic SQL-invoked routines** | **No**{.text-danger} | |
| T321-01 | User-defined functions with no overloading | No{.text-danger} | |
| T321-02 | User-defined stored procedures with no overloading | No{.text-danger} | |

View File

@ -246,7 +246,7 @@ Installing unixODBC and the ODBC driver for PostgreSQL:
$ sudo apt-get install -y unixodbc odbcinst odbc-postgresql
```
Configuring `/etc/odbc.ini` (or `~/.odbc.ini`):
Configuring `/etc/odbc.ini` (or `~/.odbc.ini` if you signed in under a user that runs ClickHouse):
``` text
[DEFAULT]
@ -321,7 +321,7 @@ You may need to edit `odbc.ini` to specify the full path to the library with the
Ubuntu OS.
Installing the driver: :
Installing the ODBC driver for connecting to MS SQL:
``` bash
$ sudo apt-get install tdsodbc freetds-bin sqsh
@ -329,7 +329,7 @@ $ sudo apt-get install tdsodbc freetds-bin sqsh
Configuring the driver:
``` bash
```bash
$ cat /etc/freetds/freetds.conf
...
@ -339,8 +339,11 @@ Configuring the driver:
tds version = 7.0
client charset = UTF-8
# test TDS connection
$ sqsh -S MSSQL -D database -U user -P password
$ cat /etc/odbcinst.ini
...
[FreeTDS]
Description = FreeTDS
@ -349,8 +352,8 @@ Configuring the driver:
FileUsage = 1
UsageCount = 5
$ cat ~/.odbc.ini
...
$ cat /etc/odbc.ini
# $ cat ~/.odbc.ini # if you signed in under a user that runs ClickHouse
[MSSQL]
Description = FreeTDS
@ -360,8 +363,15 @@ Configuring the driver:
UID = test
PWD = test
Port = 1433
# (optional) test ODBC connection (to use isql-tool install the [unixodbc](https://packages.debian.org/sid/unixodbc)-package)
$ isql -v MSSQL "user" "password"
```
Remarks:
- to determine the earliest TDS version that is supported by a particular SQL Server version, refer to the product documentation or look at [MS-TDS Product Behavior](https://docs.microsoft.com/en-us/openspecs/windows_protocols/ms-tds/135d0ebe-5c4c-4a94-99bf-1811eccb9f4a)
Configuring the dictionary in ClickHouse:
``` xml

View File

@ -111,7 +111,7 @@ dictHas('dict_name', id_expr)
**Parameters**
- `dict_name` — Name of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).
- `id_expr` — Key value. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a [UInt64](../../sql-reference/data-types/int-uint.md)-type value.
- `id_expr` — Key value. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a [UInt64](../../sql-reference/data-types/int-uint.md) or [Tuple](../../sql-reference/data-types/tuple.md)-type value depending on the dictionary configuration.
**Returned value**
@ -189,8 +189,8 @@ dictGet[Type]OrDefault('dict_name', 'attr_name', id_expr, default_value_expr)
- `dict_name` — Name of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).
- `attr_name` — Name of the column of the dictionary. [String literal](../../sql-reference/syntax.md#syntax-string-literal).
- `id_expr` — Key value. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a [UInt64](../../sql-reference/data-types/int-uint.md)-type value.
- `default_value_expr` — Value which is returned if the dictionary doesnt contain a row with the `id_expr` key. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a value in the data type configured for the `attr_name` attribute.
- `id_expr` — Key value. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning a [UInt64](../../sql-reference/data-types/int-uint.md) or [Tuple](../../sql-reference/data-types/tuple.md)-type value depending on the dictionary configuration.
- `default_value_expr` — Value returned if the dictionary doesnt contain a row with the `id_expr` key. [Expression](../../sql-reference/syntax.md#syntax-expressions) returning the value in the data type configured for the `attr_name` attribute.
**Returned value**

View File

@ -103,7 +103,7 @@ dictHas('dict_name', id)
**Параметры**
- `dict_name` — имя словаря. [Строковый литерал](../syntax.md#syntax-string-literal).
- `id_expr` — значение ключа словаря. [Выражение](../syntax.md#syntax-expressions), возвращающее значение типа [UInt64](../../sql-reference/functions/ext-dict-functions.md).
- `id_expr` — значение ключа словаря. [Выражение](../syntax.md#syntax-expressions), возвращающее значение типа [UInt64](../../sql-reference/functions/ext-dict-functions.md) или [Tuple](../../sql-reference/functions/ext-dict-functions.md) в зависимости от конфигурации словаря.
**Возвращаемое значение**
@ -179,7 +179,7 @@ dictGet[Type]OrDefault('dict_name', 'attr_name', id_expr, default_value_expr)
- `dict_name` — имя словаря. [Строковый литерал](../syntax.md#syntax-string-literal).
- `attr_name` — имя столбца словаря. [Строковый литерал](../syntax.md#syntax-string-literal).
- `id_expr` — значение ключа словаря. [Выражение](../syntax.md#syntax-expressions), возвращающее значение типа [UInt64](../../sql-reference/functions/ext-dict-functions.md).
- `id_expr` — значение ключа словаря. [Выражение](../syntax.md#syntax-expressions), возвращающее значение типа [UInt64](../../sql-reference/functions/ext-dict-functions.md) или [Tuple](../../sql-reference/functions/ext-dict-functions.md) в зависимости от конфигурации словаря.
- `default_value_expr` — значение, возвращаемое в том случае, когда словарь не содержит строки с заданным ключом `id_expr`. [Выражение](../syntax.md#syntax-expressions) возвращающее значение с типом данных, сконфигурированным для атрибута `attr_name`.
**Возвращаемое значение**

View File

@ -186,7 +186,7 @@ void AccessControlManager::addUsersConfigStorage(
{
if (auto users_config_storage = typeid_cast<std::shared_ptr<UsersConfigAccessStorage>>(storage))
{
if (users_config_storage->getStoragePath() == users_config_path_)
if (users_config_storage->isPathEqual(users_config_path_))
return;
}
}
@ -229,7 +229,7 @@ void AccessControlManager::addDiskStorage(const String & storage_name_, const St
{
if (auto disk_storage = typeid_cast<std::shared_ptr<DiskAccessStorage>>(storage))
{
if (disk_storage->isStoragePathEqual(directory_))
if (disk_storage->isPathEqual(directory_))
{
if (readonly_)
disk_storage->setReadOnly(readonly_);

View File

@ -33,6 +33,9 @@
#include <Interpreters/InterpreterShowGrantsQuery.h>
#include <Common/quoteString.h>
#include <Core/Defines.h>
#include <Poco/JSON/JSON.h>
#include <Poco/JSON/Object.h>
#include <Poco/JSON/Stringifier.h>
#include <boost/range/adaptor/map.hpp>
#include <boost/range/algorithm/copy.hpp>
#include <boost/range/algorithm_ext/push_back.hpp>
@ -342,9 +345,22 @@ DiskAccessStorage::~DiskAccessStorage()
}
bool DiskAccessStorage::isStoragePathEqual(const String & directory_path_) const
String DiskAccessStorage::getStorageParamsJSON() const
{
return getStoragePath() == makeDirectoryPathCanonical(directory_path_);
std::lock_guard lock{mutex};
Poco::JSON::Object json;
json.set("path", directory_path);
if (readonly)
json.set("readonly", readonly.load());
std::ostringstream oss;
Poco::JSON::Stringifier::stringify(json, oss);
return oss.str();
}
bool DiskAccessStorage::isPathEqual(const String & directory_path_) const
{
return getPath() == makeDirectoryPathCanonical(directory_path_);
}

View File

@ -18,12 +18,13 @@ public:
~DiskAccessStorage() override;
const char * getStorageType() const override { return STORAGE_TYPE; }
String getStorageParamsJSON() const override;
String getStoragePath() const override { return directory_path; }
bool isStoragePathEqual(const String & directory_path_) const;
String getPath() const { return directory_path; }
bool isPathEqual(const String & directory_path_) const;
void setReadOnly(bool readonly_) { readonly = readonly_; }
bool isStorageReadOnly() const override { return readonly; }
bool isReadOnly() const { return readonly; }
private:
std::optional<UUID> findImpl(EntityType type, const String & name) const override;

View File

@ -25,8 +25,9 @@ public:
/// Returns the name of this storage.
const String & getStorageName() const { return storage_name; }
virtual const char * getStorageType() const = 0;
virtual String getStoragePath() const { return {}; }
virtual bool isStorageReadOnly() const { return false; }
/// Returns a JSON with the parameters of the storage. It's up to the storage type to fill the JSON.
virtual String getStorageParamsJSON() const { return "{}"; }
using EntityType = IAccessEntity::Type;
using EntityTypeInfo = IAccessEntity::TypeInfo;

View File

@ -10,6 +10,9 @@
#include <Core/Settings.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/MD5Engine.h>
#include <Poco/JSON/JSON.h>
#include <Poco/JSON/Object.h>
#include <Poco/JSON/Stringifier.h>
#include <common/logger_useful.h>
#include <boost/range/algorithm/copy.hpp>
#include <boost/range/adaptor/map.hpp>
@ -482,12 +485,29 @@ UsersConfigAccessStorage::UsersConfigAccessStorage(const String & storage_name_,
UsersConfigAccessStorage::~UsersConfigAccessStorage() = default;
String UsersConfigAccessStorage::getStoragePath() const
String UsersConfigAccessStorage::getStorageParamsJSON() const
{
std::lock_guard lock{load_mutex};
Poco::JSON::Object json;
if (!path.empty())
json.set("path", path);
std::ostringstream oss;
Poco::JSON::Stringifier::stringify(json, oss);
return oss.str();
}
String UsersConfigAccessStorage::getPath() const
{
std::lock_guard lock{load_mutex};
return path;
}
bool UsersConfigAccessStorage::isPathEqual(const String & path_) const
{
return getPath() == path_;
}
void UsersConfigAccessStorage::setConfig(const Poco::Util::AbstractConfiguration & config)
{

View File

@ -26,8 +26,10 @@ public:
~UsersConfigAccessStorage() override;
const char * getStorageType() const override { return STORAGE_TYPE; }
String getStoragePath() const override;
bool isStorageReadOnly() const override { return true; }
String getStorageParamsJSON() const override;
String getPath() const;
bool isPathEqual(const String & path_) const;
void setConfig(const Poco::Util::AbstractConfiguration & config);

View File

@ -63,7 +63,7 @@ Block IBlockInputStream::read()
if (enabled_extremes)
updateExtremes(res);
if (limits.mode == LIMITS_CURRENT && !limits.size_limits.check(info.rows, info.bytes, "result", ErrorCodes::TOO_MANY_ROWS_OR_BYTES))
if (limits.mode == LimitsMode::LIMITS_CURRENT && !limits.size_limits.check(info.rows, info.bytes, "result", ErrorCodes::TOO_MANY_ROWS_OR_BYTES))
limit_exceeded_need_break = true;
if (quota)
@ -209,11 +209,11 @@ void IBlockInputStream::checkQuota(Block & block)
{
switch (limits.mode)
{
case LIMITS_TOTAL:
case LimitsMode::LIMITS_TOTAL:
/// Checked in `progress` method.
break;
case LIMITS_CURRENT:
case LimitsMode::LIMITS_CURRENT:
{
UInt64 total_elapsed = info.total_stopwatch.elapsedNanoseconds();
quota->used({Quota::RESULT_ROWS, block.rows()}, {Quota::RESULT_BYTES, block.bytes()}, {Quota::EXECUTION_TIME, total_elapsed - prev_elapsed});
@ -242,7 +242,7 @@ void IBlockInputStream::progressImpl(const Progress & value)
/** Check the restrictions on the amount of data to read, the speed of the query, the quota on the amount of data to read.
* NOTE: Maybe it makes sense to have them checked directly in ProcessList?
*/
if (limits.mode == LIMITS_TOTAL)
if (limits.mode == LimitsMode::LIMITS_TOTAL)
{
if (!limits.size_limits.check(total_rows_estimate, progress.read_bytes, "rows to read",
ErrorCodes::TOO_MANY_ROWS, ErrorCodes::TOO_MANY_BYTES))
@ -262,7 +262,7 @@ void IBlockInputStream::progressImpl(const Progress & value)
limits.speed_limits.throttle(progress.read_rows, progress.read_bytes, total_rows, total_elapsed_microseconds);
if (quota && limits.mode == LIMITS_TOTAL)
if (quota && limits.mode == LimitsMode::LIMITS_TOTAL)
quota->used({Quota::READ_ROWS, value.read_rows}, {Quota::READ_BYTES, value.read_bytes});
}

View File

@ -5,6 +5,7 @@
#include <DataStreams/IBlockStream_fwd.h>
#include <DataStreams/SizeLimits.h>
#include <DataStreams/ExecutionSpeedLimits.h>
#include <DataStreams/StreamLocalLimits.h>
#include <IO/Progress.h>
#include <Storages/TableLockHolder.h>
#include <Common/TypePromotion.h>
@ -173,38 +174,13 @@ public:
bool isCancelled() const;
bool isCancelledOrThrowIfKilled() const;
/** What limitations and quotas should be checked.
* LIMITS_CURRENT - checks amount of data returned by current stream only (BlockStreamProfileInfo is used for check).
* Currently it is used in root streams to check max_result_{rows,bytes} limits.
* LIMITS_TOTAL - checks total amount of read data from leaf streams (i.e. data read from disk and remote servers).
* It is checks max_{rows,bytes}_to_read in progress handler and use info from ProcessListElement::progress_in for this.
* Currently this check is performed only in leaf streams.
*/
enum LimitsMode
{
LIMITS_CURRENT,
LIMITS_TOTAL,
};
/// It is a subset of limitations from Limits.
struct LocalLimits
{
LimitsMode mode = LIMITS_CURRENT;
SizeLimits size_limits;
ExecutionSpeedLimits speed_limits;
OverflowMode timeout_overflow_mode = OverflowMode::THROW;
};
/** Set limitations that checked on each block. */
virtual void setLimits(const LocalLimits & limits_)
virtual void setLimits(const StreamLocalLimits & limits_)
{
limits = limits_;
}
const LocalLimits & getLimits() const
const StreamLocalLimits & getLimits() const
{
return limits;
}
@ -268,7 +244,7 @@ private:
/// Limitations and quotas.
LocalLimits limits;
StreamLocalLimits limits;
std::shared_ptr<const EnabledQuota> quota; /// If nullptr - the quota is not used.
UInt64 prev_elapsed = 0;

View File

@ -0,0 +1,33 @@
#pragma once
#include <DataStreams/SizeLimits.h>
#include <DataStreams/ExecutionSpeedLimits.h>
namespace DB
{
/** What limitations and quotas should be checked.
* LIMITS_CURRENT - checks amount of data returned by current stream only (BlockStreamProfileInfo is used for check).
* Currently it is used in root streams to check max_result_{rows,bytes} limits.
* LIMITS_TOTAL - checks total amount of read data from leaf streams (i.e. data read from disk and remote servers).
* It is checks max_{rows,bytes}_to_read in progress handler and use info from ProcessListElement::progress_in for this.
* Currently this check is performed only in leaf streams.
*/
enum class LimitsMode
{
LIMITS_CURRENT,
LIMITS_TOTAL,
};
/// It is a subset of limitations from Limits.
struct StreamLocalLimits
{
LimitsMode mode = LimitsMode::LIMITS_CURRENT;
SizeLimits size_limits;
ExecutionSpeedLimits speed_limits;
OverflowMode timeout_overflow_mode = OverflowMode::THROW;
};
}

View File

@ -1071,6 +1071,37 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
executeSubqueriesInSetsAndJoins(query_plan, subqueries_for_sets);
}
static StreamLocalLimits getLimitsForStorage(const Settings & settings, const SelectQueryOptions & options)
{
StreamLocalLimits limits;
limits.mode = LimitsMode::LIMITS_TOTAL;
limits.size_limits = SizeLimits(settings.max_rows_to_read, settings.max_bytes_to_read,
settings.read_overflow_mode);
limits.speed_limits.max_execution_time = settings.max_execution_time;
limits.timeout_overflow_mode = settings.timeout_overflow_mode;
/** Quota and minimal speed restrictions are checked on the initiating server of the request, and not on remote servers,
* because the initiating server has a summary of the execution of the request on all servers.
*
* But limits on data size to read and maximum execution time are reasonable to check both on initiator and
* additionally on each remote server, because these limits are checked per block of data processed,
* and remote servers may process way more blocks of data than are received by initiator.
*
* The limits to throttle maximum execution speed is also checked on all servers.
*/
if (options.to_stage == QueryProcessingStage::Complete)
{
limits.speed_limits.min_execution_rps = settings.min_execution_speed;
limits.speed_limits.min_execution_bps = settings.min_execution_speed_bytes;
}
limits.speed_limits.max_execution_rps = settings.max_execution_speed;
limits.speed_limits.max_execution_bps = settings.max_execution_speed_bytes;
limits.speed_limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed;
return limits;
}
void InterpreterSelectQuery::executeFetchColumns(
QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan,
const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere)
@ -1409,12 +1440,18 @@ void InterpreterSelectQuery::executeFetchColumns(
query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage, metadata_snapshot);
}
auto read_step = std::make_unique<ReadFromStorageStep>(
table_lock, metadata_snapshot, options, storage,
required_columns, query_info, context, processing_stage, max_block_size, max_streams);
StreamLocalLimits limits;
std::shared_ptr<const EnabledQuota> quota;
read_step->setStepDescription("Read from " + storage->getName());
query_plan.addStep(std::move(read_step));
/// Set the limits and quota for reading data, the speed and time of the query.
if (!options.ignore_limits)
limits = getLimitsForStorage(settings, options);
if (!options.ignore_quota && (options.to_stage == QueryProcessingStage::Complete))
quota = context->getQuota();
storage->read(query_plan, table_lock, metadata_snapshot, limits, std::move(quota),
required_columns, query_info, context, processing_stage, max_block_size, max_streams);
}
else
throw Exception("Logical error in InterpreterSelectQuery: nowhere to read", ErrorCodes::LOGICAL_ERROR);

View File

@ -17,6 +17,7 @@ limitations under the License. */
#include <Access/AccessFlags.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataStreams/OneBlockInputStream.h>
#include <DataStreams/StreamLocalLimits.h>
namespace DB
@ -76,8 +77,8 @@ BlockIO InterpreterWatchQuery::execute()
/// Constraints on the result, the quota on the result, and also callback for progress.
if (IBlockInputStream * stream = dynamic_cast<IBlockInputStream *>(streams[0].get()))
{
IBlockInputStream::LocalLimits limits;
limits.mode = IBlockInputStream::LIMITS_CURRENT;
StreamLocalLimits limits;
limits.mode = LimitsMode::LIMITS_CURRENT;
limits.size_limits.max_rows = settings.max_result_rows;
limits.size_limits.max_bytes = settings.max_result_bytes;
limits.size_limits.overflow_mode = settings.result_overflow_mode;

View File

@ -136,6 +136,22 @@ void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool
setupState(thread_group_);
}
inline UInt64 time_in_nanoseconds(std::chrono::time_point<std::chrono::system_clock> timepoint)
{
return std::chrono::duration_cast<std::chrono::nanoseconds>(timepoint.time_since_epoch()).count();
}
inline UInt64 time_in_microseconds(std::chrono::time_point<std::chrono::system_clock> timepoint)
{
return std::chrono::duration_cast<std::chrono::microseconds>(timepoint.time_since_epoch()).count();
}
inline UInt64 time_in_seconds(std::chrono::time_point<std::chrono::system_clock> timepoint)
{
return std::chrono::duration_cast<std::chrono::seconds>(timepoint.time_since_epoch()).count();
}
void ThreadStatus::initPerformanceCounters()
{
performance_counters_finalized = false;
@ -146,9 +162,13 @@ void ThreadStatus::initPerformanceCounters()
memory_tracker.resetCounters();
memory_tracker.setDescription("(for thread)");
query_start_time_nanoseconds = getCurrentTimeNanoseconds();
query_start_time = time(nullptr);
query_start_time_microseconds = getCurrentTimeMicroseconds();
// query_start_time_{microseconds, nanoseconds} are all constructed from the same time point
// to ensure that they are all equal upto the precision of a second.
const auto now = std::chrono::system_clock::now();
query_start_time_nanoseconds = time_in_nanoseconds(now);
query_start_time = time_in_seconds(now);
query_start_time_microseconds = time_in_microseconds(now);
++queries_started;
*last_rusage = RUsageCounters::current(query_start_time_nanoseconds);

View File

@ -250,18 +250,26 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
}
else if (const auto * asterisk_pattern = child->as<ASTColumnsMatcher>())
{
bool first_table = true;
for (const auto & table : tables_with_columns)
if (asterisk_pattern->column_list)
{
for (const auto & column : table.columns)
for (const auto & ident : asterisk_pattern->column_list->children)
node.children.emplace_back(ident->clone());
}
else
{
bool first_table = true;
for (const auto & table : tables_with_columns)
{
if (asterisk_pattern->isColumnMatching(column.name) && (first_table || !data.join_using_columns.count(column.name)))
for (const auto & column : table.columns)
{
addIdentifier(node.children, table.table, column.name, AsteriskSemantic::getAliases(*asterisk_pattern));
if (asterisk_pattern->isColumnMatching(column.name) && (first_table || !data.join_using_columns.count(column.name)))
{
addIdentifier(node.children, table.table, column.name, AsteriskSemantic::getAliases(*asterisk_pattern));
}
}
}
first_table = false;
first_table = false;
}
}
// ColumnsMatcher's transformers start to appear at child 1
for (auto it = asterisk_pattern->children.begin() + 1; it != asterisk_pattern->children.end(); ++it)

View File

@ -182,6 +182,16 @@ static void logException(Context & context, QueryLogElement & elem)
elem.exception, context.getClientInfo().current_address.toString(), joinLines(elem.query), elem.stack_trace);
}
inline UInt64 time_in_microseconds(std::chrono::time_point<std::chrono::system_clock> timepoint)
{
return std::chrono::duration_cast<std::chrono::microseconds>(timepoint.time_since_epoch()).count();
}
inline UInt64 time_in_seconds(std::chrono::time_point<std::chrono::system_clock> timepoint)
{
return std::chrono::duration_cast<std::chrono::seconds>(timepoint.time_since_epoch()).count();
}
static void onExceptionBeforeStart(const String & query_for_logging, Context & context, time_t current_time, UInt64 current_time_microseconds, ASTPtr ast)
{
@ -196,6 +206,9 @@ static void onExceptionBeforeStart(const String & query_for_logging, Context & c
elem.type = QueryLogElementType::EXCEPTION_BEFORE_START;
// all callers to onExceptionBeforeStart upstream construct the timespec for event_time and
// event_time_microseconds from the same timespec. So it can be assumed that both of these
// times are equal upto the precision of a second.
elem.event_time = current_time;
elem.query_start_time = current_time;
elem.query_start_time_microseconds = current_time_microseconds;
@ -251,8 +264,12 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
bool has_query_tail,
ReadBuffer * istr)
{
time_t current_time = time(nullptr);
UInt64 current_time_microseconds = getCurrentTimeMicroseconds();
// current_time and current_time_microseconds are both constructed from the same time point
// to ensure that both the times are equal upto the precision of a second.
const auto now = std::chrono::system_clock::now();
auto current_time = time_in_seconds(now);
auto current_time_microseconds = time_in_microseconds(now);
/// If we already executing query and it requires to execute internal query, than
/// don't replace thread context with given (it can be temporary). Otherwise, attach context to thread.
@ -382,10 +399,10 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
}
}
IBlockInputStream::LocalLimits limits;
StreamLocalLimits limits;
if (!interpreter->ignoreLimits())
{
limits.mode = IBlockInputStream::LIMITS_CURRENT;
limits.mode = LimitsMode::LIMITS_CURRENT;
limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode);
}

View File

@ -30,8 +30,12 @@ void ASTColumnsMatcher::updateTreeHashImpl(SipHash & hash_state) const
void ASTColumnsMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "COLUMNS" << (settings.hilite ? hilite_none : "") << "("
<< quoteString(original_pattern) << ")";
settings.ostr << (settings.hilite ? hilite_keyword : "") << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
if (column_list)
column_list->formatImpl(settings, state, frame);
else
settings.ostr << quoteString(original_pattern);
settings.ostr << ")";
for (ASTs::const_iterator it = children.begin() + 1; it != children.end(); ++it)
{
settings.ostr << ' ';

View File

@ -36,6 +36,8 @@ public:
bool isColumnMatching(const String & column_name) const;
void updateTreeHashImpl(SipHash & hash_state) const override;
ASTPtr column_list;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;

View File

@ -1188,6 +1188,7 @@ bool ParserAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword columns("COLUMNS");
ParserList columns_p(std::make_unique<ParserCompoundIdentifier>(), std::make_unique<ParserToken>(TokenType::Comma), false);
ParserStringLiteral regex;
if (!columns.ignore(pos, expected))
@ -1197,8 +1198,9 @@ bool ParserColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
return false;
++pos;
ASTPtr column_list;
ASTPtr regex_node;
if (!regex.parse(pos, regex_node, expected))
if (!columns_p.parse(pos, column_list, expected) && !regex.parse(pos, regex_node, expected))
return false;
if (pos->type != TokenType::ClosingRoundBracket)
@ -1206,8 +1208,17 @@ bool ParserColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
++pos;
auto res = std::make_shared<ASTColumnsMatcher>();
res->setPattern(regex_node->as<ASTLiteral &>().value.get<String>());
res->children.push_back(regex_node);
if (column_list)
{
res->column_list = column_list;
res->children.push_back(res->column_list);
}
else
{
res->setPattern(regex_node->as<ASTLiteral &>().value.get<String>());
res->children.push_back(regex_node);
}
ParserColumnsTransformers transformers_p;
ASTPtr transformer;
while (transformers_p.parse(pos, transformer, expected))

View File

@ -103,7 +103,7 @@ void PipelineExecutingBlockInputStream::setProcessListElement(QueryStatus * elem
pipeline->setProcessListElement(elem);
}
void PipelineExecutingBlockInputStream::setLimits(const IBlockInputStream::LocalLimits & limits_)
void PipelineExecutingBlockInputStream::setLimits(const StreamLocalLimits & limits_)
{
throwIfExecutionStarted(is_execution_started, "setLimits");

View File

@ -24,7 +24,7 @@ public:
/// Implement IBlockInputStream methods via QueryPipeline.
void setProgressCallback(const ProgressCallback & callback) final;
void setProcessListElement(QueryStatus * elem) final;
void setLimits(const LocalLimits & limits_) final;
void setLimits(const StreamLocalLimits & limits_) final;
void setQuota(const std::shared_ptr<const EnabledQuota> & quota_) final;
void addTotalRowsApprox(size_t value) final;

View File

@ -779,7 +779,7 @@ void Pipe::transform(const Transformer & transformer)
max_parallel_streams = std::max<size_t>(max_parallel_streams, output_ports.size());
}
void Pipe::setLimits(const ISourceWithProgress::LocalLimits & limits)
void Pipe::setLimits(const StreamLocalLimits & limits)
{
for (auto & processor : processors)
{

View File

@ -6,6 +6,8 @@
namespace DB
{
struct StreamLocalLimits;
class Pipe;
using Pipes = std::vector<Pipe>;
@ -94,7 +96,7 @@ public:
const Processors & getProcessors() const { return processors; }
/// Specify quotas and limits for every ISourceWithProgress.
void setLimits(const SourceWithProgress::LocalLimits & limits);
void setLimits(const StreamLocalLimits & limits);
void setQuota(const std::shared_ptr<const EnabledQuota> & quota);
/// Do not allow to change the table while the processors of pipe are alive.

View File

@ -56,8 +56,8 @@ void PartialSortingStep::transformPipeline(QueryPipeline & pipeline)
return std::make_shared<PartialSortingTransform>(header, sort_description, limit);
});
IBlockInputStream::LocalLimits limits;
limits.mode = IBlockInputStream::LIMITS_CURRENT;
StreamLocalLimits limits;
limits.mode = LimitsMode::LIMITS_CURRENT;
limits.size_limits = size_limits;
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr

View File

@ -13,8 +13,9 @@ namespace DB
ReadFromStorageStep::ReadFromStorageStep(
TableLockHolder table_lock_,
StorageMetadataPtr & metadata_snapshot_,
SelectQueryOptions options_,
StorageMetadataPtr metadata_snapshot_,
StreamLocalLimits & limits_,
std::shared_ptr<const EnabledQuota> quota_,
StoragePtr storage_,
const Names & required_columns_,
const SelectQueryInfo & query_info_,
@ -23,8 +24,9 @@ ReadFromStorageStep::ReadFromStorageStep(
size_t max_block_size_,
size_t max_streams_)
: table_lock(std::move(table_lock_))
, metadata_snapshot(metadata_snapshot_)
, options(std::move(options_))
, metadata_snapshot(std::move(metadata_snapshot_))
, limits(limits_)
, quota(std::move(quota_))
, storage(std::move(storage_))
, required_columns(required_columns_)
, query_info(query_info_)
@ -82,43 +84,10 @@ ReadFromStorageStep::ReadFromStorageStep(
/// Table lock is stored inside pipeline here.
pipeline->addTableLock(table_lock);
/// Set the limits and quota for reading data, the speed and time of the query.
{
const Settings & settings = context->getSettingsRef();
pipe.setLimits(limits);
IBlockInputStream::LocalLimits limits;
limits.mode = IBlockInputStream::LIMITS_TOTAL;
limits.size_limits = SizeLimits(settings.max_rows_to_read, settings.max_bytes_to_read, settings.read_overflow_mode);
limits.speed_limits.max_execution_time = settings.max_execution_time;
limits.timeout_overflow_mode = settings.timeout_overflow_mode;
/** Quota and minimal speed restrictions are checked on the initiating server of the request, and not on remote servers,
* because the initiating server has a summary of the execution of the request on all servers.
*
* But limits on data size to read and maximum execution time are reasonable to check both on initiator and
* additionally on each remote server, because these limits are checked per block of data processed,
* and remote servers may process way more blocks of data than are received by initiator.
*
* The limits to throttle maximum execution speed is also checked on all servers.
*/
if (options.to_stage == QueryProcessingStage::Complete)
{
limits.speed_limits.min_execution_rps = settings.min_execution_speed;
limits.speed_limits.min_execution_bps = settings.min_execution_speed_bytes;
}
limits.speed_limits.max_execution_rps = settings.max_execution_speed;
limits.speed_limits.max_execution_bps = settings.max_execution_speed_bytes;
limits.speed_limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed;
auto quota = context->getQuota();
if (!options.ignore_limits)
pipe.setLimits(limits);
if (!options.ignore_quota && (options.to_stage == QueryProcessingStage::Complete))
pipe.setQuota(quota);
}
if (quota)
pipe.setQuota(quota);
pipeline->init(std::move(pipe));

View File

@ -1,7 +1,7 @@
#include <Processors/QueryPlan/IQueryPlanStep.h>
#include <Core/QueryProcessingStage.h>
#include <Storages/TableLockHolder.h>
#include <Interpreters/SelectQueryOptions.h>
#include <DataStreams/StreamLocalLimits.h>
namespace DB
{
@ -16,14 +16,17 @@ struct SelectQueryInfo;
struct PrewhereInfo;
class EnabledQuota;
/// Reads from storage.
class ReadFromStorageStep : public IQueryPlanStep
{
public:
ReadFromStorageStep(
TableLockHolder table_lock,
StorageMetadataPtr & metadata_snapshot,
SelectQueryOptions options,
StorageMetadataPtr metadata_snapshot,
StreamLocalLimits & limits,
std::shared_ptr<const EnabledQuota> quota,
StoragePtr storage,
const Names & required_columns,
const SelectQueryInfo & query_info,
@ -43,7 +46,8 @@ public:
private:
TableLockHolder table_lock;
StorageMetadataPtr metadata_snapshot;
SelectQueryOptions options;
StreamLocalLimits limits;
std::shared_ptr<const EnabledQuota> quota;
StoragePtr storage;
const Names & required_columns;

View File

@ -32,7 +32,7 @@ public:
void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) { rows_before_limit.swap(counter); }
/// Implementation for methods from ISourceWithProgress.
void setLimits(const LocalLimits & limits_) final { stream->setLimits(limits_); }
void setLimits(const StreamLocalLimits & limits_) final { stream->setLimits(limits_); }
void setQuota(const std::shared_ptr<const EnabledQuota> & quota_) final { stream->setQuota(quota_); }
void setProcessListElement(QueryStatus * elem) final { stream->setProcessListElement(elem); }
void setProgressCallback(const ProgressCallback & callback) final { stream->setProgressCallback(callback); }

View File

@ -2,6 +2,7 @@
#include <Processors/ISource.h>
#include <DataStreams/IBlockInputStream.h>
#include <Common/Stopwatch.h>
#include <DataStreams/StreamLocalLimits.h>
namespace DB
{
@ -13,11 +14,8 @@ class ISourceWithProgress : public ISource
public:
using ISource::ISource;
using LocalLimits = IBlockInputStream::LocalLimits;
using LimitsMode = IBlockInputStream::LimitsMode;
/// Set limitations that checked on each chunk.
virtual void setLimits(const LocalLimits & limits_) = 0;
virtual void setLimits(const StreamLocalLimits & limits_) = 0;
/// Set the quota. If you set a quota on the amount of raw data,
/// then you should also set mode = LIMITS_TOTAL to LocalLimits with setLimits.
@ -47,10 +45,7 @@ public:
/// If enable_auto_progress flag is set, progress() will be automatically called on each generated chunk.
SourceWithProgress(Block header, bool enable_auto_progress);
using LocalLimits = IBlockInputStream::LocalLimits;
using LimitsMode = IBlockInputStream::LimitsMode;
void setLimits(const LocalLimits & limits_) final { limits = limits_; }
void setLimits(const StreamLocalLimits & limits_) final { limits = limits_; }
void setQuota(const std::shared_ptr<const EnabledQuota> & quota_) final { quota = quota_; }
void setProcessListElement(QueryStatus * elem) final { process_list_elem = elem; }
void setProgressCallback(const ProgressCallback & callback) final { progress_callback = callback; }
@ -63,7 +58,7 @@ protected:
void work() override;
private:
LocalLimits limits;
StreamLocalLimits limits;
std::shared_ptr<const EnabledQuota> quota;
ProgressCallback progress_callback;
QueryStatus * process_list_elem = nullptr;

View File

@ -18,7 +18,7 @@ void ProcessorProfileInfo::update(const Chunk & block)
bytes += block.bytes();
}
LimitsCheckingTransform::LimitsCheckingTransform(const Block & header_, LocalLimits limits_)
LimitsCheckingTransform::LimitsCheckingTransform(const Block & header_, StreamLocalLimits limits_)
: ISimpleTransform(header_, header_, false)
, limits(std::move(limits_))
{

View File

@ -4,7 +4,7 @@
#include <Poco/Timespan.h>
#include <Interpreters/ProcessList.h>
#include <DataStreams/IBlockOutputStream.h>
#include <DataStreams/StreamLocalLimits.h>
namespace DB
{
@ -26,10 +26,7 @@ class LimitsCheckingTransform : public ISimpleTransform
{
public:
using LocalLimits = IBlockInputStream::LocalLimits;
using LimitsMode = IBlockInputStream::LimitsMode;
LimitsCheckingTransform(const Block & header_, LocalLimits limits_);
LimitsCheckingTransform(const Block & header_, StreamLocalLimits limits_);
String getName() const override { return "LimitsCheckingTransform"; }
@ -39,7 +36,7 @@ protected:
void transform(Chunk & chunk) override;
private:
LocalLimits limits;
StreamLocalLimits limits;
std::shared_ptr<const EnabledQuota> quota;
UInt64 prev_elapsed = 0;

View File

@ -7,6 +7,7 @@
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Processors/Pipe.h>
#include <Processors/QueryPlan/ReadFromStorageStep.h>
#include <Interpreters/Context.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/quoteString.h>
@ -91,6 +92,27 @@ Pipe IStorage::read(
throw Exception("Method read is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
void IStorage::read(
QueryPlan & query_plan,
TableLockHolder table_lock,
StorageMetadataPtr metadata_snapshot,
StreamLocalLimits & limits,
std::shared_ptr<const EnabledQuota> quota,
const Names & column_names,
const SelectQueryInfo & query_info,
std::shared_ptr<Context> context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
unsigned num_streams)
{
auto read_step = std::make_unique<ReadFromStorageStep>(
std::move(table_lock), std::move(metadata_snapshot), limits, std::move(quota), shared_from_this(),
column_names, query_info, std::move(context), processed_stage, max_block_size, num_streams);
read_step->setStepDescription("Read from " + getName());
query_plan.addStep(std::move(read_step));
}
Pipe IStorage::alterPartition(
const ASTPtr & /* query */,
const StorageMetadataPtr & /* metadata_snapshot */,

View File

@ -48,6 +48,7 @@ using ProcessorPtr = std::shared_ptr<IProcessor>;
using Processors = std::vector<ProcessorPtr>;
class Pipe;
class QueryPlan;
class StoragePolicy;
using StoragePolicyPtr = std::shared_ptr<const StoragePolicy>;
@ -280,6 +281,21 @@ public:
size_t /*max_block_size*/,
unsigned /*num_streams*/);
/// Other version of read which adds reading step to query plan.
/// Default implementation creates ReadFromStorageStep and uses usual read.
virtual void read(
QueryPlan & query_plan,
TableLockHolder table_lock,
StorageMetadataPtr metadata_snapshot,
StreamLocalLimits & limits,
std::shared_ptr<const EnabledQuota> quota,
const Names & column_names,
const SelectQueryInfo & query_info,
std::shared_ptr<Context> context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
unsigned num_streams);
/** Writes the data to a table.
* Receives a description of the query, which can contain information about the data write method.
* Returns an object by which you can write data sequentially.

View File

@ -565,7 +565,7 @@ bool StorageKafka::streamToViews()
streams.emplace_back(stream);
// Limit read batch to maximum block size to allow DDL
IBlockInputStream::LocalLimits limits;
StreamLocalLimits limits;
limits.speed_limits.max_execution_time = kafka_settings->kafka_flush_interval_ms.changed
? kafka_settings->kafka_flush_interval_ms

View File

@ -340,7 +340,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(
if (multi_code == Coordination::Error::ZNODEEXISTS && deduplicate_block && failed_op_path == block_id_path)
{
/// Block with the same id have just appeared in table (or other replica), rollback thee insertion.
/// Block with the same id have just appeared in table (or other replica), rollback the insertion.
LOG_INFO(log, "Block with ID {} already exists; ignoring it (removing part {})", block_id, part->name);
part->is_duplicate = true;

View File

@ -752,7 +752,7 @@ bool StorageRabbitMQ::streamToViews()
streams.emplace_back(stream);
// Limit read batch to maximum block size to allow DDL
IBlockInputStream::LocalLimits limits;
StreamLocalLimits limits;
limits.speed_limits.max_execution_time = rabbitmq_settings->rabbitmq_flush_interval_ms.changed
? rabbitmq_settings->rabbitmq_flush_interval_ms

View File

@ -380,8 +380,10 @@ StorageDistributed::StorageDistributed(
if (!relative_data_path.empty())
{
storage_policy = global_context->getStoragePolicy(storage_policy_name_);
if (storage_policy->getVolumes().size() != 1)
throw Exception("Storage policy for Distributed table, should have exactly one volume", ErrorCodes::BAD_ARGUMENTS);
data_volume = storage_policy->getVolume(0);
if (storage_policy->getVolumes().size() > 1)
LOG_WARNING(log, "Storage policy for Distributed table has multiple volumes. "
"Only {} volume will be used to store data. Other will be ignored.", data_volume->getName());
}
/// Sanity check. Skip check if the table is already created to allow the server to start.
@ -599,7 +601,7 @@ void StorageDistributed::startup()
if (!storage_policy)
return;
for (const DiskPtr & disk : storage_policy->getDisks())
for (const DiskPtr & disk : data_volume->getDisks())
createDirectoryMonitors(disk->getPath());
for (const String & path : getDataPaths())
@ -633,7 +635,7 @@ void StorageDistributed::drop()
LOG_DEBUG(log, "Removing pending blocks for async INSERT from filesystem on DROP TABLE");
auto disks = storage_policy->getDisks();
auto disks = data_volume->getDisks();
for (const auto & disk : disks)
disk->removeRecursive(relative_data_path);
@ -647,7 +649,7 @@ Strings StorageDistributed::getDataPaths() const
if (relative_data_path.empty())
return paths;
for (const DiskPtr & disk : storage_policy->getDisks())
for (const DiskPtr & disk : data_volume->getDisks())
paths.push_back(disk->getPath() + relative_data_path);
return paths;
@ -866,7 +868,7 @@ void StorageDistributed::rename(const String & new_path_to_table_data, const Sto
void StorageDistributed::renameOnDisk(const String & new_path_to_table_data)
{
for (const DiskPtr & disk : storage_policy->getDisks())
for (const DiskPtr & disk : data_volume->getDisks())
{
const String path(disk->getPath());
auto new_path = path + new_path_to_table_data;

View File

@ -183,6 +183,11 @@ protected:
/// Can be empty if relative_data_path is empty. In this case, a directory for the data to be sent is not created.
StoragePolicyPtr storage_policy;
/// The main volume to store data.
/// Storage policy may have several configured volumes, but second and other volumes are used for parts movement in MergeTree engine.
/// For Distributed engine such configuration doesn't make sense and only the first (main) volume will be used to store data.
/// Other volumes will be ignored. It's needed to allow using the same multi-volume policy both for Distributed and other engines.
VolumePtr data_volume;
struct ClusterNodeData
{

View File

@ -15,8 +15,7 @@ NamesAndTypesList StorageSystemUserDirectories::getNamesAndTypes()
NamesAndTypesList names_and_types{
{"name", std::make_shared<DataTypeString>()},
{"type", std::make_shared<DataTypeString>()},
{"path", std::make_shared<DataTypeString>()},
{"readonly", std::make_shared<DataTypeUInt8>()},
{"params", std::make_shared<DataTypeString>()},
{"precedence", std::make_shared<DataTypeUInt64>()},
};
return names_and_types;
@ -31,21 +30,18 @@ void StorageSystemUserDirectories::fillData(MutableColumns & res_columns, const
size_t column_index = 0;
auto & column_name = assert_cast<ColumnString &>(*res_columns[column_index++]);
auto & column_type = assert_cast<ColumnString &>(*res_columns[column_index++]);
auto & column_path = assert_cast<ColumnString &>(*res_columns[column_index++]);
auto & column_readonly = assert_cast<ColumnUInt8 &>(*res_columns[column_index++]);
auto & column_params = assert_cast<ColumnString &>(*res_columns[column_index++]);
auto & column_precedence = assert_cast<ColumnUInt64 &>(*res_columns[column_index++]);
auto add_row = [&](const IAccessStorage & storage, size_t precedence)
{
const String & name = storage.getStorageName();
std::string_view type = storage.getStorageType();
const String & path = storage.getStoragePath();
bool readonly = storage.isStorageReadOnly();
String params = storage.getStorageParamsJSON();
column_name.insertData(name.data(), name.length());
column_type.insertData(type.data(), type.length());
column_path.insertData(path.data(), path.length());
column_readonly.insert(readonly);
column_params.insertData(params.data(), params.length());
column_precedence.insert(precedence);
};

View File

@ -1,5 +1,6 @@
from helpers.test_tools import TSV
def pytest_assertrepr_compare(op, left, right):
if isinstance(left, TSV) and isinstance(right, TSV) and op == '==':
return ['TabSeparated values differ: '] + left.diff(right)

View File

@ -1,8 +1,7 @@
import errno
import subprocess as sp
from threading import Timer
import tempfile
import os
import subprocess as sp
import tempfile
from threading import Timer
class Client:
@ -16,12 +15,13 @@ class Client:
self.command += ['--host', self.host, '--port', str(self.port), '--stacktrace']
def query(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None, database=None,
ignore_error=False):
return self.get_query_request(sql, stdin=stdin, timeout=timeout, settings=settings, user=user,
password=password, database=database, ignore_error=ignore_error).get_answer()
def query(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None, database=None, ignore_error=False):
return self.get_query_request(sql, stdin=stdin, timeout=timeout, settings=settings, user=user, password=password, database=database, ignore_error=ignore_error).get_answer()
def get_query_request(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None, database=None, ignore_error=False):
def get_query_request(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None, database=None,
ignore_error=False):
command = self.command[:]
if stdin is None:
@ -45,14 +45,17 @@ class Client:
return CommandRequest(command, stdin, timeout, ignore_error)
def query_and_get_error(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None,
database=None):
return self.get_query_request(sql, stdin=stdin, timeout=timeout, settings=settings, user=user,
password=password, database=database).get_error()
def query_and_get_error(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None, database=None):
return self.get_query_request(sql, stdin=stdin, timeout=timeout, settings=settings, user=user, password=password, database=database).get_error()
def query_and_get_answer_with_error(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None,
database=None):
return self.get_query_request(sql, stdin=stdin, timeout=timeout, settings=settings, user=user,
password=password, database=database).get_answer_and_error()
def query_and_get_answer_with_error(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None, database=None):
return self.get_query_request(sql, stdin=stdin, timeout=timeout, settings=settings, user=user, password=password, database=database).get_answer_and_error()
class QueryTimeoutExceedException(Exception):
pass
@ -90,7 +93,6 @@ class CommandRequest:
self.timer = Timer(timeout, kill_process)
self.timer.start()
def get_answer(self):
self.process.wait()
self.stdout_file.seek(0)
@ -103,11 +105,11 @@ class CommandRequest:
raise QueryTimeoutExceedException('Client timed out!')
if (self.process.returncode != 0 or stderr) and not self.ignore_error:
raise QueryRuntimeException('Client failed! Return code: {}, stderr: {}'.format(self.process.returncode, stderr))
raise QueryRuntimeException(
'Client failed! Return code: {}, stderr: {}'.format(self.process.returncode, stderr))
return stdout
def get_error(self):
self.process.wait()
self.stdout_file.seek(0)
@ -124,7 +126,6 @@ class CommandRequest:
return stderr
def get_answer_and_error(self):
self.process.wait()
self.stdout_file.seek(0)

View File

@ -1,30 +1,31 @@
import base64
import cassandra.cluster
import docker
import errno
import httplib
import logging
import os
import os.path as p
import pprint
import psycopg2
import pwd
import pymongo
import pymysql
import re
import requests
import shutil
import socket
import subprocess
import time
import urllib
import traceback
import urllib
import cassandra.cluster
import docker
import psycopg2
import pymongo
import pymysql
import requests
import xml.dom.minidom
from confluent.schemaregistry.client import CachedSchemaRegistryClient
from dicttoxml import dicttoxml
from kazoo.client import KazooClient
from kazoo.exceptions import KazooException
from minio import Minio
from confluent.schemaregistry.client import CachedSchemaRegistryClient
from .client import Client
from .hdfs_api import HDFSApi
@ -67,13 +68,14 @@ def get_odbc_bridge_path():
return '/usr/bin/clickhouse-odbc-bridge'
return path
def get_docker_compose_path():
compose_path = os.environ.get('DOCKER_COMPOSE_DIR')
if compose_path is not None:
return os.path.dirname(compose_path)
else:
if os.path.exists(os.path.dirname('/compose/')):
return os.path.dirname('/compose/') #default in docker runner container
return os.path.dirname('/compose/') # default in docker runner container
else:
print("Fallback docker_compose_path to LOCAL_DOCKER_COMPOSE_DIR: {}".format(LOCAL_DOCKER_COMPOSE_DIR))
return LOCAL_DOCKER_COMPOSE_DIR
@ -91,12 +93,12 @@ class ClickHouseCluster:
def __init__(self, base_path, name=None, base_config_dir=None, server_bin_path=None, client_bin_path=None,
odbc_bridge_bin_path=None, zookeeper_config_path=None, custom_dockerd_host=None):
for param in os.environ.keys():
print "ENV %40s %s" % (param,os.environ[param])
print "ENV %40s %s" % (param, os.environ[param])
self.base_dir = p.dirname(base_path)
self.name = name if name is not None else ''
self.base_config_dir = base_config_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR',
'/etc/clickhouse-server/')
'/etc/clickhouse-server/')
self.server_bin_path = p.realpath(
server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse'))
self.odbc_bridge_bin_path = p.realpath(odbc_bridge_bin_path or get_odbc_bridge_path())
@ -165,8 +167,10 @@ class ClickHouseCluster:
cmd += " client"
return cmd
def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries = None, macros=None,
with_zookeeper=False, with_mysql=False, with_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None,
def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None,
macros=None,
with_zookeeper=False, with_mysql=False, with_kafka=False, with_rabbitmq=False,
clickhouse_path_dir=None,
with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False,
with_redis=False, with_minio=False, with_cassandra=False,
hostname=None, env_variables=None, image="yandex/clickhouse-integration-test", tag=None,
@ -247,7 +251,8 @@ class ClickHouseCluster:
self.with_mysql = True
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')])
self.base_mysql_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]
self.project_name, '--file',
p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]
cmds.append(self.base_mysql_cmd)
@ -255,7 +260,8 @@ class ClickHouseCluster:
self.with_postgres = True
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')])
self.base_postgres_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]
self.project_name, '--file',
p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]
cmds.append(self.base_postgres_cmd)
if with_odbc_drivers and not self.with_odbc_drivers:
@ -264,7 +270,8 @@ class ClickHouseCluster:
self.with_mysql = True
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')])
self.base_mysql_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]
self.project_name, '--file',
p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]
cmds.append(self.base_mysql_cmd)
if not self.with_postgres:
@ -279,28 +286,32 @@ class ClickHouseCluster:
self.with_kafka = True
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')])
self.base_kafka_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')]
self.project_name, '--file',
p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')]
cmds.append(self.base_kafka_cmd)
if with_rabbitmq and not self.with_rabbitmq:
self.with_rabbitmq = True
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')])
self.base_rabbitmq_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')]
self.project_name, '--file',
p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')]
cmds.append(self.base_rabbitmq_cmd)
if with_hdfs and not self.with_hdfs:
self.with_hdfs = True
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')])
self.base_hdfs_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')]
self.project_name, '--file',
p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')]
cmds.append(self.base_hdfs_cmd)
if with_mongo and not self.with_mongo:
self.with_mongo = True
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')])
self.base_mongo_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')]
self.project_name, '--file',
p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')]
cmds.append(self.base_mongo_cmd)
if self.with_net_trics:
@ -311,21 +322,24 @@ class ClickHouseCluster:
self.with_redis = True
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')])
self.base_redis_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')]
self.project_name, '--file',
p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')]
if with_minio and not self.with_minio:
self.with_minio = True
self.minio_certs_dir = minio_certs_dir
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')])
self.base_minio_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')]
self.project_name, '--file',
p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')]
cmds.append(self.base_minio_cmd)
if with_cassandra and not self.with_cassandra:
self.with_cassandra = True
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')])
self.base_cassandra_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')]
self.project_name, '--file',
p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')]
return instance
@ -390,7 +404,8 @@ class ClickHouseCluster:
print("Container {} uses image {}: ".format(container_id, image_id))
pprint.pprint(image_info)
print("")
message = 'Cmd "{}" failed in container {}. Return code {}. Output: {}'.format(' '.join(cmd), container_id, exit_code, output)
message = 'Cmd "{}" failed in container {}. Return code {}. Output: {}'.format(' '.join(cmd), container_id,
exit_code, output)
if nothrow:
print(message)
else:
@ -401,7 +416,8 @@ class ClickHouseCluster:
with open(local_path, 'r') as fdata:
data = fdata.read()
encoded_data = base64.b64encode(data)
self.exec_in_container(container_id, ["bash", "-c", "echo {} | base64 --decode > {}".format(encoded_data, dest_path)],
self.exec_in_container(container_id,
["bash", "-c", "echo {} | base64 --decode > {}".format(encoded_data, dest_path)],
user='root')
def wait_mysql_to_start(self, timeout=60):
@ -650,7 +666,6 @@ class ClickHouseCluster:
subprocess_check_call(clickhouse_start_cmd)
print("ClickHouse instance created")
start_deadline = time.time() + 20.0 # seconds
for instance in self.instances.itervalues():
instance.docker_client = self.docker_client
@ -692,7 +707,7 @@ class ClickHouseCluster:
try:
subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans'])
except Exception as e:
print "Down + remove orphans failed durung shutdown. {}".format(repr(e))
print "Down + remove orphans failed durung shutdown. {}".format(repr(e))
self.is_up = False
@ -704,23 +719,26 @@ class ClickHouseCluster:
instance.client = None
if not self.zookeeper_use_tmpfs:
for i in range(1, 4):
zk_data_path = self.instances_dir + '/zkdata' + str(i)
zk_log_data_path = self.instances_dir + '/zklog' + str(i)
if os.path.exists(zk_data_path):
shutil.rmtree(zk_data_path)
if os.path.exists(zk_log_data_path):
shutil.rmtree(zk_log_data_path)
for i in range(1, 4):
zk_data_path = self.instances_dir + '/zkdata' + str(i)
zk_log_data_path = self.instances_dir + '/zklog' + str(i)
if os.path.exists(zk_data_path):
shutil.rmtree(zk_data_path)
if os.path.exists(zk_log_data_path):
shutil.rmtree(zk_log_data_path)
if sanitizer_assert_instance is not None:
raise Exception("Sanitizer assert found in {} for instance {}".format(self.docker_logs_path, sanitizer_assert_instance))
raise Exception(
"Sanitizer assert found in {} for instance {}".format(self.docker_logs_path, sanitizer_assert_instance))
def pause_container(self, instance_name):
subprocess_check_call(self.base_cmd + ['pause', instance_name])
# subprocess_check_call(self.base_cmd + ['kill', '-s SIGSTOP', instance_name])
def unpause_container(self, instance_name):
subprocess_check_call(self.base_cmd + ['unpause', instance_name])
# subprocess_check_call(self.base_cmd + ['kill', '-s SIGCONT', instance_name])
def open_bash_shell(self, instance_name):
@ -790,9 +808,12 @@ services:
class ClickHouseInstance:
def __init__(
self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs, custom_dictionaries,
macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio,
with_cassandra, server_bin_path, odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables=None,
self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs,
custom_dictionaries,
macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo,
with_redis, with_minio,
with_cassandra, server_bin_path, odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers,
hostname=None, env_variables=None,
image="yandex/clickhouse-integration-test", tag="latest",
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None):
@ -848,15 +869,19 @@ class ClickHouseInstance:
return "-fsanitize=thread" in build_opts
# Connects to the instance via clickhouse-client, sends a query (1st argument) and returns the answer
def query(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None, database=None, ignore_error=False):
return self.client.query(sql, stdin=stdin, timeout=timeout, settings=settings, user=user, password=password, database=database, ignore_error=ignore_error)
def query(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None, database=None,
ignore_error=False):
return self.client.query(sql, stdin=stdin, timeout=timeout, settings=settings, user=user, password=password,
database=database, ignore_error=ignore_error)
def query_with_retry(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None, database=None, ignore_error=False,
def query_with_retry(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None, database=None,
ignore_error=False,
retry_count=20, sleep_time=0.5, check_callback=lambda x: True):
result = None
for i in range(retry_count):
try:
result = self.query(sql, stdin=stdin, timeout=timeout, settings=settings, user=user, password=password, database=database, ignore_error=ignore_error)
result = self.query(sql, stdin=stdin, timeout=timeout, settings=settings, user=user, password=password,
database=database, ignore_error=ignore_error)
if check_callback(result):
return result
time.sleep(sleep_time)
@ -873,12 +898,16 @@ class ClickHouseInstance:
return self.client.get_query_request(*args, **kwargs)
# Connects to the instance via clickhouse-client, sends a query (1st argument), expects an error and return its code
def query_and_get_error(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None, database=None):
return self.client.query_and_get_error(sql, stdin=stdin, timeout=timeout, settings=settings, user=user, password=password, database=database)
def query_and_get_error(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None,
database=None):
return self.client.query_and_get_error(sql, stdin=stdin, timeout=timeout, settings=settings, user=user,
password=password, database=database)
# The same as query_and_get_error but ignores successful query.
def query_and_get_answer_with_error(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None, database=None):
return self.client.query_and_get_answer_with_error(sql, stdin=stdin, timeout=timeout, settings=settings, user=user, password=password, database=database)
def query_and_get_answer_with_error(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None,
database=None):
return self.client.query_and_get_answer_with_error(sql, stdin=stdin, timeout=timeout, settings=settings,
user=user, password=password, database=database)
# Connects to the instance via HTTP interface, sends a query and returns the answer
def http_query(self, sql, data=None, params=None, user=None, password=None, expect_fail_and_get_error=False):
@ -900,7 +929,8 @@ class ClickHouseInstance:
open_result = urllib.urlopen(url, data)
def http_code_and_message():
return str(open_result.getcode()) + " " + httplib.responses[open_result.getcode()] + ": " + open_result.read()
return str(open_result.getcode()) + " " + httplib.responses[
open_result.getcode()] + ": " + open_result.read()
if expect_fail_and_get_error:
if open_result.getcode() == 200:
@ -913,18 +943,19 @@ class ClickHouseInstance:
# Connects to the instance via HTTP interface, sends a query and returns the answer
def http_request(self, url, method='GET', params=None, data=None, headers=None):
url = "http://" + self.ip_address + ":8123/"+url
url = "http://" + self.ip_address + ":8123/" + url
return requests.request(method=method, url=url, params=params, data=data, headers=headers)
# Connects to the instance via HTTP interface, sends a query, expects an error and return the error message
def http_query_and_get_error(self, sql, data=None, params=None, user=None, password=None):
return self.http_query(sql=sql, data=data, params=params, user=user, password=password, expect_fail_and_get_error=True)
return self.http_query(sql=sql, data=data, params=params, user=user, password=password,
expect_fail_and_get_error=True)
def kill_clickhouse(self, stop_start_wait_sec=5):
pid = self.get_process_pid("clickhouse")
if not pid:
raise Exception("No clickhouse found")
self.exec_in_container(["bash", "-c", "kill -9 {}".format(pid)], user='root')
self.exec_in_container(["bash", "-c", "kill -9 {}".format(pid)], user='root')
time.sleep(stop_start_wait_sec)
def restore_clickhouse(self, retries=100):
@ -1030,7 +1061,8 @@ class ClickHouseInstance:
time_left = deadline - current_time
if deadline is not None and current_time >= deadline:
raise Exception("Timed out while waiting for instance `{}' with ip address {} to start. "
"Container status: {}, logs: {}".format(self.name, self.ip_address, status, handle.logs()))
"Container status: {}, logs: {}".format(self.name, self.ip_address, status,
handle.logs()))
# Repeatedly poll the instance address until there is something that listens there.
# Usually it means that ClickHouse is ready to accept queries.

View File

@ -59,7 +59,8 @@ class Row(object):
class Field(object):
def __init__(self, name, field_type, is_key=False, is_range_key=False, default=None, hierarchical=False, range_hash_type=None, default_value_for_get=None):
def __init__(self, name, field_type, is_key=False, is_range_key=False, default=None, hierarchical=False,
range_hash_type=None, default_value_for_get=None):
self.name = name
self.field_type = field_type
self.is_key = is_key
@ -123,7 +124,8 @@ class DictionaryStructure(object):
self.range_key = field
if not self.layout.is_complex and len(self.keys) > 1:
raise Exception("More than one key {} field in non complex layout {}".format(len(self.keys), self.layout.name))
raise Exception(
"More than one key {} field in non complex layout {}".format(len(self.keys), self.layout.name))
if self.layout.is_ranged and (not self.range_key or len(self.range_fields) != 2):
raise Exception("Inconsistent configuration of ranged dictionary")
@ -213,7 +215,8 @@ class DictionaryStructure(object):
if or_default:
or_default_expr = 'OrDefault'
if field.default_value_for_get is None:
raise Exception("Can create 'dictGetOrDefault' query for field {} without default_value_for_get".format(field.name))
raise Exception(
"Can create 'dictGetOrDefault' query for field {} without default_value_for_get".format(field.name))
val = field.default_value_for_get
if isinstance(val, str):
@ -259,15 +262,16 @@ class DictionaryStructure(object):
def get_get_or_default_expressions(self, dict_name, field, row):
if not self.layout.is_ranged:
return [
self._get_dict_get_common_expression(dict_name, field, row, or_default=True, with_type=False, has=False),
self._get_dict_get_common_expression(dict_name, field, row, or_default=True, with_type=False,
has=False),
self._get_dict_get_common_expression(dict_name, field, row, or_default=True, with_type=True, has=False),
]
return []
def get_has_expressions(self, dict_name, field, row):
if not self.layout.is_ranged:
return [self._get_dict_get_common_expression(dict_name, field, row, or_default=False, with_type=False, has=True)]
return [self._get_dict_get_common_expression(dict_name, field, row, or_default=False, with_type=False,
has=True)]
return []
def get_hierarchical_expressions(self, dict_name, row):
@ -290,7 +294,7 @@ class DictionaryStructure(object):
"dictIsIn('{dict_name}', {child_key}, {parent_key})".format(
dict_name=dict_name,
child_key=child_key_expr,
parent_key=parent_key_expr,)
parent_key=parent_key_expr, )
]
return []
@ -364,7 +368,8 @@ class Dictionary(object):
return ['select {}'.format(expr) for expr in self.structure.get_get_expressions(self.name, field, row)]
def get_select_get_or_default_queries(self, field, row):
return ['select {}'.format(expr) for expr in self.structure.get_get_or_default_expressions(self.name, field, row)]
return ['select {}'.format(expr) for expr in
self.structure.get_get_or_default_expressions(self.name, field, row)]
def get_select_has_queries(self, field, row):
return ['select {}'.format(expr) for expr in self.structure.get_has_expressions(self.name, field, row)]

View File

@ -1,14 +1,15 @@
# -*- coding: utf-8 -*-
import warnings
import pymysql.cursors
import pymongo
import cassandra.cluster
import redis
import aerospike
from tzlocal import get_localzone
import datetime
import os
import uuid
import warnings
import aerospike
import cassandra.cluster
import pymongo
import pymysql.cursors
import redis
from tzlocal import get_localzone
class ExternalSource(object):
@ -89,12 +90,12 @@ class SourceMySQL(ExternalSource):
<db>test</db>
<table>{tbl}</table>
</mysql>'''.format(
hostname=self.docker_hostname,
port=self.docker_port,
user=self.user,
password=self.password,
tbl=table_name,
)
hostname=self.docker_hostname,
port=self.docker_port,
user=self.user,
password=self.password,
tbl=table_name,
)
def prepare(self, structure, table_name, cluster):
self.create_mysql_conn()
@ -160,7 +161,8 @@ class SourceMongo(ExternalSource):
if field.field_type == "Date":
self.converters[field.name] = lambda x: datetime.datetime.strptime(x, "%Y-%m-%d")
elif field.field_type == "DateTime":
self.converters[field.name] = lambda x: get_localzone().localize(datetime.datetime.strptime(x, "%Y-%m-%d %H:%M:%S"))
self.converters[field.name] = lambda x: get_localzone().localize(
datetime.datetime.strptime(x, "%Y-%m-%d %H:%M:%S"))
else:
self.converters[field.name] = lambda x: x
@ -180,6 +182,7 @@ class SourceMongo(ExternalSource):
result = tbl.insert_many(to_insert)
class SourceMongoURI(SourceMongo):
def compatible_with_layout(self, layout):
# It is enough to test one layout for this dictionary, since we're
@ -200,6 +203,7 @@ class SourceMongoURI(SourceMongo):
tbl=table_name,
)
class SourceClickHouse(ExternalSource):
def get_source_str(self, table_name):
@ -284,7 +288,8 @@ class SourceFile(ExternalSource):
sorted_row.append(str(row.data[name]))
str_data = '\t'.join(sorted_row)
self.node.exec_in_container(["bash", "-c", "echo \"{row}\" >> {fname}".format(row=str_data, fname=path)], user="root")
self.node.exec_in_container(["bash", "-c", "echo \"{row}\" >> {fname}".format(row=str_data, fname=path)],
user="root")
def compatible_with_layout(self, layout):
return 'cache' not in layout.name and 'direct' not in layout.name
@ -324,7 +329,8 @@ class _SourceExecutableBase(ExternalSource):
sorted_row.append(str(row.data[name]))
str_data = '\t'.join(sorted_row)
self.node.exec_in_container(["bash", "-c", "echo \"{row}\" >> {fname}".format(row=str_data, fname=path)], user='root')
self.node.exec_in_container(["bash", "-c", "echo \"{row}\" >> {fname}".format(row=str_data, fname=path)],
user='root')
class SourceExecutableCache(_SourceExecutableBase):
@ -344,12 +350,14 @@ class SourceExecutableHashed(_SourceExecutableBase):
def compatible_with_layout(self, layout):
return 'cache' in layout.name
class SourceHTTPBase(ExternalSource):
class SourceHTTPBase(ExternalSource):
PORT_COUNTER = 5555
def get_source_str(self, table_name):
self.http_port = SourceHTTPBase.PORT_COUNTER
url = "{schema}://{host}:{port}/".format(schema=self._get_schema(), host=self.docker_hostname, port=self.http_port)
url = "{schema}://{host}:{port}/".format(schema=self._get_schema(), host=self.docker_hostname,
port=self.http_port)
SourceHTTPBase.PORT_COUNTER += 1
return '''
<http>
@ -395,7 +403,8 @@ class SourceHTTPBase(ExternalSource):
sorted_row.append(str(row.data[name]))
str_data = '\t'.join(sorted_row)
self.node.exec_in_container(["bash", "-c", "echo \"{row}\" >> {fname}".format(row=str_data, fname=path)], user='root')
self.node.exec_in_container(["bash", "-c", "echo \"{row}\" >> {fname}".format(row=str_data, fname=path)],
user='root')
class SourceHTTP(SourceHTTPBase):
@ -407,6 +416,7 @@ class SourceHTTPS(SourceHTTPBase):
def _get_schema(self):
return "https"
class SourceCassandra(ExternalSource):
TYPE_MAPPING = {
'UInt8': 'tinyint',
@ -426,7 +436,8 @@ class SourceCassandra(ExternalSource):
}
def __init__(self, name, internal_hostname, internal_port, docker_hostname, docker_port, user, password):
ExternalSource.__init__(self, name, internal_hostname, internal_port, docker_hostname, docker_port, user, password)
ExternalSource.__init__(self, name, internal_hostname, internal_port, docker_hostname, docker_port, user,
password)
self.structure = dict()
def get_source_str(self, table_name):
@ -448,13 +459,14 @@ class SourceCassandra(ExternalSource):
def prepare(self, structure, table_name, cluster):
self.client = cassandra.cluster.Cluster([self.internal_hostname], port=self.internal_port)
self.session = self.client.connect()
self.session.execute("create keyspace if not exists test with replication = {'class': 'SimpleStrategy', 'replication_factor' : 1};")
self.session.execute(
"create keyspace if not exists test with replication = {'class': 'SimpleStrategy', 'replication_factor' : 1};")
self.session.execute('drop table if exists test."{}"'.format(table_name))
self.structure[table_name] = structure
columns = ['"' + col.name + '" ' + self.TYPE_MAPPING[col.field_type] for col in structure.get_all_fields()]
keys = ['"' + col.name + '"' for col in structure.keys]
query = 'create table test."{name}" ({columns}, primary key ({pk}));'.format(
name=table_name, columns=', '.join(columns), pk=', '.join(keys))
name=table_name, columns=', '.join(columns), pk=', '.join(keys))
self.session.execute(query)
self.prepared = True
@ -470,14 +482,16 @@ class SourceCassandra(ExternalSource):
names_and_types = [(field.name, field.field_type) for field in self.structure[table_name].get_all_fields()]
columns = ['"' + col[0] + '"' for col in names_and_types]
insert = 'insert into test."{table}" ({columns}) values ({args})'.format(
table=table_name, columns=','.join(columns), args=','.join(['%s']*len(columns)))
table=table_name, columns=','.join(columns), args=','.join(['%s'] * len(columns)))
for row in data:
values = [self.get_value_to_insert(row.get_value_by_name(col[0]), col[1]) for col in names_and_types]
self.session.execute(insert, values)
class SourceRedis(ExternalSource):
def __init__(
self, name, internal_hostname, internal_port, docker_hostname, docker_port, user, password, db_index, storage_type
self, name, internal_hostname, internal_port, docker_hostname, docker_port, user, password, db_index,
storage_type
):
super(SourceRedis, self).__init__(
name, internal_hostname, internal_port, docker_hostname, docker_port, user, password
@ -503,7 +517,8 @@ class SourceRedis(ExternalSource):
)
def prepare(self, structure, table_name, cluster):
self.client = redis.StrictRedis(host=self.internal_hostname, port=self.internal_port, db=self.db_index, password=self.password or None)
self.client = redis.StrictRedis(host=self.internal_hostname, port=self.internal_port, db=self.db_index,
password=self.password or None)
self.prepared = True
self.ordered_names = structure.get_ordered_names()
@ -521,11 +536,12 @@ class SourceRedis(ExternalSource):
def compatible_with_layout(self, layout):
return layout.is_simple and self.storage_type == "simple" or layout.is_complex and self.storage_type == "hash_map"
class SourceAerospike(ExternalSource):
def __init__(self, name, internal_hostname, internal_port,
docker_hostname, docker_port, user, password):
ExternalSource.__init__(self, name, internal_hostname, internal_port,
docker_hostname, docker_port, user, password)
docker_hostname, docker_port, user, password)
self.namespace = "test"
self.set = "test_set"
@ -543,7 +559,7 @@ class SourceAerospike(ExternalSource):
def prepare(self, structure, table_name, cluster):
config = {
'hosts': [ (self.internal_hostname, self.internal_port) ]
'hosts': [(self.internal_hostname, self.internal_port)]
}
self.client = aerospike.client(config).connect()
self.prepared = True
@ -580,7 +596,7 @@ class SourceAerospike(ExternalSource):
self.client.put(key, {"bin_value": value[1]}, policy={"key": aerospike.POLICY_KEY_SEND})
assert self.client.exists(key)
else:
assert("VALUES SIZE != 2")
assert ("VALUES SIZE != 2")
# print(values)

View File

@ -1,10 +1,12 @@
#-*- coding: utf-8 -*-
# -*- coding: utf-8 -*-
import StringIO
import gzip
import requests
import subprocess
from tempfile import NamedTemporaryFile
import requests
class HDFSApi(object):
def __init__(self, user):
self.host = "localhost"
@ -13,11 +15,15 @@ class HDFSApi(object):
self.user = user
def read_data(self, path):
response = requests.get("http://{host}:{port}/webhdfs/v1{path}?op=OPEN".format(host=self.host, port=self.http_proxy_port, path=path), allow_redirects=False)
response = requests.get(
"http://{host}:{port}/webhdfs/v1{path}?op=OPEN".format(host=self.host, port=self.http_proxy_port,
path=path), allow_redirects=False)
if response.status_code != 307:
response.raise_for_status()
additional_params = '&'.join(response.headers['Location'].split('&')[1:2])
response_data = requests.get("http://{host}:{port}/webhdfs/v1{path}?op=OPEN&{params}".format(host=self.host, port=self.http_data_port, path=path, params=additional_params))
response_data = requests.get(
"http://{host}:{port}/webhdfs/v1{path}?op=OPEN&{params}".format(host=self.host, port=self.http_data_port,
path=path, params=additional_params))
if response_data.status_code != 200:
response_data.raise_for_status()
@ -25,7 +31,9 @@ class HDFSApi(object):
# Requests can't put file
def _curl_to_put(self, filename, path, params):
url = "http://{host}:{port}/webhdfs/v1{path}?op=CREATE&{params}".format(host=self.host, port=self.http_data_port, path=path, params=params)
url = "http://{host}:{port}/webhdfs/v1{path}?op=CREATE&{params}".format(host=self.host,
port=self.http_data_port, path=path,
params=params)
cmd = "curl -s -i -X PUT -T {fname} '{url}'".format(fname=filename, url=url)
output = subprocess.check_output(cmd, shell=True)
return output
@ -36,13 +44,15 @@ class HDFSApi(object):
named_file.write(content)
named_file.flush()
response = requests.put(
"http://{host}:{port}/webhdfs/v1{path}?op=CREATE".format(host=self.host, port=self.http_proxy_port, path=path, user=self.user),
"http://{host}:{port}/webhdfs/v1{path}?op=CREATE".format(host=self.host, port=self.http_proxy_port,
path=path, user=self.user),
allow_redirects=False
)
if response.status_code != 307:
response.raise_for_status()
additional_params = '&'.join(response.headers['Location'].split('&')[1:2] + ["user.name={}".format(self.user), "overwrite=true"])
additional_params = '&'.join(
response.headers['Location'].split('&')[1:2] + ["user.name={}".format(self.user), "overwrite=true"])
output = self._curl_to_put(fpath, path, additional_params)
if "201 Created" not in output:
raise Exception("Can't create file on hdfs:\n {}".format(output))

View File

@ -1,9 +1,9 @@
# -*- coding: utf-8 -*-
import argparse
from BaseHTTPServer import BaseHTTPRequestHandler, HTTPServer
import csv
import socket
import ssl
import csv
from BaseHTTPServer import BaseHTTPRequestHandler, HTTPServer
# Decorator used to see if authentication works for external dictionary who use a HTTP source.
@ -15,6 +15,7 @@ def check_auth(fn):
req.send_response(401)
else:
fn(req)
return wrapper
@ -37,7 +38,7 @@ def start_server(server_address, data_path, schema, cert_path, address_family):
self.send_header('Content-type', 'text/tsv')
self.end_headers()
def __send_data(self, only_ids = None):
def __send_data(self, only_ids=None):
with open(data_path, 'r') as fl:
reader = csv.reader(fl, delimiter='\t')
for row in reader:

View File

@ -1,12 +1,9 @@
import os.path as p
import os
import subprocess
import time
import os
import docker
from .cluster import CLICKHOUSE_ROOT_DIR
class PartitionManager:
"""Allows introducing failures in the network between docker containers.
@ -23,21 +20,18 @@ class PartitionManager:
def __init__(self):
self._iptables_rules = []
def drop_instance_zk_connections(self, instance, action='DROP'):
self._check_instance(instance)
self._add_rule({'source': instance.ip_address, 'destination_port': 2181, 'action': action})
self._add_rule({'destination': instance.ip_address, 'source_port': 2181, 'action': action})
def restore_instance_zk_connections(self, instance, action='DROP'):
self._check_instance(instance)
self._delete_rule({'source': instance.ip_address, 'destination_port': 2181, 'action': action})
self._delete_rule({'destination': instance.ip_address, 'source_port': 2181, 'action': action})
def partition_instances(self, left, right, port=None, action='DROP'):
self._check_instance(left)
self._check_instance(right)
@ -51,7 +45,6 @@ class PartitionManager:
self._add_rule(create_rule(left, right))
self._add_rule(create_rule(right, left))
def heal_all(self):
while self._iptables_rules:
rule = self._iptables_rules.pop()
@ -66,7 +59,6 @@ class PartitionManager:
for rule in rules:
self._add_rule(rule)
@staticmethod
def _check_instance(instance):
if instance.ip_address is None:
@ -152,7 +144,6 @@ class _NetworkManager:
ret.extend(['-j'] + action.split())
return ret
def __init__(
self,
container_expire_timeout=50, container_exit_timeout=60):
@ -175,7 +166,8 @@ class _NetworkManager:
except docker.errors.NotFound:
pass
self._container = self._docker_client.containers.run('yandex/clickhouse-integration-helper', auto_remove=True,
self._container = self._docker_client.containers.run('yandex/clickhouse-integration-helper',
auto_remove=True,
command=('sleep %s' % self.container_exit_timeout),
detach=True, network_mode='host')
container_id = self._container.id

View File

@ -1,6 +1,7 @@
import difflib
import time
class TSV:
"""Helper to get pretty diffs between expected and actual tab-separated value files"""
@ -40,17 +41,22 @@ class TSV:
def toMat(contents):
return [line.split("\t") for line in contents.split("\n") if line.strip()]
def assert_eq_with_retry(instance, query, expectation, retry_count=20, sleep_time=0.5, stdin=None, timeout=None, settings=None, user=None, ignore_error=False):
def assert_eq_with_retry(instance, query, expectation, retry_count=20, sleep_time=0.5, stdin=None, timeout=None,
settings=None, user=None, ignore_error=False):
expectation_tsv = TSV(expectation)
for i in xrange(retry_count):
try:
if TSV(instance.query(query, user=user, stdin=stdin, timeout=timeout, settings=settings, ignore_error=ignore_error)) == expectation_tsv:
if TSV(instance.query(query, user=user, stdin=stdin, timeout=timeout, settings=settings,
ignore_error=ignore_error)) == expectation_tsv:
break
time.sleep(sleep_time)
except Exception as ex:
print "assert_eq_with_retry retry {} exception {}".format(i + 1, ex)
time.sleep(sleep_time)
else:
val = TSV(instance.query(query, user=user, stdin=stdin, timeout=timeout, settings=settings, ignore_error=ignore_error))
val = TSV(instance.query(query, user=user, stdin=stdin, timeout=timeout, settings=settings,
ignore_error=ignore_error))
if expectation_tsv != val:
raise AssertionError("'{}' != '{}'\n{}".format(expectation_tsv, val, '\n'.join(expectation_tsv.diff(val, n1="expectation", n2="query"))))
raise AssertionError("'{}' != '{}'\n{}".format(expectation_tsv, val, '\n'.join(
expectation_tsv.diff(val, n1="expectation", n2="query"))))

View File

@ -11,9 +11,10 @@ import uexpect
prompt = ':\) '
end_of_block = r'.*\r\n.*\r\n'
class client(object):
def __init__(self, command=None, name='', log=None):
self.client = uexpect.spawn(['/bin/bash','--noediting'])
self.client = uexpect.spawn(['/bin/bash', '--noediting'])
if command is None:
command = '/usr/bin/clickhouse-client'
self.client.command = command

View File

@ -13,13 +13,12 @@
# limitations under the License.
import os
import pty
import time
import sys
import re
from threading import Thread, Event
from subprocess import Popen
import time
from Queue import Queue, Empty
from subprocess import Popen
from threading import Thread, Event
class TimeoutError(Exception):
def __init__(self, timeout):
@ -28,6 +27,7 @@ class TimeoutError(Exception):
def __str__(self):
return 'Timeout %.3fs' % float(self.timeout)
class ExpectTimeoutError(Exception):
def __init__(self, pattern, timeout, buffer):
self.pattern = pattern
@ -43,6 +43,7 @@ class ExpectTimeoutError(Exception):
s += 'or \'%s\'' % ','.join(['%x' % ord(c) for c in self.buffer[:]])
return s
class IO(object):
class EOF(object):
pass
@ -59,7 +60,7 @@ class IO(object):
self._prefix = prefix
def write(self, data):
self._logger.write(('\n' + data).replace('\n','\n' + self._prefix))
self._logger.write(('\n' + data).replace('\n', '\n' + self._prefix))
def flush(self):
self._logger.flush()
@ -165,7 +166,7 @@ class IO(object):
data = ''
timeleft = timeout
try:
while timeleft >= 0 :
while timeleft >= 0:
start_time = time.time()
data += self.queue.get(timeout=timeleft)
if data:
@ -182,6 +183,7 @@ class IO(object):
return data
def spawn(command):
master, slave = pty.openpty()
process = Popen(command, preexec_fn=os.setsid, stdout=slave, stdin=slave, stderr=slave, bufsize=1)
@ -193,7 +195,8 @@ def spawn(command):
thread.daemon = True
thread.start()
return IO(process, master, queue, reader={'thread':thread, 'kill_event':reader_kill_event})
return IO(process, master, queue, reader={'thread': thread, 'kill_event': reader_kill_event})
def reader(process, out, queue, kill_event):
while True:

View File

@ -24,6 +24,7 @@ system_logs = [
# decrease timeout for the test to show possible issues.
timeout = pytest.mark.timeout(30)
@pytest.fixture(scope='module', autouse=True)
def start_cluster():
try:
@ -32,10 +33,12 @@ def start_cluster():
finally:
cluster.shutdown()
@pytest.fixture(scope='function')
def flush_logs():
node.query('SYSTEM FLUSH LOGS')
@timeout
@pytest.mark.parametrize('table,exists', system_logs)
def test_system_logs(flush_logs, table, exists):
@ -45,6 +48,7 @@ def test_system_logs(flush_logs, table, exists):
else:
assert "Table {} doesn't exist".format(table) in node.query_and_get_error(q)
# Logic is tricky, let's check that there is no hang in case of message queue
# is not empty (this is another code path in the code).
@timeout

View File

@ -1,13 +1,12 @@
import time
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.client import QueryRuntimeException
cluster = ClickHouseCluster(__file__)
ch1 = cluster.add_instance('ch1', main_configs=["configs/config.d/clusters.xml"], with_zookeeper=True)
ch2 = cluster.add_instance('ch2', main_configs=["configs/config.d/clusters.xml"], with_zookeeper=True)
ch3 = cluster.add_instance('ch3', main_configs=["configs/config.d/clusters.xml"], with_zookeeper=True)
@pytest.fixture(scope="module", autouse=True)
def started_cluster():
try:

View File

@ -1,13 +1,10 @@
import time
import pytest
from helpers.cluster import ClickHouseCluster
from multiprocessing.dummy import Pool
from helpers.client import QueryRuntimeException, QueryTimeoutExceedException
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True)
node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True)
@ -34,80 +31,80 @@ def prepare_single_pair_with_setting(first_node, second_node, group):
# Two tables with adaptive granularity
first_node.query(
'''
CREATE TABLE table_by_default(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_by_default', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 10485760
'''.format(g=group))
second_node.query(
'''
CREATE TABLE table_by_default(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_by_default', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 10485760
'''.format(g=group))
# Two tables with fixed granularity
first_node.query(
'''
CREATE TABLE table_with_fixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_fixed_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 0
'''.format(g=group))
second_node.query(
'''
CREATE TABLE table_with_fixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_fixed_granularity', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 0
'''.format(g=group))
# Two tables with different granularity
with pytest.raises(QueryRuntimeException):
first_node.query(
'''
CREATE TABLE table_with_different_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_different_granularity', '1')
CREATE TABLE table_by_default(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_by_default', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 10485760
'''.format(g=group))
second_node.query(
second_node.query(
'''
CREATE TABLE table_with_different_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_different_granularity', '2')
CREATE TABLE table_by_default(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_by_default', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 10485760
'''.format(g=group))
# Two tables with fixed granularity
first_node.query(
'''
CREATE TABLE table_with_fixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_fixed_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 0
'''.format(g=group))
# Two tables with different granularity, but enabled mixed parts
first_node.query(
second_node.query(
'''
CREATE TABLE table_with_mixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_mixed_granularity', '1')
CREATE TABLE table_with_fixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_fixed_granularity', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 10485760, enable_mixed_granularity_parts=1
SETTINGS index_granularity_bytes = 0
'''.format(g=group))
# Two tables with different granularity
with pytest.raises(QueryRuntimeException):
first_node.query(
'''
CREATE TABLE table_with_different_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_different_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 10485760
'''.format(g=group))
second_node.query(
'''
CREATE TABLE table_with_mixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_mixed_granularity', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 0, enable_mixed_granularity_parts=1
'''.format(g=group))
'''
CREATE TABLE table_with_different_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_different_granularity', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 0
'''.format(g=group))
# Two tables with different granularity, but enabled mixed parts
first_node.query(
'''
CREATE TABLE table_with_mixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_mixed_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 10485760, enable_mixed_granularity_parts=1
'''.format(g=group))
second_node.query(
'''
CREATE TABLE table_with_mixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_mixed_granularity', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 0, enable_mixed_granularity_parts=1
'''.format(g=group))
def prepare_single_pair_without_setting(first_node, second_node, group):
@ -116,21 +113,21 @@ def prepare_single_pair_without_setting(first_node, second_node, group):
# Two tables with fixed granularity
first_node.query(
'''
CREATE TABLE table_with_fixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_fixed_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
'''.format(g=group))
'''
CREATE TABLE table_with_fixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_fixed_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
'''.format(g=group))
second_node.query(
'''
CREATE TABLE table_with_fixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_fixed_granularity', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 0
'''.format(g=group))
'''
CREATE TABLE table_with_fixed_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{g}/table_with_fixed_granularity', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes = 0
'''.format(g=group))
@pytest.fixture(scope="module")
@ -160,7 +157,8 @@ def start_static_cluster():
def test_different_versions_cluster(start_static_cluster, first_node, second_node, table):
counter = 1
for n1, n2 in ((first_node, second_node), (second_node, first_node)):
n1.query("INSERT INTO {tbl} VALUES (toDate('2018-10-01'), {c1}, 333), (toDate('2018-10-02'), {c2}, 444)".format(tbl=table, c1=counter * 2, c2=counter * 2 + 1))
n1.query("INSERT INTO {tbl} VALUES (toDate('2018-10-01'), {c1}, 333), (toDate('2018-10-02'), {c2}, 444)".format(
tbl=table, c1=counter * 2, c2=counter * 2 + 1))
n2.query("SYSTEM SYNC REPLICA {tbl}".format(tbl=table))
assert_eq_with_retry(n2, "SELECT count() from {tbl}".format(tbl=table), str(counter * 2))
n1.query("DETACH TABLE {tbl}".format(tbl=table))
@ -175,73 +173,74 @@ def test_different_versions_cluster(start_static_cluster, first_node, second_nod
assert_eq_with_retry(n2, "SELECT count() from {tbl}".format(tbl=table), str(counter * 2))
counter += 1
@pytest.fixture(scope="module")
def start_dynamic_cluster():
try:
cluster.start()
node7.query(
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/7/table_with_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/7/table_with_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
node7.query(
'''
CREATE TABLE table_with_adaptive_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/7/table_with_adaptive_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes=10485760
''')
'''
CREATE TABLE table_with_adaptive_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/7/table_with_adaptive_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
SETTINGS index_granularity_bytes=10485760
''')
node8.query(
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/8/table_with_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/8/table_with_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
node9.query(
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/9/table_with_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/9/table_with_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
node10.query(
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/10/table_with_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/10/table_with_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
node11.query(
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard11/table_with_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard11/table_with_default_granularity', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
node12.query(
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard11/table_with_default_granularity', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
'''
CREATE TABLE table_with_default_granularity(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard11/table_with_default_granularity', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
yield cluster
finally:
cluster.shutdown()
@pytest.mark.parametrize(
('n', 'tables'),
[
@ -251,13 +250,16 @@ def start_dynamic_cluster():
)
def test_version_single_node_update(start_dynamic_cluster, n, tables):
for table in tables:
n.query("INSERT INTO {tbl} VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)".format(tbl=table))
n.query(
"INSERT INTO {tbl} VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)".format(tbl=table))
n.restart_with_latest_version()
for table in tables:
assert n.query("SELECT count() from {tbl}".format(tbl=table)) == '2\n'
n.query("INSERT INTO {tbl} VALUES (toDate('2018-10-01'), 3, 333), (toDate('2018-10-02'), 4, 444)".format(tbl=table))
n.query(
"INSERT INTO {tbl} VALUES (toDate('2018-10-01'), 3, 333), (toDate('2018-10-02'), 4, 444)".format(tbl=table))
assert n.query("SELECT count() from {tbl}".format(tbl=table)) == '4\n'
@pytest.mark.parametrize(
('node',),
[
@ -266,10 +268,13 @@ def test_version_single_node_update(start_dynamic_cluster, n, tables):
]
)
def test_mixed_granularity_single_node(start_dynamic_cluster, node):
node.query("INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)")
node.query("INSERT INTO table_with_default_granularity VALUES (toDate('2018-09-01'), 1, 333), (toDate('2018-09-02'), 2, 444)")
node.query(
"INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)")
node.query(
"INSERT INTO table_with_default_granularity VALUES (toDate('2018-09-01'), 1, 333), (toDate('2018-09-02'), 2, 444)")
def callback(n):
<<<<<<< HEAD
new_config = """
<yandex><merge_tree>
<enable_mixed_granularity_parts>1</enable_mixed_granularity_parts>
@ -278,21 +283,34 @@ def test_mixed_granularity_single_node(start_dynamic_cluster, node):
n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml", new_config)
n.replace_config("/etc/clickhouse-server/config.d/merge_tree_settings.xml", new_config)
=======
n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml",
"<yandex><merge_tree><enable_mixed_granularity_parts>1</enable_mixed_granularity_parts></merge_tree></yandex>")
n.replace_config("/etc/clickhouse-server/config.d/merge_tree_settings.xml",
"<yandex><merge_tree><enable_mixed_granularity_parts>1</enable_mixed_granularity_parts></merge_tree></yandex>")
>>>>>>> upstream/master
node.restart_with_latest_version(callback_onstop=callback)
node.query("SYSTEM RELOAD CONFIG")
assert_eq_with_retry(node, "SELECT value FROM system.merge_tree_settings WHERE name='enable_mixed_granularity_parts'", '1')
assert_eq_with_retry(node,
"SELECT value FROM system.merge_tree_settings WHERE name='enable_mixed_granularity_parts'",
'1')
assert node.query("SELECT count() from table_with_default_granularity") == '4\n'
node.query("INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 3, 333), (toDate('2018-10-02'), 4, 444)")
node.query(
"INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 3, 333), (toDate('2018-10-02'), 4, 444)")
assert node.query("SELECT count() from table_with_default_granularity") == '6\n'
node.query("OPTIMIZE TABLE table_with_default_granularity PARTITION 201810 FINAL")
assert node.query("SELECT count() from table_with_default_granularity") == '6\n'
path_to_merged_part = node.query("SELECT path FROM system.parts WHERE table = 'table_with_default_granularity' AND active=1 ORDER BY partition DESC LIMIT 1").strip()
node.exec_in_container(["bash", "-c", "find {p} -name '*.mrk2' | grep '.*'".format(p=path_to_merged_part)]) # check that we have adaptive files
path_to_merged_part = node.query(
"SELECT path FROM system.parts WHERE table = 'table_with_default_granularity' AND active=1 ORDER BY partition DESC LIMIT 1").strip()
node.exec_in_container(["bash", "-c", "find {p} -name '*.mrk2' | grep '.*'".format(
p=path_to_merged_part)]) # check that we have adaptive files
path_to_old_part = node.query("SELECT path FROM system.parts WHERE table = 'table_with_default_granularity' AND active=1 ORDER BY partition ASC LIMIT 1").strip()
path_to_old_part = node.query(
"SELECT path FROM system.parts WHERE table = 'table_with_default_granularity' AND active=1 ORDER BY partition ASC LIMIT 1").strip()
node.exec_in_container(["bash", "-c", "find {p} -name '*.mrk' | grep '.*'".format(p=path_to_old_part)]) # check that we have non adaptive files
node.exec_in_container(["bash", "-c", "find {p} -name '*.mrk' | grep '.*'".format(
p=path_to_old_part)]) # check that we have non adaptive files
node.query("ALTER TABLE table_with_default_granularity UPDATE dummy = dummy + 1 WHERE 1")
# still works
@ -301,15 +319,19 @@ def test_mixed_granularity_single_node(start_dynamic_cluster, node):
node.query("ALTER TABLE table_with_default_granularity MODIFY COLUMN dummy String")
node.query("ALTER TABLE table_with_default_granularity ADD COLUMN dummy2 Float64")
#still works
# still works
assert node.query("SELECT count() from table_with_default_granularity") == '6\n'
@pytest.mark.skip(reason="flaky")
def test_version_update_two_nodes(start_dynamic_cluster):
node11.query("INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)")
node11.query(
"INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)")
node12.query("SYSTEM SYNC REPLICA table_with_default_granularity", timeout=20)
assert node12.query("SELECT COUNT() FROM table_with_default_granularity") == '2\n'
def callback(n):
<<<<<<< HEAD
new_config = """
<yandex><merge_tree>
<enable_mixed_granularity_parts>0</enable_mixed_granularity_parts>
@ -318,35 +340,44 @@ def test_version_update_two_nodes(start_dynamic_cluster):
n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml", new_config)
n.replace_config("/etc/clickhouse-server/config.d/merge_tree_settings.xml", new_config)
=======
n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml",
"<yandex><merge_tree><enable_mixed_granularity_parts>0</enable_mixed_granularity_parts></merge_tree></yandex>")
n.replace_config("/etc/clickhouse-server/config.d/merge_tree_settings.xml",
"<yandex><merge_tree><enable_mixed_granularity_parts>0</enable_mixed_granularity_parts></merge_tree></yandex>")
>>>>>>> upstream/master
node12.restart_with_latest_version(callback_onstop=callback)
node12.query("INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 3, 333), (toDate('2018-10-02'), 4, 444)")
node12.query(
"INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 3, 333), (toDate('2018-10-02'), 4, 444)")
node11.query("SYSTEM SYNC REPLICA table_with_default_granularity", timeout=20)
assert node11.query("SELECT COUNT() FROM table_with_default_granularity") == '4\n'
node12.query(
'''
CREATE TABLE table_with_default_granularity_new(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard11/table_with_default_granularity_new', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
'''
CREATE TABLE table_with_default_granularity_new(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard11/table_with_default_granularity_new', '2')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
node11.query(
'''
CREATE TABLE table_with_default_granularity_new(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard11/table_with_default_granularity_new', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
'''
CREATE TABLE table_with_default_granularity_new(date Date, id UInt32, dummy UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard11/table_with_default_granularity_new', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
''')
node12.query("INSERT INTO table_with_default_granularity_new VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)")
node12.query(
"INSERT INTO table_with_default_granularity_new VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)")
with pytest.raises(QueryTimeoutExceedException):
node11.query("SYSTEM SYNC REPLICA table_with_default_granularity_new", timeout=20)
node12.query("INSERT INTO table_with_default_granularity_new VALUES (toDate('2018-10-01'), 3, 333), (toDate('2018-10-02'), 4, 444)")
node12.query(
"INSERT INTO table_with_default_granularity_new VALUES (toDate('2018-10-01'), 3, 333), (toDate('2018-10-02'), 4, 444)")
node11.restart_with_latest_version(callback_onstop=callback) # just to be sure
node11.restart_with_latest_version(callback_onstop=callback) # just to be sure
for i in range(3):
try:
@ -362,7 +393,8 @@ def test_version_update_two_nodes(start_dynamic_cluster):
assert node11.query("SELECT COUNT() FROM table_with_default_granularity_new") == "4\n"
assert node12.query("SELECT COUNT() FROM table_with_default_granularity_new") == "4\n"
node11.query("INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 5, 333), (toDate('2018-10-02'), 6, 444)")
node11.query(
"INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 5, 333), (toDate('2018-10-02'), 6, 444)")
for i in range(3):
try:
node12.query("SYSTEM SYNC REPLICA table_with_default_granularity", timeout=120)

View File

@ -7,7 +7,9 @@ node1 = cluster.add_instance('node1', with_zookeeper=True)
node2 = cluster.add_instance('node2', with_zookeeper=True)
# no adaptive granularity by default
node3 = cluster.add_instance('node3', image='yandex/clickhouse-server', tag='19.9.5.36', with_installed_binary=True, stay_alive=True)
node3 = cluster.add_instance('node3', image='yandex/clickhouse-server', tag='19.9.5.36', with_installed_binary=True,
stay_alive=True)
@pytest.fixture(scope="module")
def start_cluster():
@ -20,7 +22,6 @@ def start_cluster():
def test_attach_detach(start_cluster):
node1.query("""
CREATE TABLE test (key UInt64)
ENGINE = ReplicatedMergeTree('/clickhouse/test', '1')
@ -58,7 +59,8 @@ def test_mutate_with_mixed_granularity(start_cluster):
ENGINE = MergeTree
ORDER BY key PARTITION BY date""")
node3.query("INSERT INTO test SELECT toDate('2019-10-01') + number % 5, number, toString(number), toString(number * number) FROM numbers(500)")
node3.query(
"INSERT INTO test SELECT toDate('2019-10-01') + number % 5, number, toString(number), toString(number * number) FROM numbers(500)")
assert node3.query("SELECT COUNT() FROM test") == "500\n"
@ -68,7 +70,8 @@ def test_mutate_with_mixed_granularity(start_cluster):
node3.query("ALTER TABLE test MODIFY SETTING enable_mixed_granularity_parts = 1")
node3.query("INSERT INTO test SELECT toDate('2019-10-01') + number % 5, number, toString(number), toString(number * number) FROM numbers(500, 500)")
node3.query(
"INSERT INTO test SELECT toDate('2019-10-01') + number % 5, number, toString(number), toString(number * number) FROM numbers(500, 500)")
assert node3.query("SELECT COUNT() FROM test") == "1000\n"
assert node3.query("SELECT COUNT() FROM test WHERE key % 100 == 0") == "10\n"

View File

@ -1,17 +1,15 @@
import time
import pytest
from helpers.cluster import ClickHouseCluster
from multiprocessing.dummy import Pool
from helpers.client import QueryRuntimeException, QueryTimeoutExceedException
from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', with_zookeeper=True)
node2 = cluster.add_instance('node2', with_zookeeper=True)
node3 = cluster.add_instance('node3', with_zookeeper=True, image='yandex/clickhouse-server', tag='19.1.14', with_installed_binary=True)
node3 = cluster.add_instance('node3', with_zookeeper=True, image='yandex/clickhouse-server', tag='19.1.14',
with_installed_binary=True)
@pytest.fixture(scope="module")
def start_cluster():
@ -23,11 +21,14 @@ def start_cluster():
finally:
cluster.shutdown()
def test_creating_table_different_setting(start_cluster):
node1.query("CREATE TABLE t1 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t1', '1') ORDER BY tuple(c1) SETTINGS index_granularity_bytes = 0")
node1.query(
"CREATE TABLE t1 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t1', '1') ORDER BY tuple(c1) SETTINGS index_granularity_bytes = 0")
node1.query("INSERT INTO t1 VALUES('x', 'y')")
node2.query("CREATE TABLE t1 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t1', '2') ORDER BY tuple(c1) SETTINGS enable_mixed_granularity_parts = 0")
node2.query(
"CREATE TABLE t1 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t1', '2') ORDER BY tuple(c1) SETTINGS enable_mixed_granularity_parts = 0")
node1.query("INSERT INTO t1 VALUES('a', 'b')")
node2.query("SYSTEM SYNC REPLICA t1", timeout=5)
@ -49,22 +50,26 @@ def test_creating_table_different_setting(start_cluster):
node1.query("SELECT count() FROM t1") == "3\n"
node2.query("SELECT count() FROM t1") == "2\n"
path_part = node1.query("SELECT path FROM system.parts WHERE table = 't1' AND active=1 ORDER BY partition DESC LIMIT 1").strip()
path_part = node1.query(
"SELECT path FROM system.parts WHERE table = 't1' AND active=1 ORDER BY partition DESC LIMIT 1").strip()
with pytest.raises(Exception): # check that we have no adaptive files
with pytest.raises(Exception): # check that we have no adaptive files
node1.exec_in_container(["bash", "-c", "find {p} -name '*.mrk2' | grep '.*'".format(p=path_part)])
path_part = node2.query("SELECT path FROM system.parts WHERE table = 't1' AND active=1 ORDER BY partition DESC LIMIT 1").strip()
path_part = node2.query(
"SELECT path FROM system.parts WHERE table = 't1' AND active=1 ORDER BY partition DESC LIMIT 1").strip()
with pytest.raises(Exception): # check that we have no adaptive files
with pytest.raises(Exception): # check that we have no adaptive files
node2.exec_in_container(["bash", "-c", "find {p} -name '*.mrk2' | grep '.*'".format(p=path_part)])
def test_old_node_with_new_node(start_cluster):
node3.query("CREATE TABLE t2 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t2', '3') ORDER BY tuple(c1)")
node3.query(
"CREATE TABLE t2 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t2', '3') ORDER BY tuple(c1)")
node3.query("INSERT INTO t2 VALUES('x', 'y')")
node2.query("CREATE TABLE t2 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t2', '2') ORDER BY tuple(c1) SETTINGS enable_mixed_granularity_parts = 0")
node2.query(
"CREATE TABLE t2 (c1 String, c2 String) ENGINE=ReplicatedMergeTree('/clickhouse/t2', '2') ORDER BY tuple(c1) SETTINGS enable_mixed_granularity_parts = 0")
node3.query("INSERT INTO t2 VALUES('a', 'b')")
node2.query("SYSTEM SYNC REPLICA t2", timeout=5)
@ -86,12 +91,14 @@ def test_old_node_with_new_node(start_cluster):
node3.query("SELECT count() FROM t2") == "3\n"
node2.query("SELECT count() FROM t2") == "2\n"
path_part = node3.query("SELECT path FROM system.parts WHERE table = 't2' AND active=1 ORDER BY partition DESC LIMIT 1").strip()
path_part = node3.query(
"SELECT path FROM system.parts WHERE table = 't2' AND active=1 ORDER BY partition DESC LIMIT 1").strip()
with pytest.raises(Exception): # check that we have no adaptive files
with pytest.raises(Exception): # check that we have no adaptive files
node3.exec_in_container(["bash", "-c", "find {p} -name '*.mrk2' | grep '.*'".format(p=path_part)])
path_part = node2.query("SELECT path FROM system.parts WHERE table = 't2' AND active=1 ORDER BY partition DESC LIMIT 1").strip()
path_part = node2.query(
"SELECT path FROM system.parts WHERE table = 't2' AND active=1 ORDER BY partition DESC LIMIT 1").strip()
with pytest.raises(Exception): # check that we have no adaptive files
with pytest.raises(Exception): # check that we have no adaptive files
node2.exec_in_container(["bash", "-c", "find {p} -name '*.mrk2' | grep '.*'".format(p=path_part)])

View File

@ -1,22 +1,20 @@
import time
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.client import QueryRuntimeException, QueryTimeoutExceedException
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1')
node2 = cluster.add_instance('node2')
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
for node in [node1, node2]:
node.query("create table da_memory_efficient_shard(A Int64, B Int64) Engine=MergeTree order by A partition by B % 2;")
node.query(
"create table da_memory_efficient_shard(A Int64, B Int64) Engine=MergeTree order by A partition by B % 2;")
node1.query("insert into da_memory_efficient_shard select number, number from numbers(100000);")
node2.query("insert into da_memory_efficient_shard select number + 100000, number from numbers(100000);")
@ -28,19 +26,24 @@ def start_cluster():
def test_remote(start_cluster):
node1.query("set distributed_aggregation_memory_efficient = 1, group_by_two_level_threshold = 1, group_by_two_level_threshold_bytes=1")
res = node1.query("select sum(a) from (SELECT B, uniqExact(A) a FROM remote('node{1,2}', default.da_memory_efficient_shard) GROUP BY B)")
node1.query(
"set distributed_aggregation_memory_efficient = 1, group_by_two_level_threshold = 1, group_by_two_level_threshold_bytes=1")
res = node1.query(
"select sum(a) from (SELECT B, uniqExact(A) a FROM remote('node{1,2}', default.da_memory_efficient_shard) GROUP BY B)")
assert res == '200000\n'
node1.query("set distributed_aggregation_memory_efficient = 0")
res = node1.query("select sum(a) from (SELECT B, uniqExact(A) a FROM remote('node{1,2}', default.da_memory_efficient_shard) GROUP BY B)")
res = node1.query(
"select sum(a) from (SELECT B, uniqExact(A) a FROM remote('node{1,2}', default.da_memory_efficient_shard) GROUP BY B)")
assert res == '200000\n'
node1.query("set distributed_aggregation_memory_efficient = 1, group_by_two_level_threshold = 1, group_by_two_level_threshold_bytes=1")
res = node1.query("SELECT fullHostName() AS h, uniqExact(A) AS a FROM remote('node{1,2}', default.da_memory_efficient_shard) GROUP BY h ORDER BY h;")
node1.query(
"set distributed_aggregation_memory_efficient = 1, group_by_two_level_threshold = 1, group_by_two_level_threshold_bytes=1")
res = node1.query(
"SELECT fullHostName() AS h, uniqExact(A) AS a FROM remote('node{1,2}', default.da_memory_efficient_shard) GROUP BY h ORDER BY h;")
assert res == 'node1\t100000\nnode2\t100000\n'
node1.query("set distributed_aggregation_memory_efficient = 0")
res = node1.query("SELECT fullHostName() AS h, uniqExact(A) AS a FROM remote('node{1,2}', default.da_memory_efficient_shard) GROUP BY h ORDER BY h;")
res = node1.query(
"SELECT fullHostName() AS h, uniqExact(A) AS a FROM remote('node{1,2}', default.da_memory_efficient_shard) GROUP BY h ORDER BY h;")
assert res == 'node1\t100000\nnode2\t100000\n'

View File

@ -1,32 +1,32 @@
import os
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
server = cluster.add_instance('server', user_configs=["configs/users.d/network.xml"])
clientA1 = cluster.add_instance('clientA1', hostname = 'clientA1.com')
clientA2 = cluster.add_instance('clientA2', hostname = 'clientA2.com')
clientA3 = cluster.add_instance('clientA3', hostname = 'clientA3.com')
clientB1 = cluster.add_instance('clientB1', hostname = 'clientB001.ru')
clientB2 = cluster.add_instance('clientB2', hostname = 'clientB002.ru')
clientB3 = cluster.add_instance('clientB3', hostname = 'xxx.clientB003.rutracker.com')
clientC1 = cluster.add_instance('clientC1', hostname = 'clientC01.ru')
clientC2 = cluster.add_instance('clientC2', hostname = 'xxx.clientC02.ru')
clientC3 = cluster.add_instance('clientC3', hostname = 'xxx.clientC03.rutracker.com')
clientD1 = cluster.add_instance('clientD1', hostname = 'clientD0001.ru')
clientD2 = cluster.add_instance('clientD2', hostname = 'xxx.clientD0002.ru')
clientD3 = cluster.add_instance('clientD3', hostname = 'clientD0003.ru')
clientA1 = cluster.add_instance('clientA1', hostname='clientA1.com')
clientA2 = cluster.add_instance('clientA2', hostname='clientA2.com')
clientA3 = cluster.add_instance('clientA3', hostname='clientA3.com')
clientB1 = cluster.add_instance('clientB1', hostname='clientB001.ru')
clientB2 = cluster.add_instance('clientB2', hostname='clientB002.ru')
clientB3 = cluster.add_instance('clientB3', hostname='xxx.clientB003.rutracker.com')
clientC1 = cluster.add_instance('clientC1', hostname='clientC01.ru')
clientC2 = cluster.add_instance('clientC2', hostname='xxx.clientC02.ru')
clientC3 = cluster.add_instance('clientC3', hostname='xxx.clientC03.rutracker.com')
clientD1 = cluster.add_instance('clientD1', hostname='clientD0001.ru')
clientD2 = cluster.add_instance('clientD2', hostname='xxx.clientD0002.ru')
clientD3 = cluster.add_instance('clientD3', hostname='clientD0003.ru')
def check_clickhouse_is_ok(client_node, server_node):
assert client_node.exec_in_container(["bash", "-c", "/usr/bin/curl -s {}:8123 ".format(server_node.hostname)]) == "Ok.\n"
assert client_node.exec_in_container(
["bash", "-c", "/usr/bin/curl -s {}:8123 ".format(server_node.hostname)]) == "Ok.\n"
def query_from_one_node_to_another(client_node, server_node, query):
check_clickhouse_is_ok(client_node, server_node)
return client_node.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host {} --query {!r}".format(server_node.hostname, query)])
return client_node.exec_in_container(
["bash", "-c", "/usr/bin/clickhouse client --host {} --query {!r}".format(server_node.hostname, query)])
def query(node, query):
@ -53,8 +53,8 @@ def test_allowed_host():
# Reverse DNS lookup currently isn't working as expected in this test.
# For example, it gives something like "vitbartestallowedclienthosts_clientB1_1.vitbartestallowedclienthosts_default" instead of "clientB001.ru".
# Maybe we should setup the test network better.
#expected_to_pass.extend([clientB1, clientB2, clientB3, clientC1, clientC2, clientD1, clientD3])
#expected_to_fail.extend([clientC3, clientD2])
# expected_to_pass.extend([clientB1, clientB2, clientB3, clientC1, clientC2, clientD1, clientD3])
# expected_to_fail.extend([clientC3, clientD2])
for client_node in expected_to_pass:
assert query_from_one_node_to_another(client_node, server, "SELECT * FROM test_table") == "5\n"

View File

@ -1,8 +1,6 @@
import time
import pytest
from helpers.hdfs_api import HDFSApi
from helpers.cluster import ClickHouseCluster
from helpers.hdfs_api import HDFSApi
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/config_with_hosts.xml'])
@ -21,26 +19,37 @@ def start_cluster():
finally:
cluster.shutdown()
def test_config_with_hosts(start_cluster):
assert node1.query("CREATE TABLE table_test_1_1 (word String) Engine=URL('http://host:80', HDFS)") == ""
assert node1.query("CREATE TABLE table_test_1_2 (word String) Engine=URL('https://yandex.ru', CSV)") == ""
assert "not allowed" in node1.query_and_get_error("CREATE TABLE table_test_1_4 (word String) Engine=URL('https://host:123', S3)")
assert "not allowed" in node1.query_and_get_error("CREATE TABLE table_test_1_4 (word String) Engine=URL('https://yandex2.ru', CSV)")
assert "not allowed" in node1.query_and_get_error(
"CREATE TABLE table_test_1_4 (word String) Engine=URL('https://host:123', S3)")
assert "not allowed" in node1.query_and_get_error(
"CREATE TABLE table_test_1_4 (word String) Engine=URL('https://yandex2.ru', CSV)")
def test_config_with_only_primary_hosts(start_cluster):
assert node2.query("CREATE TABLE table_test_2_1 (word String) Engine=URL('https://host:80', CSV)") == ""
assert node2.query("CREATE TABLE table_test_2_2 (word String) Engine=URL('https://host:123', S3)") == ""
assert node2.query("CREATE TABLE table_test_2_3 (word String) Engine=URL('https://yandex.ru', CSV)") == ""
assert node2.query("CREATE TABLE table_test_2_4 (word String) Engine=URL('https://yandex.ru:87', HDFS)") == ""
assert "not allowed" in node2.query_and_get_error("CREATE TABLE table_test_2_5 (word String) Engine=URL('https://host', HDFS)")
assert "not allowed" in node2.query_and_get_error("CREATE TABLE table_test_2_5 (word String) Engine=URL('https://host:234', CSV)")
assert "not allowed" in node2.query_and_get_error("CREATE TABLE table_test_2_6 (word String) Engine=URL('https://yandex2.ru', S3)")
assert "not allowed" in node2.query_and_get_error(
"CREATE TABLE table_test_2_5 (word String) Engine=URL('https://host', HDFS)")
assert "not allowed" in node2.query_and_get_error(
"CREATE TABLE table_test_2_5 (word String) Engine=URL('https://host:234', CSV)")
assert "not allowed" in node2.query_and_get_error(
"CREATE TABLE table_test_2_6 (word String) Engine=URL('https://yandex2.ru', S3)")
def test_config_with_only_regexp_hosts(start_cluster):
assert node3.query("CREATE TABLE table_test_3_1 (word String) Engine=URL('https://host:80', HDFS)") == ""
assert node3.query("CREATE TABLE table_test_3_2 (word String) Engine=URL('https://yandex.ru', CSV)") == ""
assert "not allowed" in node3.query_and_get_error("CREATE TABLE table_test_3_3 (word String) Engine=URL('https://host', CSV)")
assert "not allowed" in node3.query_and_get_error("CREATE TABLE table_test_3_4 (word String) Engine=URL('https://yandex2.ru', S3)")
assert "not allowed" in node3.query_and_get_error(
"CREATE TABLE table_test_3_3 (word String) Engine=URL('https://host', CSV)")
assert "not allowed" in node3.query_and_get_error(
"CREATE TABLE table_test_3_4 (word String) Engine=URL('https://yandex2.ru', S3)")
def test_config_without_allowed_hosts(start_cluster):
assert node4.query("CREATE TABLE table_test_4_1 (word String) Engine=URL('https://host:80', CSV)") == ""
@ -48,27 +57,60 @@ def test_config_without_allowed_hosts(start_cluster):
assert node4.query("CREATE TABLE table_test_4_3 (word String) Engine=URL('https://yandex.ru', CSV)") == ""
assert node4.query("CREATE TABLE table_test_4_4 (word String) Engine=URL('ftp://something.com', S3)") == ""
def test_table_function_remote(start_cluster):
assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-{1|2}', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1})
assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-1,example01-02-1', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1})
assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remote('example01-0{1,2}-1', system, events", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1})
assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remote('example01-0{1,2}-{1|2}', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1})
assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-{01..02}-{1|2}', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1})
assert "not allowed" in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-1,example01-03-1', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1})
assert "not allowed" in node6.query_and_get_error("SELECT * FROM remote('example01-01-{1|3}', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1})
assert "not allowed" in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-0{1,3}-1', system, metrics)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1})
assert "not allowed in config.xml" not in node6.query_and_get_error(
"SELECT * FROM remoteSecure('example01-01-{1|2}', system, events)",
settings={"connections_with_failover_max_tries": 1, "connect_timeout_with_failover_ms": 1000,
"connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout": 1})
assert "not allowed in config.xml" not in node6.query_and_get_error(
"SELECT * FROM remoteSecure('example01-01-1,example01-02-1', system, events)",
settings={"connections_with_failover_max_tries": 1, "connect_timeout_with_failover_ms": 1000,
"connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout": 1})
assert "not allowed in config.xml" not in node6.query_and_get_error(
"SELECT * FROM remote('example01-0{1,2}-1', system, events",
settings={"connections_with_failover_max_tries": 1, "connect_timeout_with_failover_ms": 1000,
"connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout": 1})
assert "not allowed in config.xml" not in node6.query_and_get_error(
"SELECT * FROM remote('example01-0{1,2}-{1|2}', system, events)",
settings={"connections_with_failover_max_tries": 1, "connect_timeout_with_failover_ms": 1000,
"connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout": 1})
assert "not allowed in config.xml" not in node6.query_and_get_error(
"SELECT * FROM remoteSecure('example01-{01..02}-{1|2}', system, events)",
settings={"connections_with_failover_max_tries": 1, "connect_timeout_with_failover_ms": 1000,
"connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout": 1})
assert "not allowed" in node6.query_and_get_error(
"SELECT * FROM remoteSecure('example01-01-1,example01-03-1', system, events)",
settings={"connections_with_failover_max_tries": 1, "connect_timeout_with_failover_ms": 1000,
"connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout": 1})
assert "not allowed" in node6.query_and_get_error("SELECT * FROM remote('example01-01-{1|3}', system, events)",
settings={"connections_with_failover_max_tries": 1,
"connect_timeout_with_failover_ms": 1000,
"connect_timeout_with_failover_secure_ms": 1000,
"connect_timeout": 1, "send_timeout": 1})
assert "not allowed" in node6.query_and_get_error(
"SELECT * FROM remoteSecure('example01-0{1,3}-1', system, metrics)",
settings={"connections_with_failover_max_tries": 1, "connect_timeout_with_failover_ms": 1000,
"connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout": 1})
assert node6.query("SELECT * FROM remote('localhost', system, events)") != ""
assert node6.query("SELECT * FROM remoteSecure('localhost', system, metrics)") != ""
assert "URL \"localhost:800\" is not allowed in config.xml" in node6.query_and_get_error("SELECT * FROM remoteSecure('localhost:800', system, events)")
assert "URL \"localhost:800\" is not allowed in config.xml" in node6.query_and_get_error("SELECT * FROM remote('localhost:800', system, metrics)")
assert "URL \"localhost:800\" is not allowed in config.xml" in node6.query_and_get_error(
"SELECT * FROM remoteSecure('localhost:800', system, events)")
assert "URL \"localhost:800\" is not allowed in config.xml" in node6.query_and_get_error(
"SELECT * FROM remote('localhost:800', system, metrics)")
def test_redirect(start_cluster):
hdfs_api = HDFSApi("root")
hdfs_api.write_data("/simple_storage", "1\t\n")
assert hdfs_api.read_data("/simple_storage") == "1\t\n"
node7.query("CREATE TABLE table_test_7_1 (word String) ENGINE=URL('http://hdfs1:50070/webhdfs/v1/simple_storage?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', CSV)")
node7.query(
"CREATE TABLE table_test_7_1 (word String) ENGINE=URL('http://hdfs1:50070/webhdfs/v1/simple_storage?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', CSV)")
assert "not allowed" in node7.query_and_get_error("SET max_http_get_redirects=1; SELECT * from table_test_7_1")
def test_HDFS(start_cluster):
assert "not allowed" in node7.query_and_get_error("CREATE TABLE table_test_7_2 (word String) ENGINE=HDFS('http://hdfs1:50075/webhdfs/v1/simple_storage?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'CSV')")
assert "not allowed" in node7.query_and_get_error("SELECT * FROM hdfs('http://hdfs1:50075/webhdfs/v1/simple_storage?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV', 'word String')")
assert "not allowed" in node7.query_and_get_error(
"CREATE TABLE table_test_7_2 (word String) ENGINE=HDFS('http://hdfs1:50075/webhdfs/v1/simple_storage?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'CSV')")
assert "not allowed" in node7.query_and_get_error(
"SELECT * FROM hdfs('http://hdfs1:50075/webhdfs/v1/simple_storage?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', 'TSV', 'word String')")

View File

@ -2,14 +2,13 @@ import pytest
from helpers.client import QueryRuntimeException
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1',
main_configs=['configs/logs_config.xml'])
main_configs=['configs/logs_config.xml'])
node2 = cluster.add_instance('node2',
main_configs=['configs/logs_config.xml'])
main_configs=['configs/logs_config.xml'])
@pytest.fixture(scope="module")
@ -39,7 +38,6 @@ def test_alter_codec_pk(started_cluster):
with pytest.raises(QueryRuntimeException):
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt32 CODEC(Delta, LZ4)".format(name=name))
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 DEFAULT 3 CODEC(Delta, LZ4)".format(name=name))
node1.query("INSERT INTO {name} (value) VALUES (1)".format(name=name))

View File

@ -1,14 +1,15 @@
import pytest
import time
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', with_zookeeper=True)
node2 = cluster.add_instance('node2', with_zookeeper=True)
@pytest.fixture(scope="module")
def started_cluster():
try:

View File

@ -39,7 +39,7 @@ def test_event_time_microseconds_field(started_cluster):
node1.query(query_create)
node1.query('''INSERT INTO replica.test VALUES (1, now())''')
node1.query("SYSTEM FLUSH LOGS;")
#query assumes that the event_time field is accurate
# query assumes that the event_time field is accurate
equals_query = '''WITH (
(
SELECT event_time_microseconds

View File

@ -1,12 +1,12 @@
import time
import pytest
from helpers.network import PartitionManager
from helpers.cluster import ClickHouseCluster
from helpers.network import PartitionManager
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=["configs/config.d/zookeeper_session_timeout.xml", "configs/remote_servers.xml"], with_zookeeper=True)
node1 = cluster.add_instance('node1', main_configs=["configs/config.d/zookeeper_session_timeout.xml",
"configs/remote_servers.xml"], with_zookeeper=True)
@pytest.fixture(scope="module")
@ -25,12 +25,13 @@ def start_cluster():
finally:
cluster.shutdown()
def test_atomic_delete_with_stopped_zookeeper(start_cluster):
node1.query("insert into zktest.atomic_drop_table values (8192)")
with PartitionManager() as pm:
pm.drop_instance_zk_connections(node1)
error = node1.query_and_get_error("DROP TABLE zktest.atomic_drop_table") #Table won't drop
error = node1.query_and_get_error("DROP TABLE zktest.atomic_drop_table") # Table won't drop
assert error != ""
time.sleep(5)

View File

@ -17,7 +17,8 @@ def start_cluster():
def test_attach_without_checksums(start_cluster):
node1.query("CREATE TABLE test (date Date, key Int32, value String) Engine=MergeTree ORDER BY key PARTITION by date")
node1.query(
"CREATE TABLE test (date Date, key Int32, value String) Engine=MergeTree ORDER BY key PARTITION by date")
node1.query("INSERT INTO test SELECT toDate('2019-10-01'), number, toString(number) FROM numbers(100)")
@ -29,9 +30,13 @@ def test_attach_without_checksums(start_cluster):
assert node1.query("SELECT COUNT() FROM test") == "0\n"
# to be sure output not empty
node1.exec_in_container(['bash', '-c', 'find /var/lib/clickhouse/data/default/test/detached -name "checksums.txt" | grep -e ".*" '], privileged=True, user='root')
node1.exec_in_container(
['bash', '-c', 'find /var/lib/clickhouse/data/default/test/detached -name "checksums.txt" | grep -e ".*" '],
privileged=True, user='root')
node1.exec_in_container(['bash', '-c', 'find /var/lib/clickhouse/data/default/test/detached -name "checksums.txt" -delete'], privileged=True, user='root')
node1.exec_in_container(
['bash', '-c', 'find /var/lib/clickhouse/data/default/test/detached -name "checksums.txt" -delete'],
privileged=True, user='root')
node1.query("ALTER TABLE test ATTACH PARTITION '2019-10-01'")

View File

@ -30,7 +30,8 @@ def test_authentication_pass():
def test_authentication_fail():
# User doesn't exist.
assert "vasya: Authentication failed" in instance.query_and_get_error("SELECT currentUser()", user = 'vasya')
assert "vasya: Authentication failed" in instance.query_and_get_error("SELECT currentUser()", user='vasya')
# Wrong password.
assert "masha: Authentication failed" in instance.query_and_get_error("SELECT currentUser()", user = 'masha', password = '123')
assert "masha: Authentication failed" in instance.query_and_get_error("SELECT currentUser()", user='masha',
password='123')

View File

@ -1,10 +1,9 @@
import os.path
import pytest
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance')
q = instance.query
@ -47,7 +46,7 @@ def backup_restore(started_cluster):
expected = TSV('1970-01-02\t1\n1970-01-03\t2\n1970-01-04\t3\n1970-02-01\t31\n1970-02-02\t32\n1970-02-03\t33')
res = q("SELECT * FROM test.tbl ORDER BY p")
assert(TSV(res) == expected)
assert (TSV(res) == expected)
q("ALTER TABLE test.tbl FREEZE")
@ -69,7 +68,7 @@ def test_restore(backup_restore):
# Validate the attached parts are identical to the backup.
expected = TSV('1970-01-02\t1\n1970-01-03\t2\n1970-01-04\t3\n1970-02-01\t31\n1970-02-02\t32\n1970-02-03\t33')
res = q("SELECT * FROM test.tbl1 ORDER BY p")
assert(TSV(res) == expected)
assert (TSV(res) == expected)
q("ALTER TABLE test.tbl1 UPDATE k=10 WHERE 1")
q("SELECT sleep(2)")
@ -77,7 +76,7 @@ def test_restore(backup_restore):
# Validate mutation has been applied to all attached parts.
expected = TSV('1970-01-02\t10\n1970-01-03\t10\n1970-01-04\t10\n1970-02-01\t10\n1970-02-02\t10\n1970-02-03\t10')
res = q("SELECT * FROM test.tbl1 ORDER BY p")
assert(TSV(res) == expected)
assert (TSV(res) == expected)
q("DROP TABLE IF EXISTS test.tbl1")
@ -91,7 +90,7 @@ def test_attach_partition(backup_restore):
expected = TSV('1970-01-04\t3\n1970-01-05\t4\n1970-02-03\t33\n1970-02-04\t34')
res = q("SELECT * FROM test.tbl2 ORDER BY p")
assert(TSV(res) == expected)
assert (TSV(res) == expected)
copy_backup_to_detached('test', 'tbl', 'tbl2')
@ -102,17 +101,19 @@ def test_attach_partition(backup_restore):
q("ALTER TABLE test.tbl2 ATTACH PARTITION 197002")
q("SELECT sleep(2)")
expected = TSV('1970-01-02\t1\n1970-01-03\t2\n1970-01-04\t3\n1970-01-04\t3\n1970-01-05\t4\n1970-02-01\t31\n1970-02-02\t32\n1970-02-03\t33\n1970-02-03\t33\n1970-02-04\t34')
expected = TSV(
'1970-01-02\t1\n1970-01-03\t2\n1970-01-04\t3\n1970-01-04\t3\n1970-01-05\t4\n1970-02-01\t31\n1970-02-02\t32\n1970-02-03\t33\n1970-02-03\t33\n1970-02-04\t34')
res = q("SELECT * FROM test.tbl2 ORDER BY p")
assert(TSV(res) == expected)
assert (TSV(res) == expected)
q("ALTER TABLE test.tbl2 UPDATE k=10 WHERE 1")
q("SELECT sleep(2)")
# Validate mutation has been applied to all attached parts.
expected = TSV('1970-01-02\t10\n1970-01-03\t10\n1970-01-04\t10\n1970-01-04\t10\n1970-01-05\t10\n1970-02-01\t10\n1970-02-02\t10\n1970-02-03\t10\n1970-02-03\t10\n1970-02-04\t10')
expected = TSV(
'1970-01-02\t10\n1970-01-03\t10\n1970-01-04\t10\n1970-01-04\t10\n1970-01-05\t10\n1970-02-01\t10\n1970-02-02\t10\n1970-02-03\t10\n1970-02-03\t10\n1970-02-04\t10')
res = q("SELECT * FROM test.tbl2 ORDER BY p")
assert(TSV(res) == expected)
assert (TSV(res) == expected)
q("DROP TABLE IF EXISTS test.tbl2")
@ -126,7 +127,7 @@ def test_replace_partition(backup_restore):
expected = TSV('1970-01-04\t3\n1970-01-05\t4\n1970-02-03\t33\n1970-02-04\t34')
res = q("SELECT * FROM test.tbl3 ORDER BY p")
assert(TSV(res) == expected)
assert (TSV(res) == expected)
copy_backup_to_detached('test', 'tbl', 'tbl3')
@ -138,7 +139,7 @@ def test_replace_partition(backup_restore):
expected = TSV('1970-01-04\t3\n1970-01-05\t4\n1970-02-01\t31\n1970-02-02\t32\n1970-02-03\t33')
res = q("SELECT * FROM test.tbl3 ORDER BY p")
assert(TSV(res) == expected)
assert (TSV(res) == expected)
q("ALTER TABLE test.tbl3 UPDATE k=10 WHERE 1")
q("SELECT sleep(2)")
@ -146,6 +147,6 @@ def test_replace_partition(backup_restore):
# Validate mutation has been applied to all copied parts.
expected = TSV('1970-01-04\t10\n1970-01-05\t10\n1970-02-01\t10\n1970-02-02\t10\n1970-02-03\t10')
res = q("SELECT * FROM test.tbl3 ORDER BY p")
assert(TSV(res) == expected)
assert (TSV(res) == expected)
q("DROP TABLE IF EXISTS test.tbl3")

View File

@ -1,13 +1,15 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', with_zookeeper=True, image='yandex/clickhouse-server', tag='19.4.5.35', stay_alive=True, with_installed_binary=True)
node2 = cluster.add_instance('node2', with_zookeeper=True, image='yandex/clickhouse-server', tag='19.4.5.35', stay_alive=True, with_installed_binary=True)
node3 = cluster.add_instance('node3', with_zookeeper=True, image='yandex/clickhouse-server', tag='19.4.5.35', stay_alive=True, with_installed_binary=True)
node1 = cluster.add_instance('node1', with_zookeeper=True, image='yandex/clickhouse-server', tag='19.4.5.35',
stay_alive=True, with_installed_binary=True)
node2 = cluster.add_instance('node2', with_zookeeper=True, image='yandex/clickhouse-server', tag='19.4.5.35',
stay_alive=True, with_installed_binary=True)
node3 = cluster.add_instance('node3', with_zookeeper=True, image='yandex/clickhouse-server', tag='19.4.5.35',
stay_alive=True, with_installed_binary=True)
node4 = cluster.add_instance('node4')
@ -33,13 +35,15 @@ def test_backup_from_old_version(started_cluster):
node1.restart_with_latest_version()
node1.query("CREATE TABLE dest_table (A Int64, B String, Y String) ENGINE = ReplicatedMergeTree('/test/dest_table1', '1') ORDER BY tuple()")
node1.query(
"CREATE TABLE dest_table (A Int64, B String, Y String) ENGINE = ReplicatedMergeTree('/test/dest_table1', '1') ORDER BY tuple()")
node1.query("INSERT INTO dest_table VALUES(2, '2', 'Hello')")
assert node1.query("SELECT COUNT() FROM dest_table") == "1\n"
node1.exec_in_container(['bash', '-c', 'cp -r /var/lib/clickhouse/shadow/1/data/default/source_table/all_1_1_0/ /var/lib/clickhouse/data/default/dest_table/detached'])
node1.exec_in_container(['bash', '-c',
'cp -r /var/lib/clickhouse/shadow/1/data/default/source_table/all_1_1_0/ /var/lib/clickhouse/data/default/dest_table/detached'])
assert node1.query("SELECT COUNT() FROM dest_table") == "1\n"
@ -69,13 +73,15 @@ def test_backup_from_old_version_setting(started_cluster):
node2.restart_with_latest_version()
node2.query("CREATE TABLE dest_table (A Int64, B String, Y String) ENGINE = ReplicatedMergeTree('/test/dest_table2', '1') ORDER BY tuple() SETTINGS enable_mixed_granularity_parts = 1")
node2.query(
"CREATE TABLE dest_table (A Int64, B String, Y String) ENGINE = ReplicatedMergeTree('/test/dest_table2', '1') ORDER BY tuple() SETTINGS enable_mixed_granularity_parts = 1")
node2.query("INSERT INTO dest_table VALUES(2, '2', 'Hello')")
assert node2.query("SELECT COUNT() FROM dest_table") == "1\n"
node2.exec_in_container(['bash', '-c', 'cp -r /var/lib/clickhouse/shadow/1/data/default/source_table/all_1_1_0/ /var/lib/clickhouse/data/default/dest_table/detached'])
node2.exec_in_container(['bash', '-c',
'cp -r /var/lib/clickhouse/shadow/1/data/default/source_table/all_1_1_0/ /var/lib/clickhouse/data/default/dest_table/detached'])
assert node2.query("SELECT COUNT() FROM dest_table") == "1\n"
@ -89,7 +95,7 @@ def test_backup_from_old_version_setting(started_cluster):
assert node2.query("SELECT sum(A) FROM dest_table") == "3\n"
assert node1.query("CHECK TABLE dest_table") == "1\n"
assert node2.query("CHECK TABLE dest_table") == "1\n"
def test_backup_from_old_version_config(started_cluster):
@ -104,17 +110,20 @@ def test_backup_from_old_version_config(started_cluster):
node3.query("ALTER TABLE source_table FREEZE PARTITION tuple();")
def callback(n):
n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml", "<yandex><merge_tree><enable_mixed_granularity_parts>1</enable_mixed_granularity_parts></merge_tree></yandex>")
n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml",
"<yandex><merge_tree><enable_mixed_granularity_parts>1</enable_mixed_granularity_parts></merge_tree></yandex>")
node3.restart_with_latest_version(callback_onstop=callback)
node3.query("CREATE TABLE dest_table (A Int64, B String, Y String) ENGINE = ReplicatedMergeTree('/test/dest_table3', '1') ORDER BY tuple() SETTINGS enable_mixed_granularity_parts = 1")
node3.query(
"CREATE TABLE dest_table (A Int64, B String, Y String) ENGINE = ReplicatedMergeTree('/test/dest_table3', '1') ORDER BY tuple() SETTINGS enable_mixed_granularity_parts = 1")
node3.query("INSERT INTO dest_table VALUES(2, '2', 'Hello')")
assert node3.query("SELECT COUNT() FROM dest_table") == "1\n"
node3.exec_in_container(['bash', '-c', 'cp -r /var/lib/clickhouse/shadow/1/data/default/source_table/all_1_1_0/ /var/lib/clickhouse/data/default/dest_table/detached'])
node3.exec_in_container(['bash', '-c',
'cp -r /var/lib/clickhouse/shadow/1/data/default/source_table/all_1_1_0/ /var/lib/clickhouse/data/default/dest_table/detached'])
assert node3.query("SELECT COUNT() FROM dest_table") == "1\n"
@ -128,7 +137,7 @@ def test_backup_from_old_version_config(started_cluster):
assert node3.query("SELECT sum(A) FROM dest_table") == "3\n"
assert node1.query("CHECK TABLE dest_table") == "1\n"
assert node3.query("CHECK TABLE dest_table") == "1\n"
def test_backup_and_alter(started_cluster):
@ -144,7 +153,8 @@ def test_backup_and_alter(started_cluster):
node4.query("ALTER TABLE backup_table DROP PARTITION tuple()")
node4.exec_in_container(['bash', '-c', 'cp -r /var/lib/clickhouse/shadow/1/data/default/backup_table/all_1_1_0/ /var/lib/clickhouse/data/default/backup_table/detached'])
node4.exec_in_container(['bash', '-c',
'cp -r /var/lib/clickhouse/shadow/1/data/default/backup_table/all_1_1_0/ /var/lib/clickhouse/data/default/backup_table/detached'])
node4.query("ALTER TABLE backup_table ATTACH PARTITION tuple()")

View File

@ -1,22 +1,22 @@
import pytest
import helpers.client as client
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', with_zookeeper=True, image='yandex/clickhouse-server', tag='19.17.8.54', stay_alive=True, with_installed_binary=True)
node2 = cluster.add_instance('node2', main_configs=['configs/wide_parts_only.xml'], with_zookeeper=True)
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
for i, node in enumerate([node1, node2]):
node.query(
'''CREATE TABLE t(date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/t', '{}')
PARTITION BY toYYYYMM(date)
ORDER BY id'''.format(i))
'''CREATE TABLE t(date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/t', '{}')
PARTITION BY toYYYYMM(date)
ORDER BY id'''.format(i))
yield cluster

View File

@ -1,16 +1,18 @@
import pytest
import helpers.client as client
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1',
with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37', stay_alive=True, with_installed_binary=True)
with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37', stay_alive=True,
with_installed_binary=True)
node2 = cluster.add_instance('node2',
with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37', stay_alive=True, with_installed_binary=True)
with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37', stay_alive=True,
with_installed_binary=True)
node3 = cluster.add_instance('node3', with_zookeeper=False)
node4 = cluster.add_instance('node4', with_zookeeper=False)
@pytest.fixture(scope="module")
def start_cluster():
try:
@ -20,6 +22,7 @@ def start_cluster():
finally:
cluster.shutdown()
# We will test that serialization of internal state of "avg" function is compatible between different versions.
# TODO Implement versioning of serialization format for aggregate function states.
# NOTE This test is too ad-hoc.
@ -35,18 +38,18 @@ def test_backward_compatability(start_cluster):
node3.query("INSERT INTO tab VALUES (3)")
node4.query("INSERT INTO tab VALUES (4)")
assert(node1.query("SELECT avg(x) FROM remote('node{1..4}', default, tab)") == '2.5\n')
assert(node2.query("SELECT avg(x) FROM remote('node{1..4}', default, tab)") == '2.5\n')
assert(node3.query("SELECT avg(x) FROM remote('node{1..4}', default, tab)") == '2.5\n')
assert(node4.query("SELECT avg(x) FROM remote('node{1..4}', default, tab)") == '2.5\n')
assert (node1.query("SELECT avg(x) FROM remote('node{1..4}', default, tab)") == '2.5\n')
assert (node2.query("SELECT avg(x) FROM remote('node{1..4}', default, tab)") == '2.5\n')
assert (node3.query("SELECT avg(x) FROM remote('node{1..4}', default, tab)") == '2.5\n')
assert (node4.query("SELECT avg(x) FROM remote('node{1..4}', default, tab)") == '2.5\n')
# Also check with persisted aggregate function state
node1.query("create table state (x AggregateFunction(avg, UInt64)) engine = Log")
node1.query("INSERT INTO state SELECT avgState(arrayJoin(CAST([1, 2, 3, 4] AS Array(UInt64))))")
assert(node1.query("SELECT avgMerge(x) FROM state") == '2.5\n')
assert (node1.query("SELECT avgMerge(x) FROM state") == '2.5\n')
node1.restart_with_latest_version()
assert(node1.query("SELECT avgMerge(x) FROM state") == '2.5\n')
assert (node1.query("SELECT avgMerge(x) FROM state") == '2.5\n')

View File

@ -1,13 +1,15 @@
import pytest
import helpers.client as client
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37', stay_alive=True, with_installed_binary=True)
node2 = cluster.add_instance('node2', with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37', stay_alive=True, with_installed_binary=True)
node1 = cluster.add_instance('node1', with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37',
stay_alive=True, with_installed_binary=True)
node2 = cluster.add_instance('node2', with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37',
stay_alive=True, with_installed_binary=True)
node3 = cluster.add_instance('node3', with_zookeeper=False)
@pytest.fixture(scope="module")
def start_cluster():
try:
@ -23,6 +25,7 @@ def test_backward_compatability(start_cluster):
node2.query("create table tab (s String) engine = MergeTree order by s")
node1.query("insert into tab select number from numbers(50)")
node2.query("insert into tab select number from numbers(1000000)")
res = node3.query("select s, count() from remote('node{1,2}', default, tab) group by s order by toUInt64(s) limit 50")
res = node3.query(
"select s, count() from remote('node{1,2}', default, tab) group by s order by toUInt64(s) limit 50")
print(res)
assert res == ''.join('{}\t2\n'.format(i) for i in range(50))

View File

@ -1,7 +1,5 @@
import time
import pytest
from contextlib import contextmanager
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
@ -9,7 +7,8 @@ cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'])
node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'])
#test reproducing issue https://github.com/ClickHouse/ClickHouse/issues/3162
# test reproducing issue https://github.com/ClickHouse/ClickHouse/issues/3162
@pytest.fixture(scope="module")
def started_cluster():
try:
@ -44,7 +43,9 @@ CREATE TABLE dist_test (
finally:
cluster.shutdown()
def test(started_cluster):
node1.query("INSERT INTO local_test (t, shard, col1, col2) VALUES (1000, 0, 'x', 'y')")
node2.query("INSERT INTO local_test (t, shard, col1, col2) VALUES (1000, 1, 'foo', 'bar')")
assert node1.query("SELECT col1, col2 FROM dist_test WHERE (t < 3600000) AND (col1 = 'foo') ORDER BY t ASC") == "foo\tbar\n"
assert node1.query(
"SELECT col1, col2 FROM dist_test WHERE (t < 3600000) AND (col1 = 'foo') ORDER BY t ASC") == "foo\tbar\n"

View File

@ -1,9 +1,6 @@
import time
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__)
@ -35,15 +32,22 @@ def started_cluster():
def corrupt_data_part_on_disk(node, table, part_name):
part_path = node.query("SELECT path FROM system.parts WHERE table = '{}' and name = '{}'".format(table, part_name)).strip()
node.exec_in_container(['bash', '-c', 'cd {p} && ls *.bin | head -n 1 | xargs -I{{}} sh -c \'echo "1" >> $1\' -- {{}}'.format(p=part_path)], privileged=True)
part_path = node.query(
"SELECT path FROM system.parts WHERE table = '{}' and name = '{}'".format(table, part_name)).strip()
node.exec_in_container(['bash', '-c',
'cd {p} && ls *.bin | head -n 1 | xargs -I{{}} sh -c \'echo "1" >> $1\' -- {{}}'.format(
p=part_path)], privileged=True)
def remove_checksums_on_disk(node, table, part_name):
part_path = node.query("SELECT path FROM system.parts WHERE table = '{}' and name = '{}'".format(table, part_name)).strip()
part_path = node.query(
"SELECT path FROM system.parts WHERE table = '{}' and name = '{}'".format(table, part_name)).strip()
node.exec_in_container(['bash', '-c', 'rm -r {p}/checksums.txt'.format(p=part_path)], privileged=True)
def remove_part_from_disk(node, table, part_name):
part_path = node.query("SELECT path FROM system.parts WHERE table = '{}' and name = '{}'".format(table, part_name)).strip()
part_path = node.query(
"SELECT path FROM system.parts WHERE table = '{}' and name = '{}'".format(table, part_name)).strip()
if not part_path:
raise Exception("Part " + part_name + "doesn't exist")
node.exec_in_container(['bash', '-c', 'rm -r {p}/*'.format(p=part_path)], privileged=True)
@ -51,35 +55,42 @@ def remove_part_from_disk(node, table, part_name):
def test_check_normal_table_corruption(started_cluster):
node1.query("INSERT INTO non_replicated_mt VALUES (toDate('2019-02-01'), 1, 10), (toDate('2019-02-01'), 2, 12)")
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201902", settings={"check_query_single_value_result": 0}) == "201902_1_1_0\t1\t\n"
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201902",
settings={"check_query_single_value_result": 0}) == "201902_1_1_0\t1\t\n"
remove_checksums_on_disk(node1, "non_replicated_mt", "201902_1_1_0")
assert node1.query("CHECK TABLE non_replicated_mt", settings={"check_query_single_value_result": 0}).strip() == "201902_1_1_0\t1\tChecksums recounted and written to disk."
assert node1.query("CHECK TABLE non_replicated_mt", settings={
"check_query_single_value_result": 0}).strip() == "201902_1_1_0\t1\tChecksums recounted and written to disk."
assert node1.query("SELECT COUNT() FROM non_replicated_mt") == "2\n"
remove_checksums_on_disk(node1, "non_replicated_mt", "201902_1_1_0")
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201902", settings={"check_query_single_value_result": 0}).strip() == "201902_1_1_0\t1\tChecksums recounted and written to disk."
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201902", settings={
"check_query_single_value_result": 0}).strip() == "201902_1_1_0\t1\tChecksums recounted and written to disk."
assert node1.query("SELECT COUNT() FROM non_replicated_mt") == "2\n"
corrupt_data_part_on_disk(node1, "non_replicated_mt", "201902_1_1_0")
assert node1.query("CHECK TABLE non_replicated_mt", settings={"check_query_single_value_result": 0}).strip() == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 16."
assert node1.query("CHECK TABLE non_replicated_mt", settings={
"check_query_single_value_result": 0}).strip() == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 16."
assert node1.query("CHECK TABLE non_replicated_mt", settings={"check_query_single_value_result": 0}).strip() == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 16."
assert node1.query("CHECK TABLE non_replicated_mt", settings={
"check_query_single_value_result": 0}).strip() == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 16."
node1.query("INSERT INTO non_replicated_mt VALUES (toDate('2019-01-01'), 1, 10), (toDate('2019-01-01'), 2, 12)")
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "201901_2_2_0\t1\t\n"
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201901",
settings={"check_query_single_value_result": 0}) == "201901_2_2_0\t1\t\n"
corrupt_data_part_on_disk(node1, "non_replicated_mt", "201901_2_2_0")
remove_checksums_on_disk(node1, "non_replicated_mt", "201901_2_2_0")
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "201901_2_2_0\t0\tCheck of part finished with error: \\'Cannot read all data. Bytes read: 2. Bytes expected: 16.\\'\n"
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201901", settings={
"check_query_single_value_result": 0}) == "201901_2_2_0\t0\tCheck of part finished with error: \\'Cannot read all data. Bytes read: 2. Bytes expected: 16.\\'\n"
def test_check_replicated_table_simple(started_cluster):
@ -91,16 +102,20 @@ def test_check_replicated_table_simple(started_cluster):
assert node1.query("SELECT count() from replicated_mt") == "2\n"
assert node2.query("SELECT count() from replicated_mt") == "2\n"
assert node1.query("CHECK TABLE replicated_mt", settings={"check_query_single_value_result": 0}) == "201902_0_0_0\t1\t\n"
assert node2.query("CHECK TABLE replicated_mt", settings={"check_query_single_value_result": 0}) == "201902_0_0_0\t1\t\n"
assert node1.query("CHECK TABLE replicated_mt",
settings={"check_query_single_value_result": 0}) == "201902_0_0_0\t1\t\n"
assert node2.query("CHECK TABLE replicated_mt",
settings={"check_query_single_value_result": 0}) == "201902_0_0_0\t1\t\n"
node2.query("INSERT INTO replicated_mt VALUES (toDate('2019-01-02'), 3, 10), (toDate('2019-01-02'), 4, 12)")
node1.query("SYSTEM SYNC REPLICA replicated_mt")
assert node1.query("SELECT count() from replicated_mt") == "4\n"
assert node2.query("SELECT count() from replicated_mt") == "4\n"
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "201901_0_0_0\t1\t\n"
assert node2.query("CHECK TABLE replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "201901_0_0_0\t1\t\n"
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901",
settings={"check_query_single_value_result": 0}) == "201901_0_0_0\t1\t\n"
assert node2.query("CHECK TABLE replicated_mt PARTITION 201901",
settings={"check_query_single_value_result": 0}) == "201901_0_0_0\t1\t\n"
def test_check_replicated_table_corruption(started_cluster):
@ -113,18 +128,25 @@ def test_check_replicated_table_corruption(started_cluster):
assert node1.query("SELECT count() from replicated_mt") == "4\n"
assert node2.query("SELECT count() from replicated_mt") == "4\n"
part_name = node1.query("SELECT name from system.parts where table = 'replicated_mt' and partition_id = '201901' and active = 1").strip()
part_name = node1.query(
"SELECT name from system.parts where table = 'replicated_mt' and partition_id = '201901' and active = 1").strip()
corrupt_data_part_on_disk(node1, "replicated_mt", part_name)
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "{p}\t0\tPart {p} looks broken. Removing it and queueing a fetch.\n".format(p=part_name)
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901", settings={
"check_query_single_value_result": 0}) == "{p}\t0\tPart {p} looks broken. Removing it and queueing a fetch.\n".format(
p=part_name)
node1.query("SYSTEM SYNC REPLICA replicated_mt")
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "{}\t1\t\n".format(part_name)
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901",
settings={"check_query_single_value_result": 0}) == "{}\t1\t\n".format(part_name)
assert node1.query("SELECT count() from replicated_mt") == "4\n"
remove_part_from_disk(node2, "replicated_mt", part_name)
assert node2.query("CHECK TABLE replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "{p}\t0\tPart {p} looks broken. Removing it and queueing a fetch.\n".format(p=part_name)
assert node2.query("CHECK TABLE replicated_mt PARTITION 201901", settings={
"check_query_single_value_result": 0}) == "{p}\t0\tPart {p} looks broken. Removing it and queueing a fetch.\n".format(
p=part_name)
node1.query("SYSTEM SYNC REPLICA replicated_mt")
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "{}\t1\t\n".format(part_name)
assert node1.query("CHECK TABLE replicated_mt PARTITION 201901",
settings={"check_query_single_value_result": 0}) == "{}\t1\t\n".format(part_name)
assert node1.query("SELECT count() from replicated_mt") == "4\n"

View File

@ -1,13 +1,13 @@
import time
import pytest
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.network import PartitionManager
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', with_zookeeper=True)
@pytest.fixture(scope="module")
def start_cluster():
try:
@ -16,6 +16,7 @@ def start_cluster():
finally:
cluster.shutdown()
# This tests if the data directory for a table is cleaned up if there is a Zookeeper
# connection exception during a CreateQuery operation involving ReplicatedMergeTree tables.
# Test flow is as follows:
@ -48,20 +49,30 @@ def test_cleanup_dir_after_bad_zk_conn(start_cluster):
node1.query('''INSERT INTO replica.test VALUES (1, now())''')
assert "1\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''')
def test_cleanup_dir_after_wrong_replica_name(start_cluster):
node1.query("CREATE TABLE test2_r1 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test2/', 'r1') ORDER BY n")
error = node1.query_and_get_error("CREATE TABLE test2_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test2/', 'r1') ORDER BY n")
node1.query(
"CREATE TABLE test2_r1 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test2/', 'r1') ORDER BY n")
error = node1.query_and_get_error(
"CREATE TABLE test2_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test2/', 'r1') ORDER BY n")
assert "already exists" in error
node1.query("CREATE TABLE test_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test2/', 'r2') ORDER BY n")
node1.query(
"CREATE TABLE test_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test2/', 'r2') ORDER BY n")
def test_cleanup_dir_after_wrong_zk_path(start_cluster):
node1.query("CREATE TABLE test3_r1 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test3/', 'r1') ORDER BY n")
error = node1.query_and_get_error("CREATE TABLE test3_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/', 'r2') ORDER BY n")
node1.query(
"CREATE TABLE test3_r1 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test3/', 'r1') ORDER BY n")
error = node1.query_and_get_error(
"CREATE TABLE test3_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/', 'r2') ORDER BY n")
assert "Cannot create" in error
node1.query("CREATE TABLE test3_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test3/', 'r2') ORDER BY n")
node1.query(
"CREATE TABLE test3_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test3/', 'r2') ORDER BY n")
def test_attach_without_zk(start_cluster):
node1.query("CREATE TABLE test4_r1 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test4/', 'r1') ORDER BY n")
node1.query(
"CREATE TABLE test4_r1 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test4/', 'r1') ORDER BY n")
node1.query("DETACH TABLE test4_r1")
with PartitionManager() as pm:
pm._add_rule({'probability': 0.5, 'source': node1.ip_address, 'destination_port': 2181, 'action': 'DROP'})

View File

@ -7,6 +7,7 @@ cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
@pytest.fixture(scope="module")
def start_cluster():
try:

View File

@ -1,15 +1,15 @@
import os
import random
import sys
import time
from contextlib import contextmanager
import docker
import kazoo
import pytest
import docker
import random
from contextlib import contextmanager
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__))
sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR))
@ -18,9 +18,10 @@ MOVING_FAIL_PROBABILITY = 0.2
cluster = ClickHouseCluster(__file__)
def check_all_hosts_sucesfully_executed(tsv_content, num_hosts):
M = TSV.toMat(tsv_content)
hosts = [(l[0], l[1]) for l in M] # (host, port)
hosts = [(l[0], l[1]) for l in M] # (host, port)
codes = [l[2] for l in M]
assert len(hosts) == num_hosts and len(set(hosts)) == num_hosts, "\n" + tsv_content
@ -39,14 +40,14 @@ def started_cluster():
global cluster
try:
clusters_schema = {
"0" : {
"0" : ["0", "1"],
"1" : ["0"]
},
"1" : {
"0" : ["0", "1"],
"1" : ["0"]
}
"0": {
"0": ["0", "1"],
"1": ["0"]
},
"1": {
"0": ["0", "1"],
"1": ["0"]
}
}
for cluster_name, shards in clusters_schema.iteritems():
@ -54,10 +55,11 @@ def started_cluster():
for replica_name in replicas:
name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name)
cluster.add_instance(name,
main_configs=["configs/conf.d/query_log.xml", "configs/conf.d/ddl.xml", "configs/conf.d/clusters.xml"],
user_configs=["configs/users.xml"],
macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name},
with_zookeeper=True)
main_configs=["configs/conf.d/query_log.xml", "configs/conf.d/ddl.xml",
"configs/conf.d/clusters.xml"],
user_configs=["configs/users.xml"],
macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name},
with_zookeeper=True)
cluster.start()
yield cluster
@ -70,24 +72,27 @@ class Task1:
def __init__(self, cluster):
self.cluster = cluster
self.zk_task_path="/clickhouse-copier/task_simple"
self.zk_task_path = "/clickhouse-copier/task_simple"
self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task0_description.xml'), 'r').read()
def start(self):
instance = cluster.instances['s0_0_0']
for cluster_num in ["0", "1"]:
ddl_check_query(instance, "DROP DATABASE IF EXISTS default ON CLUSTER cluster{}".format(cluster_num))
ddl_check_query(instance, "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{} ENGINE=Ordinary".format(cluster_num))
ddl_check_query(instance,
"CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{} ENGINE=Ordinary".format(
cluster_num))
ddl_check_query(instance, "CREATE TABLE hits ON CLUSTER cluster0 (d UInt64, d1 UInt64 MATERIALIZED d+1) " +
"ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/hits', '{replica}') " +
"PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16")
ddl_check_query(instance, "CREATE TABLE hits_all ON CLUSTER cluster0 (d UInt64) ENGINE=Distributed(cluster0, default, hits, d)")
ddl_check_query(instance, "CREATE TABLE hits_all ON CLUSTER cluster1 (d UInt64) ENGINE=Distributed(cluster1, default, hits, d + 1)")
instance.query("INSERT INTO hits_all SELECT * FROM system.numbers LIMIT 1002", settings={"insert_distributed_sync": 1})
"ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/hits', '{replica}') " +
"PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16")
ddl_check_query(instance,
"CREATE TABLE hits_all ON CLUSTER cluster0 (d UInt64) ENGINE=Distributed(cluster0, default, hits, d)")
ddl_check_query(instance,
"CREATE TABLE hits_all ON CLUSTER cluster1 (d UInt64) ENGINE=Distributed(cluster1, default, hits, d + 1)")
instance.query("INSERT INTO hits_all SELECT * FROM system.numbers LIMIT 1002",
settings={"insert_distributed_sync": 1})
def check(self):
assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM hits_all")) == TSV("1002\n")
@ -107,31 +112,44 @@ class Task2:
def __init__(self, cluster):
self.cluster = cluster
self.zk_task_path="/clickhouse-copier/task_month_to_week_partition"
self.zk_task_path = "/clickhouse-copier/task_month_to_week_partition"
self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task_month_to_week_description.xml'), 'r').read()
def start(self):
instance = cluster.instances['s0_0_0']
for cluster_num in ["0", "1"]:
ddl_check_query(instance, "DROP DATABASE IF EXISTS default ON CLUSTER cluster{}".format(cluster_num))
ddl_check_query(instance, "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{} ENGINE=Ordinary".format(cluster_num))
ddl_check_query(instance,
"CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{} ENGINE=Ordinary".format(
cluster_num))
ddl_check_query(instance, "CREATE TABLE a ON CLUSTER cluster0 (date Date, d UInt64, d1 UInt64 ALIAS d+1) ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/a', '{replica}', date, intHash64(d), (date, intHash64(d)), 8192)")
ddl_check_query(instance, "CREATE TABLE a_all ON CLUSTER cluster0 (date Date, d UInt64) ENGINE=Distributed(cluster0, default, a, d)")
ddl_check_query(instance,
"CREATE TABLE a ON CLUSTER cluster0 (date Date, d UInt64, d1 UInt64 ALIAS d+1) ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/a', '{replica}', date, intHash64(d), (date, intHash64(d)), 8192)")
ddl_check_query(instance,
"CREATE TABLE a_all ON CLUSTER cluster0 (date Date, d UInt64) ENGINE=Distributed(cluster0, default, a, d)")
instance.query("INSERT INTO a_all SELECT toDate(17581 + number) AS date, number AS d FROM system.numbers LIMIT 85", settings={"insert_distributed_sync": 1})
instance.query(
"INSERT INTO a_all SELECT toDate(17581 + number) AS date, number AS d FROM system.numbers LIMIT 85",
settings={"insert_distributed_sync": 1})
def check(self):
assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM cluster(cluster0, default, a)")) == TSV("85\n")
assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count(), uniqExact(date) FROM cluster(cluster1, default, b)")) == TSV("85\t85\n")
assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM cluster(cluster0, default, a)")) == TSV(
"85\n")
assert TSV(self.cluster.instances['s1_0_0'].query(
"SELECT count(), uniqExact(date) FROM cluster(cluster1, default, b)")) == TSV("85\t85\n")
assert TSV(self.cluster.instances['s1_0_0'].query("SELECT DISTINCT jumpConsistentHash(intHash64(d), 2) FROM b")) == TSV("0\n")
assert TSV(self.cluster.instances['s1_1_0'].query("SELECT DISTINCT jumpConsistentHash(intHash64(d), 2) FROM b")) == TSV("1\n")
assert TSV(self.cluster.instances['s1_0_0'].query(
"SELECT DISTINCT jumpConsistentHash(intHash64(d), 2) FROM b")) == TSV("0\n")
assert TSV(self.cluster.instances['s1_1_0'].query(
"SELECT DISTINCT jumpConsistentHash(intHash64(d), 2) FROM b")) == TSV("1\n")
assert TSV(self.cluster.instances['s1_0_0'].query("SELECT uniqExact(partition) IN (12, 13) FROM system.parts WHERE active AND database='default' AND table='b'")) == TSV("1\n")
assert TSV(self.cluster.instances['s1_1_0'].query("SELECT uniqExact(partition) IN (12, 13) FROM system.parts WHERE active AND database='default' AND table='b'")) == TSV("1\n")
assert TSV(self.cluster.instances['s1_0_0'].query(
"SELECT uniqExact(partition) IN (12, 13) FROM system.parts WHERE active AND database='default' AND table='b'")) == TSV(
"1\n")
assert TSV(self.cluster.instances['s1_1_0'].query(
"SELECT uniqExact(partition) IN (12, 13) FROM system.parts WHERE active AND database='default' AND table='b'")) == TSV(
"1\n")
instance = cluster.instances['s0_0_0']
ddl_check_query(instance, "DROP TABLE a ON CLUSTER cluster0")
@ -142,11 +160,10 @@ class Task_test_block_size:
def __init__(self, cluster):
self.cluster = cluster
self.zk_task_path="/clickhouse-copier/task_test_block_size"
self.zk_task_path = "/clickhouse-copier/task_test_block_size"
self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task_test_block_size.xml'), 'r').read()
self.rows = 1000000
def start(self):
instance = cluster.instances['s0_0_0']
@ -155,11 +172,13 @@ class Task_test_block_size:
ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/test_block_size', '{replica}')
ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d)""", 2)
instance.query("INSERT INTO test_block_size SELECT toDate(0) AS partition, number as d FROM system.numbers LIMIT {}".format(self.rows))
instance.query(
"INSERT INTO test_block_size SELECT toDate(0) AS partition, number as d FROM system.numbers LIMIT {}".format(
self.rows))
def check(self):
assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count() FROM cluster(cluster1, default, test_block_size)")) == TSV("{}\n".format(self.rows))
assert TSV(self.cluster.instances['s1_0_0'].query(
"SELECT count() FROM cluster(cluster1, default, test_block_size)")) == TSV("{}\n".format(self.rows))
instance = cluster.instances['s0_0_0']
ddl_check_query(instance, "DROP TABLE test_block_size ON CLUSTER shard_0_0", 2)
@ -170,17 +189,15 @@ class Task_no_index:
def __init__(self, cluster):
self.cluster = cluster
self.zk_task_path="/clickhouse-copier/task_no_index"
self.zk_task_path = "/clickhouse-copier/task_no_index"
self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task_no_index.xml'), 'r').read()
self.rows = 1000000
def start(self):
instance = cluster.instances['s0_0_0']
instance.query("create table ontime (Year UInt16, FlightDate String) ENGINE = Memory")
instance.query("insert into ontime values (2016, 'test6'), (2017, 'test7'), (2018, 'test8')")
def check(self):
assert TSV(self.cluster.instances['s1_1_0'].query("SELECT Year FROM ontime22")) == TSV("2017\n")
instance = cluster.instances['s0_0_0']
@ -193,17 +210,16 @@ class Task_no_arg:
def __init__(self, cluster):
self.cluster = cluster
self.zk_task_path="/clickhouse-copier/task_no_arg"
self.zk_task_path = "/clickhouse-copier/task_no_arg"
self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task_no_arg.xml'), 'r').read()
self.rows = 1000000
def start(self):
instance = cluster.instances['s0_0_0']
instance.query("create table copier_test1 (date Date, id UInt32) engine = MergeTree PARTITION BY date ORDER BY date SETTINGS index_granularity = 8192")
instance.query(
"create table copier_test1 (date Date, id UInt32) engine = MergeTree PARTITION BY date ORDER BY date SETTINGS index_granularity = 8192")
instance.query("insert into copier_test1 values ('2016-01-01', 10);")
def check(self):
assert TSV(self.cluster.instances['s1_1_0'].query("SELECT date FROM copier_test1_1")) == TSV("2016-01-01\n")
instance = cluster.instances['s0_0_0']
@ -227,15 +243,14 @@ def execute_task(task, cmd_options):
zk.ensure_path(zk_task_path)
zk.create(zk_task_path + "/description", task.copier_task_config)
# Run cluster-copier processes on each node
docker_api = docker.from_env().api
copiers_exec_ids = []
cmd = ['/usr/bin/clickhouse', 'copier',
'--config', '/etc/clickhouse-server/config-copier.xml',
'--task-path', zk_task_path,
'--base-dir', '/var/log/clickhouse-server/copier']
'--config', '/etc/clickhouse-server/config-copier.xml',
'--task-path', zk_task_path,
'--base-dir', '/var/log/clickhouse-server/copier']
cmd += cmd_options
copiers = random.sample(cluster.instances.keys(), 3)
@ -243,7 +258,8 @@ def execute_task(task, cmd_options):
for instance_name in copiers:
instance = cluster.instances[instance_name]
container = instance.get_docker_handle()
instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs/config-copier.xml"), "/etc/clickhouse-server/config-copier.xml")
instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs/config-copier.xml"),
"/etc/clickhouse-server/config-copier.xml")
print "Copied copier config to {}".format(instance.name)
exec_id = docker_api.exec_create(container.id, cmd, stderr=True)
output = docker_api.exec_start(exec_id).decode('utf8')
@ -277,7 +293,6 @@ def execute_task(task, cmd_options):
True
]
)
def test_copy_simple(started_cluster, use_sample_offset):
if use_sample_offset:
execute_task(Task1(started_cluster), ['--experimental-use-sample-offset', '1'])
@ -292,7 +307,6 @@ def test_copy_simple(started_cluster, use_sample_offset):
True
]
)
def test_copy_with_recovering(started_cluster, use_sample_offset):
if use_sample_offset:
execute_task(Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY),
@ -300,6 +314,7 @@ def test_copy_with_recovering(started_cluster, use_sample_offset):
else:
execute_task(Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)])
@pytest.mark.parametrize(
('use_sample_offset'),
[
@ -307,7 +322,6 @@ def test_copy_with_recovering(started_cluster, use_sample_offset):
True
]
)
def test_copy_with_recovering_after_move_faults(started_cluster, use_sample_offset):
if use_sample_offset:
execute_task(Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY),
@ -315,29 +329,36 @@ def test_copy_with_recovering_after_move_faults(started_cluster, use_sample_offs
else:
execute_task(Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)])
@pytest.mark.timeout(600)
def test_copy_month_to_week_partition(started_cluster):
execute_task(Task2(started_cluster), [])
@pytest.mark.timeout(600)
def test_copy_month_to_week_partition_with_recovering(started_cluster):
execute_task(Task2(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)])
@pytest.mark.timeout(600)
def test_copy_month_to_week_partition_with_recovering_after_move_faults(started_cluster):
execute_task(Task2(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)])
def test_block_size(started_cluster):
execute_task(Task_test_block_size(started_cluster), [])
def test_no_index(started_cluster):
execute_task(Task_no_index(started_cluster), [])
def test_no_arg(started_cluster):
execute_task(Task_no_arg(started_cluster), [])
if __name__ == '__main__':
with contextmanager(started_cluster)() as cluster:
for name, instance in cluster.instances.items():
print name, instance.ip_address
raw_input("Cluster created, press any key to destroy...")
for name, instance in cluster.instances.items():
print name, instance.ip_address
raw_input("Cluster created, press any key to destroy...")

View File

@ -1,13 +1,10 @@
import os
import os.path as p
import sys
import time
import datetime
import pytest
from contextlib import contextmanager
import docker
from kazoo.client import KazooClient
import docker
import pytest
CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__))
sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR))
@ -18,13 +15,14 @@ COPYING_FAIL_PROBABILITY = 0.33
MOVING_FAIL_PROBABILITY = 0.1
cluster = None
@pytest.fixture(scope="function")
def started_cluster():
global cluster
try:
clusters_schema = {
"0" : {"0" : ["0"]},
"1" : {"0" : ["0"]}
"0": {"0": ["0"]},
"1": {"0": ["0"]}
}
cluster = ClickHouseCluster(__file__)
@ -50,12 +48,11 @@ class TaskTrivial:
def __init__(self, cluster, use_sample_offset):
self.cluster = cluster
if use_sample_offset:
self.zk_task_path="/clickhouse-copier/task_trivial_use_sample_offset"
self.zk_task_path = "/clickhouse-copier/task_trivial_use_sample_offset"
else:
self.zk_task_path="/clickhouse-copier/task_trivial"
self.zk_task_path = "/clickhouse-copier/task_trivial"
self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task_trivial.xml'), 'r').read()
def start(self):
source = cluster.instances['s0_0_0']
destination = cluster.instances['s1_0_0']
@ -68,8 +65,8 @@ class TaskTrivial:
"ENGINE=ReplicatedMergeTree('/clickhouse/tables/source_trivial_cluster/1/trivial', '1') "
"PARTITION BY d % 5 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16")
source.query("INSERT INTO trivial SELECT * FROM system.numbers LIMIT 1002", settings={"insert_distributed_sync": 1})
source.query("INSERT INTO trivial SELECT * FROM system.numbers LIMIT 1002",
settings={"insert_distributed_sync": 1})
def check(self):
source = cluster.instances['s0_0_0']
@ -138,7 +135,6 @@ def execute_task(task, cmd_options):
True
]
)
def test_trivial_copy(started_cluster, use_sample_offset):
if use_sample_offset:
execute_task(TaskTrivial(started_cluster, use_sample_offset), ['--experimental-use-sample-offset', '1'])
@ -146,6 +142,7 @@ def test_trivial_copy(started_cluster, use_sample_offset):
print("AAAAA")
execute_task(TaskTrivial(started_cluster, use_sample_offset), [])
@pytest.mark.parametrize(
('use_sample_offset'),
[
@ -153,7 +150,6 @@ def test_trivial_copy(started_cluster, use_sample_offset):
True
]
)
def test_trivial_copy_with_copy_fault(started_cluster, use_sample_offset):
if use_sample_offset:
execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY),
@ -161,6 +157,7 @@ def test_trivial_copy_with_copy_fault(started_cluster, use_sample_offset):
else:
execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)])
@pytest.mark.parametrize(
('use_sample_offset'),
[
@ -168,7 +165,6 @@ def test_trivial_copy_with_copy_fault(started_cluster, use_sample_offset):
True
]
)
def test_trivial_copy_with_move_fault(started_cluster, use_sample_offset):
if use_sample_offset:
execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY),

View File

@ -1,8 +1,7 @@
import time
from multiprocessing.dummy import Pool
import pytest
from multiprocessing.dummy import Pool
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
@ -10,6 +9,7 @@ cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', user_configs=['configs/user_restrictions.xml'])
node2 = cluster.add_instance('node2', user_configs=['configs/user_restrictions.xml'])
@pytest.fixture(scope="module")
def started_cluster():
try:
@ -21,6 +21,7 @@ def started_cluster():
finally:
cluster.shutdown()
def test_exception_message(started_cluster):
assert node1.query("select number from nums order by number") == "0\n1\n"
@ -30,7 +31,7 @@ def test_exception_message(started_cluster):
busy_pool = Pool(3)
busy_pool.map_async(node_busy, xrange(3))
time.sleep(1) # wait a little until polling starts
time.sleep(1) # wait a little until polling starts
try:
assert node2.query("select number from remote('node1', 'default', 'nums')", user='good') == "0\n1\n"
except Exception as ex:

View File

@ -1,12 +1,9 @@
import time
import pytest
import helpers.client as client
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/fast_background_pool.xml'], with_zookeeper=True)
node2 = cluster.add_instance('node2', main_configs=['configs/fast_background_pool.xml'], with_zookeeper=True)
@ -24,14 +21,16 @@ def started_cluster():
def count_ttl_merges_in_queue(node, table):
result = node.query("SELECT count() FROM system.replication_queue WHERE merge_type = 'TTL_DELETE' and table = '{}'".format(table))
result = node.query(
"SELECT count() FROM system.replication_queue WHERE merge_type = 'TTL_DELETE' and table = '{}'".format(table))
if not result:
return 0
return int(result.strip())
def count_ttl_merges_in_background_pool(node, table):
result = node.query("SELECT count() FROM system.merges WHERE merge_type = 'TTL_DELETE' and table = '{}'".format(table))
result = node.query(
"SELECT count() FROM system.merges WHERE merge_type = 'TTL_DELETE' and table = '{}'".format(table))
if not result:
return 0
return int(result.strip())
@ -55,12 +54,14 @@ def count_running_mutations(node, table):
# but it revealed a bug when we assign different merges to the same part
# on the borders of partitions.
def test_no_ttl_merges_in_busy_pool(started_cluster):
node1.query("CREATE TABLE test_ttl (d DateTime, key UInt64, data UInt64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0, number_of_free_entries_in_pool_to_execute_mutation = 0")
node1.query(
"CREATE TABLE test_ttl (d DateTime, key UInt64, data UInt64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0, number_of_free_entries_in_pool_to_execute_mutation = 0")
node1.query("SYSTEM STOP TTL MERGES")
for i in range(1, 7):
node1.query("INSERT INTO test_ttl SELECT now() - INTERVAL 1 MONTH + number - 1, {}, number FROM numbers(5)".format(i))
node1.query(
"INSERT INTO test_ttl SELECT now() - INTERVAL 1 MONTH + number - 1, {}, number FROM numbers(5)".format(i))
node1.query("ALTER TABLE test_ttl UPDATE data = data + 1 WHERE sleepEachRow(1) = 0")
@ -71,16 +72,22 @@ def test_no_ttl_merges_in_busy_pool(started_cluster):
node1.query("SYSTEM START TTL MERGES")
rows_count = []
while count_running_mutations(node1, "test_ttl") == 6:
print "Mutations count after start TTL", count_running_mutations(node1, "test_ttl")
assert node1.query("SELECT count() FROM test_ttl") == "30\n"
rows_count.append(int(node1.query("SELECT count() FROM test_ttl").strip()))
time.sleep(0.5)
# at least several seconds we didn't run any TTL merges and rows count equal
# to the original value
assert sum([1 for count in rows_count if count == 30]) > 4
assert_eq_with_retry(node1, "SELECT COUNT() FROM test_ttl", "0")
def test_limited_ttl_merges_in_empty_pool(started_cluster):
node1.query("CREATE TABLE test_ttl_v2 (d DateTime, key UInt64, data UInt64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0")
node1.query(
"CREATE TABLE test_ttl_v2 (d DateTime, key UInt64, data UInt64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0")
node1.query("SYSTEM STOP TTL MERGES")
@ -102,7 +109,8 @@ def test_limited_ttl_merges_in_empty_pool(started_cluster):
def test_limited_ttl_merges_in_empty_pool_replicated(started_cluster):
node1.query("CREATE TABLE replicated_ttl (d DateTime, key UInt64, data UInt64) ENGINE = ReplicatedMergeTree('/test/t', '1') ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0")
node1.query(
"CREATE TABLE replicated_ttl (d DateTime, key UInt64, data UInt64) ENGINE = ReplicatedMergeTree('/test/t', '1') ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0")
node1.query("SYSTEM STOP TTL MERGES")
@ -129,14 +137,17 @@ def test_limited_ttl_merges_in_empty_pool_replicated(started_cluster):
def test_limited_ttl_merges_two_replicas(started_cluster):
# Actually this test quite fast and often we cannot catch any merges.
# To check for sure just add some sleeps in mergePartsToTemporaryPart
node1.query("CREATE TABLE replicated_ttl_2 (d DateTime, key UInt64, data UInt64) ENGINE = ReplicatedMergeTree('/test/t2', '1') ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0")
node2.query("CREATE TABLE replicated_ttl_2 (d DateTime, key UInt64, data UInt64) ENGINE = ReplicatedMergeTree('/test/t2', '2') ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0")
node1.query(
"CREATE TABLE replicated_ttl_2 (d DateTime, key UInt64, data UInt64) ENGINE = ReplicatedMergeTree('/test/t2', '1') ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0")
node2.query(
"CREATE TABLE replicated_ttl_2 (d DateTime, key UInt64, data UInt64) ENGINE = ReplicatedMergeTree('/test/t2', '2') ORDER BY tuple() PARTITION BY key TTL d + INTERVAL 1 MONTH SETTINGS merge_with_ttl_timeout = 0")
node1.query("SYSTEM STOP TTL MERGES")
node2.query("SYSTEM STOP TTL MERGES")
for i in range(100):
node1.query("INSERT INTO replicated_ttl_2 SELECT now() - INTERVAL 1 MONTH, {}, number FROM numbers(10000)".format(i))
node1.query(
"INSERT INTO replicated_ttl_2 SELECT now() - INTERVAL 1 MONTH, {}, number FROM numbers(10000)".format(i))
node2.query("SYSTEM SYNC REPLICA replicated_ttl_2", timeout=10)
assert node1.query("SELECT COUNT() FROM replicated_ttl_2") == "1000000\n"
@ -150,7 +161,8 @@ def test_limited_ttl_merges_two_replicas(started_cluster):
while True:
merges_with_ttl_count_node1.add(count_ttl_merges_in_background_pool(node1, "replicated_ttl_2"))
merges_with_ttl_count_node2.add(count_ttl_merges_in_background_pool(node2, "replicated_ttl_2"))
if node1.query("SELECT COUNT() FROM replicated_ttl_2") == "0\n" and node2.query("SELECT COUNT() FROM replicated_ttl_2") == "0\n":
if node1.query("SELECT COUNT() FROM replicated_ttl_2") == "0\n" and node2.query(
"SELECT COUNT() FROM replicated_ttl_2") == "0\n":
break
# Both replicas can assign merges with TTL. If one will perform better than

View File

@ -1,6 +1,6 @@
import os
import pytest
import pytest
from helpers.cluster import ClickHouseCluster
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
@ -9,6 +9,7 @@ cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', main_configs=["configs/config.d/bad.xml"])
caught_exception = ""
@pytest.fixture(scope="module")
def start_cluster():
global caught_exception
@ -17,6 +18,7 @@ def start_cluster():
except Exception as e:
caught_exception = str(e)
def test_work(start_cluster):
print(caught_exception)
assert caught_exception.find("Root element doesn't have the corresponding root element as the config file.") != -1

View File

@ -1,21 +1,26 @@
import time
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', user_configs=['configs/config_no_substs.xml']) # hardcoded value 33333
node2 = cluster.add_instance('node2', user_configs=['configs/config_env.xml'], env_variables={"MAX_QUERY_SIZE": "55555"})
node1 = cluster.add_instance('node1', user_configs=['configs/config_no_substs.xml']) # hardcoded value 33333
node2 = cluster.add_instance('node2', user_configs=['configs/config_env.xml'],
env_variables={"MAX_QUERY_SIZE": "55555"})
node3 = cluster.add_instance('node3', user_configs=['configs/config_zk.xml'], with_zookeeper=True)
node4 = cluster.add_instance('node4', user_configs=['configs/config_incl.xml'], main_configs=['configs/max_query_size.xml']) # include value 77777
node4 = cluster.add_instance('node4', user_configs=['configs/config_incl.xml'],
main_configs=['configs/max_query_size.xml']) # include value 77777
node5 = cluster.add_instance('node5', user_configs=['configs/config_allow_databases.xml'])
node6 = cluster.add_instance('node6', user_configs=['configs/config_include_from_env.xml'], env_variables={"INCLUDE_FROM_ENV": "/etc/clickhouse-server/config.d/max_query_size.xml"}, main_configs=['configs/max_query_size.xml'])
node6 = cluster.add_instance('node6', user_configs=['configs/config_include_from_env.xml'],
env_variables={"INCLUDE_FROM_ENV": "/etc/clickhouse-server/config.d/max_query_size.xml"},
main_configs=['configs/max_query_size.xml'])
@pytest.fixture(scope="module")
def start_cluster():
try:
def create_zk_roots(zk):
zk.create(path="/setting/max_query_size", value="77777", makepath=True)
cluster.add_zookeeper_startup_command(create_zk_roots)
cluster.start()
@ -23,25 +28,36 @@ def start_cluster():
finally:
cluster.shutdown()
def test_config(start_cluster):
assert node1.query("select value from system.settings where name = 'max_query_size'") == "33333\n"
assert node2.query("select value from system.settings where name = 'max_query_size'") == "55555\n"
assert node3.query("select value from system.settings where name = 'max_query_size'") == "77777\n"
assert node4.query("select value from system.settings where name = 'max_query_size'") == "99999\n"
assert node6.query("select value from system.settings where name = 'max_query_size'") == "99999\n"
assert node1.query("select value from system.settings where name = 'max_query_size'") == "33333\n"
assert node2.query("select value from system.settings where name = 'max_query_size'") == "55555\n"
assert node3.query("select value from system.settings where name = 'max_query_size'") == "77777\n"
assert node4.query("select value from system.settings where name = 'max_query_size'") == "99999\n"
assert node6.query("select value from system.settings where name = 'max_query_size'") == "99999\n"
def test_allow_databases(start_cluster):
node5.query("CREATE DATABASE db1")
node5.query("CREATE TABLE db1.test_table(date Date, k1 String, v1 Int32) ENGINE = MergeTree(date, (k1, date), 8192)")
node5.query(
"CREATE TABLE db1.test_table(date Date, k1 String, v1 Int32) ENGINE = MergeTree(date, (k1, date), 8192)")
node5.query("INSERT INTO db1.test_table VALUES('2000-01-01', 'test_key', 1)")
assert node5.query("SELECT name FROM system.databases WHERE name = 'db1'") == "db1\n"
assert node5.query("SELECT name FROM system.tables WHERE database = 'db1' AND name = 'test_table' ") == "test_table\n"
assert node5.query("SELECT name FROM system.columns WHERE database = 'db1' AND table = 'test_table'") == "date\nk1\nv1\n"
assert node5.query("SELECT name FROM system.parts WHERE database = 'db1' AND table = 'test_table'") == "20000101_20000101_1_1_0\n"
assert node5.query("SELECT name FROM system.parts_columns WHERE database = 'db1' AND table = 'test_table'") == "20000101_20000101_1_1_0\n20000101_20000101_1_1_0\n20000101_20000101_1_1_0\n"
assert node5.query(
"SELECT name FROM system.tables WHERE database = 'db1' AND name = 'test_table' ") == "test_table\n"
assert node5.query(
"SELECT name FROM system.columns WHERE database = 'db1' AND table = 'test_table'") == "date\nk1\nv1\n"
assert node5.query(
"SELECT name FROM system.parts WHERE database = 'db1' AND table = 'test_table'") == "20000101_20000101_1_1_0\n"
assert node5.query(
"SELECT name FROM system.parts_columns WHERE database = 'db1' AND table = 'test_table'") == "20000101_20000101_1_1_0\n20000101_20000101_1_1_0\n20000101_20000101_1_1_0\n"
assert node5.query("SELECT name FROM system.databases WHERE name = 'db1'", user="test_allow").strip() == ""
assert node5.query("SELECT name FROM system.tables WHERE database = 'db1' AND name = 'test_table'", user="test_allow").strip() == ""
assert node5.query("SELECT name FROM system.columns WHERE database = 'db1' AND table = 'test_table'", user="test_allow").strip() == ""
assert node5.query("SELECT name FROM system.parts WHERE database = 'db1' AND table = 'test_table'", user="test_allow").strip() == ""
assert node5.query("SELECT name FROM system.parts_columns WHERE database = 'db1' AND table = 'test_table'", user="test_allow").strip() == ""
assert node5.query("SELECT name FROM system.tables WHERE database = 'db1' AND name = 'test_table'",
user="test_allow").strip() == ""
assert node5.query("SELECT name FROM system.columns WHERE database = 'db1' AND table = 'test_table'",
user="test_allow").strip() == ""
assert node5.query("SELECT name FROM system.parts WHERE database = 'db1' AND table = 'test_table'",
user="test_allow").strip() == ""
assert node5.query("SELECT name FROM system.parts_columns WHERE database = 'db1' AND table = 'test_table'",
user="test_allow").strip() == ""

View File

@ -1,12 +1,8 @@
import os
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.network import PartitionManager
from helpers.test_tools import assert_eq_with_retry
CLICKHOUSE_DATABASE = 'test'
@ -46,13 +42,14 @@ def test_consistent_part_after_move_partition(start_cluster):
# insert into all replicas
for i in range(100):
node1.query('INSERT INTO `{database}`.src VALUES ({value} % 2, {value})'.format(database=CLICKHOUSE_DATABASE,
value=i))
value=i))
query_source = 'SELECT COUNT(*) FROM `{database}`.src'.format(database=CLICKHOUSE_DATABASE)
query_dest = 'SELECT COUNT(*) FROM `{database}`.dest'.format(database=CLICKHOUSE_DATABASE)
assert_eq_with_retry(node2, query_source, node1.query(query_source))
assert_eq_with_retry(node2, query_dest, node1.query(query_dest))
node1.query('ALTER TABLE `{database}`.src MOVE PARTITION 1 TO TABLE `{database}`.dest'.format(database=CLICKHOUSE_DATABASE))
node1.query(
'ALTER TABLE `{database}`.src MOVE PARTITION 1 TO TABLE `{database}`.dest'.format(database=CLICKHOUSE_DATABASE))
assert_eq_with_retry(node2, query_source, node1.query(query_source))
assert_eq_with_retry(node2, query_dest, node1.query(query_dest))

View File

@ -8,19 +8,20 @@ from helpers.test_tools import assert_eq_with_retry
def fill_nodes(nodes, shard):
for node in nodes:
node.query(
'''
CREATE DATABASE test;
CREATE TABLE test_table(date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}')
ORDER BY id PARTITION BY toYYYYMM(date)
SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0;
'''.format(shard=shard, replica=node.name))
'''
CREATE DATABASE test;
CREATE TABLE test_table(date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}')
ORDER BY id PARTITION BY toYYYYMM(date)
SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0;
'''.format(shard=shard, replica=node.name))
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
@pytest.fixture(scope="module")
def start_cluster():
try:
@ -56,5 +57,3 @@ def test_inconsistent_parts_if_drop_while_replica_not_active(start_cluster):
# the first replica will be cloned from the second
pm.heal_all()
assert_eq_with_retry(node1, "SELECT count(*) FROM test_table", node2.query("SELECT count(*) FROM test_table"))

View File

@ -2,18 +2,17 @@ import time
from contextlib import contextmanager
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.network import PartitionManager
from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
node3 = cluster.add_instance('node3', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
@pytest.fixture(scope="module")
def started_cluster():
try:

View File

@ -2,7 +2,8 @@ import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', main_configs=["configs/config.d/text_log.xml"], user_configs=["configs/users.d/custom_settings.xml"])
node = cluster.add_instance('node', main_configs=["configs/config.d/text_log.xml"],
user_configs=["configs/users.d/custom_settings.xml"])
@pytest.fixture(scope="module", autouse=True)
@ -22,9 +23,10 @@ def test():
assert node.query("SELECT getSetting('custom_d')") == "some text\n"
assert "custom_a = -5, custom_b = 10000000000, custom_c = -4.325, custom_d = \\'some text\\'" \
in node.query("SHOW CREATE SETTINGS PROFILE default")
in node.query("SHOW CREATE SETTINGS PROFILE default")
assert "no settings profile" in node.query_and_get_error("SHOW CREATE SETTINGS PROFILE profile_with_unknown_setting")
assert "no settings profile" in node.query_and_get_error(
"SHOW CREATE SETTINGS PROFILE profile_with_unknown_setting")
assert "no settings profile" in node.query_and_get_error("SHOW CREATE SETTINGS PROFILE profile_illformed_setting")
@ -33,9 +35,9 @@ def test_invalid_settings():
node.query("SYSTEM FLUSH LOGS")
assert node.query("SELECT COUNT() FROM system.text_log WHERE"
" message LIKE '%Could not parse profile `profile_illformed_setting`%'"
" AND message LIKE '%Couldn\\'t restore Field from dump%'") == "1\n"
" message LIKE '%Could not parse profile `profile_illformed_setting`%'"
" AND message LIKE '%Couldn\\'t restore Field from dump%'") == "1\n"
assert node.query("SELECT COUNT() FROM system.text_log WHERE"
" message LIKE '%Could not parse profile `profile_with_unknown_setting`%'"
" AND message LIKE '%Setting x is neither a builtin setting nor started with the prefix \\'custom_\\'%'") == "1\n"
" message LIKE '%Could not parse profile `profile_with_unknown_setting`%'"
" AND message LIKE '%Setting x is neither a builtin setting nor started with the prefix \\'custom_\\'%'") == "1\n"

View File

@ -10,7 +10,6 @@ node3 = cluster.add_instance('node3', main_configs=['configs/remote_servers.xml'
node4 = cluster.add_instance('node4', main_configs=['configs/remote_servers.xml'], with_zookeeper=True)
@pytest.fixture(scope="module")
def started_cluster():
try:
@ -18,10 +17,14 @@ def started_cluster():
for i, node in enumerate([node1, node2]):
node.query("CREATE DATABASE testdb")
node.query('''CREATE TABLE testdb.test_table(id UInt32, val String) ENGINE = ReplicatedMergeTree('/clickhouse/test/test_table1', '{}') ORDER BY id;'''.format(i))
node.query(
'''CREATE TABLE testdb.test_table(id UInt32, val String) ENGINE = ReplicatedMergeTree('/clickhouse/test/test_table1', '{}') ORDER BY id;'''.format(
i))
for i, node in enumerate([node3, node4]):
node.query("CREATE DATABASE testdb")
node.query('''CREATE TABLE testdb.test_table(id UInt32, val String) ENGINE = ReplicatedMergeTree('/clickhouse/test/test_table2', '{}') ORDER BY id;'''.format(i))
node.query(
'''CREATE TABLE testdb.test_table(id UInt32, val String) ENGINE = ReplicatedMergeTree('/clickhouse/test/test_table2', '{}') ORDER BY id;'''.format(
i))
yield cluster
finally:
@ -34,7 +37,8 @@ def test_alter(started_cluster):
node2.query("SYSTEM SYNC REPLICA testdb.test_table")
node4.query("SYSTEM SYNC REPLICA testdb.test_table")
node1.query("ALTER TABLE testdb.test_table ON CLUSTER test_cluster ADD COLUMN somecolumn UInt8 AFTER val", settings={"replication_alter_partitions_sync": "2"})
node1.query("ALTER TABLE testdb.test_table ON CLUSTER test_cluster ADD COLUMN somecolumn UInt8 AFTER val",
settings={"replication_alter_partitions_sync": "2"})
node1.query("SYSTEM SYNC REPLICA testdb.test_table")
node2.query("SYSTEM SYNC REPLICA testdb.test_table")

View File

@ -1,7 +1,7 @@
import string
import random
import pytest
import string
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
@ -21,12 +21,14 @@ def start_cluster():
def get_compression_codec_byte(node, table_name, part_name):
cmd = "tail -c +17 /var/lib/clickhouse/data/default/{}/{}/data1.bin | od -x -N 1 | head -n 1 | awk '{{print $2}}'".format(table_name, part_name)
cmd = "tail -c +17 /var/lib/clickhouse/data/default/{}/{}/data1.bin | od -x -N 1 | head -n 1 | awk '{{print $2}}'".format(
table_name, part_name)
return node.exec_in_container(["bash", "-c", cmd]).strip()
def get_second_multiple_codec_byte(node, table_name, part_name):
cmd = "tail -c +17 /var/lib/clickhouse/data/default/{}/{}/data1.bin | od -x -j 11 -N 1 | head -n 1 | awk '{{print $2}}'".format(table_name, part_name)
cmd = "tail -c +17 /var/lib/clickhouse/data/default/{}/{}/data1.bin | od -x -j 11 -N 1 | head -n 1 | awk '{{print $2}}'".format(
table_name, part_name)
return node.exec_in_container(["bash", "-c", cmd]).strip()
@ -74,16 +76,22 @@ def test_default_codec_single(start_cluster):
# Same codec for all
assert get_compression_codec_byte(node1, "compression_table", "1_0_0_0") == CODECS_MAPPING['ZSTD']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_0_0_0'") == "ZSTD(10)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_0_0_0'") == "ZSTD(10)\n"
assert node1.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_0_0_0'") == "ZSTD(10)\n"
assert node2.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_0_0_0'") == "ZSTD(10)\n"
assert get_compression_codec_byte(node1, "compression_table", "2_0_0_0") == CODECS_MAPPING['ZSTD']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_0_0_0'") == "ZSTD(10)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_0_0_0'") == "ZSTD(10)\n"
assert node1.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_0_0_0'") == "ZSTD(10)\n"
assert node2.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_0_0_0'") == "ZSTD(10)\n"
assert get_compression_codec_byte(node1, "compression_table", "3_0_0_0") == CODECS_MAPPING['ZSTD']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_0_0_0'") == "ZSTD(10)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_0_0_0'") == "ZSTD(10)\n"
assert node1.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_0_0_0'") == "ZSTD(10)\n"
assert node2.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_0_0_0'") == "ZSTD(10)\n"
# just to be sure that replication works
node1.query("OPTIMIZE TABLE compression_table FINAL")
@ -101,16 +109,22 @@ def test_default_codec_single(start_cluster):
node2.query("SYSTEM FLUSH LOGS")
assert get_compression_codec_byte(node1, "compression_table", "1_0_0_1") == CODECS_MAPPING['ZSTD']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_0_0_1'") == "ZSTD(10)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_0_0_1'") == "ZSTD(10)\n"
assert node1.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_0_0_1'") == "ZSTD(10)\n"
assert node2.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_0_0_1'") == "ZSTD(10)\n"
assert get_compression_codec_byte(node1, "compression_table", "2_0_0_1") == CODECS_MAPPING['LZ4HC']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_0_0_1'") == "LZ4HC(5)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_0_0_1'") == "LZ4HC(5)\n"
assert node1.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_0_0_1'") == "LZ4HC(5)\n"
assert node2.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_0_0_1'") == "LZ4HC(5)\n"
assert get_compression_codec_byte(node1, "compression_table", "3_0_0_1") == CODECS_MAPPING['LZ4']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_0_0_1'") == "LZ4\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_0_0_1'") == "LZ4\n"
assert node1.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_0_0_1'") == "LZ4\n"
assert node2.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_0_0_1'") == "LZ4\n"
assert node1.query("SELECT COUNT() FROM compression_table") == "3\n"
assert node2.query("SELECT COUNT() FROM compression_table") == "3\n"
@ -137,18 +151,24 @@ def test_default_codec_multiple(start_cluster):
# Same codec for all
assert get_compression_codec_byte(node1, "compression_table_multiple", "1_0_0_0") == CODECS_MAPPING['Multiple']
assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "1_0_0_0") == CODECS_MAPPING['ZSTD']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '1_0_0_0'") == "ZSTD(10)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '1_0_0_0'") == "ZSTD(10)\n"
assert node1.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '1_0_0_0'") == "ZSTD(10)\n"
assert node2.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '1_0_0_0'") == "ZSTD(10)\n"
assert get_compression_codec_byte(node1, "compression_table_multiple", "2_0_0_0") == CODECS_MAPPING['Multiple']
assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "2_0_0_0") == CODECS_MAPPING['ZSTD']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '2_0_0_0'") == "ZSTD(10)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '2_0_0_0'") == "ZSTD(10)\n"
assert node1.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '2_0_0_0'") == "ZSTD(10)\n"
assert node2.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '2_0_0_0'") == "ZSTD(10)\n"
assert get_compression_codec_byte(node1, "compression_table_multiple", "3_0_0_0") == CODECS_MAPPING['Multiple']
assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "3_0_0_0") == CODECS_MAPPING['ZSTD']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '3_0_0_0'") == "ZSTD(10)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '3_0_0_0'") == "ZSTD(10)\n"
assert node1.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '3_0_0_0'") == "ZSTD(10)\n"
assert node2.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '3_0_0_0'") == "ZSTD(10)\n"
node2.query("SYSTEM SYNC REPLICA compression_table_multiple", timeout=15)
@ -156,18 +176,24 @@ def test_default_codec_multiple(start_cluster):
assert get_compression_codec_byte(node1, "compression_table_multiple", "1_0_0_1") == CODECS_MAPPING['Multiple']
assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "1_0_0_1") == CODECS_MAPPING['ZSTD']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '1_0_0_1'") == "ZSTD(10)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '1_0_0_1'") == "ZSTD(10)\n"
assert node1.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '1_0_0_1'") == "ZSTD(10)\n"
assert node2.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '1_0_0_1'") == "ZSTD(10)\n"
assert get_compression_codec_byte(node1, "compression_table_multiple", "2_0_0_1") == CODECS_MAPPING['Multiple']
assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "2_0_0_1") == CODECS_MAPPING['LZ4HC']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '2_0_0_1'") == "LZ4HC(5)\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '2_0_0_1'") == "LZ4HC(5)\n"
assert node1.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '2_0_0_1'") == "LZ4HC(5)\n"
assert node2.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '2_0_0_1'") == "LZ4HC(5)\n"
assert get_compression_codec_byte(node1, "compression_table_multiple", "3_0_0_1") == CODECS_MAPPING['Multiple']
assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "3_0_0_1") == CODECS_MAPPING['LZ4']
assert node1.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '3_0_0_1'") == "LZ4\n"
assert node2.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '3_0_0_1'") == "LZ4\n"
assert node1.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '3_0_0_1'") == "LZ4\n"
assert node2.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table_multiple' and name = '3_0_0_1'") == "LZ4\n"
assert node1.query("SELECT COUNT() FROM compression_table_multiple") == "3\n"
assert node2.query("SELECT COUNT() FROM compression_table_multiple") == "3\n"
@ -187,15 +213,21 @@ def test_default_codec_version_update(start_cluster):
node3.restart_with_latest_version()
assert node3.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_1_1_0'") == "ZSTD(1)\n"
assert node3.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_2_2_0'") == "ZSTD(1)\n"
assert node3.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_3_3_0'") == "ZSTD(1)\n"
assert node3.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_1_1_0'") == "ZSTD(1)\n"
assert node3.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_2_2_0'") == "ZSTD(1)\n"
assert node3.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_3_3_0'") == "ZSTD(1)\n"
node3.query("OPTIMIZE TABLE compression_table FINAL")
assert node3.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_1_1_1'") == "ZSTD(10)\n"
assert node3.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_2_2_1'") == "LZ4HC(5)\n"
assert node3.query("SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_3_3_1'") == "LZ4\n"
assert node3.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '1_1_1_1'") == "ZSTD(10)\n"
assert node3.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '2_2_2_1'") == "LZ4HC(5)\n"
assert node3.query(
"SELECT default_compression_codec FROM system.parts WHERE table = 'compression_table' and name = '3_3_3_1'") == "LZ4\n"
assert get_compression_codec_byte(node1, "compression_table_multiple", "2_0_0_1") == CODECS_MAPPING['Multiple']
assert get_second_multiple_codec_byte(node1, "compression_table_multiple", "2_0_0_1") == CODECS_MAPPING['LZ4HC']
assert get_compression_codec_byte(node1, "compression_table_multiple", "3_0_0_1") == CODECS_MAPPING['Multiple']

View File

@ -1,12 +1,20 @@
import time
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
ch1 = cluster.add_instance('ch1', main_configs=["configs/config.d/clusters.xml", "configs/config.d/distributed_ddl.xml"], with_zookeeper=True)
ch2 = cluster.add_instance('ch2', main_configs=["configs/config.d/clusters.xml", "configs/config.d/distributed_ddl.xml"], with_zookeeper=True)
ch3 = cluster.add_instance('ch3', main_configs=["configs/config.d/clusters.xml", "configs/config.d/distributed_ddl.xml"], with_zookeeper=True)
ch4 = cluster.add_instance('ch4', main_configs=["configs/config.d/clusters.xml", "configs/config.d/distributed_ddl.xml"], with_zookeeper=True)
ch1 = cluster.add_instance('ch1',
main_configs=["configs/config.d/clusters.xml", "configs/config.d/distributed_ddl.xml"],
with_zookeeper=True)
ch2 = cluster.add_instance('ch2',
main_configs=["configs/config.d/clusters.xml", "configs/config.d/distributed_ddl.xml"],
with_zookeeper=True)
ch3 = cluster.add_instance('ch3',
main_configs=["configs/config.d/clusters.xml", "configs/config.d/distributed_ddl.xml"],
with_zookeeper=True)
ch4 = cluster.add_instance('ch4',
main_configs=["configs/config.d/clusters.xml", "configs/config.d/distributed_ddl.xml"],
with_zookeeper=True)
@pytest.fixture(scope="module")
def started_cluster():
@ -20,13 +28,17 @@ def started_cluster():
def test_default_database_on_cluster(started_cluster):
ch1.query(database='test_default_database', sql="CREATE TABLE test_local_table ON CLUSTER 'cluster' (column UInt8) ENGINE = Memory;")
ch1.query(database='test_default_database',
sql="CREATE TABLE test_local_table ON CLUSTER 'cluster' (column UInt8) ENGINE = Memory;")
for node in [ch1, ch2, ch3, ch4]:
assert node.query("SHOW TABLES FROM test_default_database FORMAT TSV") == "test_local_table\n"
ch1.query(database='test_default_database', sql="CREATE TABLE test_distributed_table ON CLUSTER 'cluster' (column UInt8) ENGINE = Distributed(cluster, currentDatabase(), 'test_local_table');")
ch1.query(database='test_default_database',
sql="CREATE TABLE test_distributed_table ON CLUSTER 'cluster' (column UInt8) ENGINE = Distributed(cluster, currentDatabase(), 'test_local_table');")
for node in [ch1, ch2, ch3, ch4]:
assert node.query("SHOW TABLES FROM test_default_database FORMAT TSV") == "test_distributed_table\ntest_local_table\n"
assert node.query("SHOW CREATE TABLE test_default_database.test_distributed_table FORMAT TSV") == "CREATE TABLE test_default_database.test_distributed_table\\n(\\n `column` UInt8\\n)\\nENGINE = Distributed(\\'cluster\\', \\'test_default_database\\', \\'test_local_table\\')\n"
assert node.query(
"SHOW TABLES FROM test_default_database FORMAT TSV") == "test_distributed_table\ntest_local_table\n"
assert node.query(
"SHOW CREATE TABLE test_default_database.test_distributed_table FORMAT TSV") == "CREATE TABLE test_default_database.test_distributed_table\\n(\\n `column` UInt8\\n)\\nENGINE = Distributed(\\'cluster\\', \\'test_default_database\\', \\'test_local_table\\')\n"

View File

@ -1,7 +1,6 @@
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
import re
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance')
@ -11,7 +10,7 @@ instance = cluster.add_instance('instance')
def started_cluster():
try:
cluster.start()
instance.query("CREATE USER john")
instance.query("CREATE ROLE rx")
instance.query("CREATE ROLE ry")
@ -32,41 +31,41 @@ def test_set_default_roles():
assert instance.query("SHOW CURRENT ROLES", user="john") == ""
instance.query("GRANT rx, ry TO john")
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['rx', 0, 1], ['ry', 0, 1]] )
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV([['rx', 0, 1], ['ry', 0, 1]])
instance.query("SET DEFAULT ROLE NONE TO john")
assert instance.query("SHOW CURRENT ROLES", user="john") == ""
instance.query("SET DEFAULT ROLE rx TO john")
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['rx', 0, 1]] )
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV([['rx', 0, 1]])
instance.query("SET DEFAULT ROLE ry TO john")
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['ry', 0, 1]] )
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV([['ry', 0, 1]])
instance.query("SET DEFAULT ROLE ALL TO john")
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['rx', 0, 1], ['ry', 0, 1]] )
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV([['rx', 0, 1], ['ry', 0, 1]])
instance.query("SET DEFAULT ROLE ALL EXCEPT rx TO john")
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['ry', 0, 1]] )
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV([['ry', 0, 1]])
def test_alter_user():
assert instance.query("SHOW CURRENT ROLES", user="john") == ""
instance.query("GRANT rx, ry TO john")
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['rx', 0, 1], ['ry', 0, 1]] )
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV([['rx', 0, 1], ['ry', 0, 1]])
instance.query("ALTER USER john DEFAULT ROLE NONE")
assert instance.query("SHOW CURRENT ROLES", user="john") == ""
instance.query("ALTER USER john DEFAULT ROLE rx")
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['rx', 0, 1]] )
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV([['rx', 0, 1]])
instance.query("ALTER USER john DEFAULT ROLE ALL")
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['rx', 0, 1], ['ry', 0, 1]] )
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV([['rx', 0, 1], ['ry', 0, 1]])
instance.query("ALTER USER john DEFAULT ROLE ALL EXCEPT rx")
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV( [['ry', 0, 1]] )
assert instance.query("SHOW CURRENT ROLES", user="john") == TSV([['ry', 0, 1]])
def test_wrong_set_default_role():

View File

@ -1,14 +1,14 @@
import pytest
import os
import sys
import time
import os, sys
import pytest
sys.path.insert(0, os.path.dirname(os.path.dirname(os.path.abspath(__file__))))
import helpers
from helpers.cluster import ClickHouseCluster
from helpers.network import PartitionManager
cluster = ClickHouseCluster(__file__)
# Cluster with 2 shards of 2 replicas each. node_1_1 is the instance with Distributed table.
@ -19,6 +19,7 @@ node_1_2 = cluster.add_instance('node_1_2', with_zookeeper=True)
node_2_1 = cluster.add_instance('node_2_1', with_zookeeper=True)
node_2_2 = cluster.add_instance('node_2_2', with_zookeeper=True)
@pytest.fixture(scope="module")
def started_cluster():
try:
@ -30,7 +31,7 @@ def started_cluster():
node.query('''
CREATE TABLE replicated (d Date, x UInt32) ENGINE =
ReplicatedMergeTree('/clickhouse/tables/{shard}/replicated', '{instance}', d, d, 8192)'''
.format(shard=shard, instance=node.name))
.format(shard=shard, instance=node.name))
node_1_1.query(
"CREATE TABLE distributed (d Date, x UInt32) ENGINE = "
@ -51,7 +52,7 @@ def test(started_cluster):
node_1_2.query("INSERT INTO replicated VALUES ('2017-05-08', 1)")
node_2_2.query("INSERT INTO replicated VALUES ('2017-05-08', 2)")
time.sleep(1) # accrue replica delay
time.sleep(1) # accrue replica delay
assert node_1_1.query("SELECT sum(x) FROM replicated").strip() == '0'
assert node_1_2.query("SELECT sum(x) FROM replicated").strip() == '1'
@ -78,7 +79,7 @@ SELECT sum(x) FROM distributed WITH TOTALS SETTINGS
pm.drop_instance_zk_connections(node_1_2)
pm.drop_instance_zk_connections(node_2_2)
time.sleep(4) # allow pings to zookeeper to timeout (must be greater than ZK session timeout).
time.sleep(4) # allow pings to zookeeper to timeout (must be greater than ZK session timeout).
# At this point all replicas are stale, but the query must still go to second replicas which are the least stale ones.
assert instance_with_dist_table.query('''

View File

@ -9,7 +9,7 @@ instance = cluster.add_instance('instance')
def started_cluster():
try:
cluster.start()
instance.query("CREATE USER mira")
instance.query("CREATE TABLE test_table(x Int32, y Int32) ENGINE=Log")
instance.query("INSERT INTO test_table VALUES (5,6)")

View File

@ -1,11 +1,12 @@
import pytest
import math
import os
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout
from helpers.external_sources import SourceMySQL, SourceClickHouse, SourceFile, SourceExecutableCache, SourceExecutableHashed
from helpers.external_sources import SourceMongo, SourceMongoURI, SourceHTTP, SourceHTTPS, SourceRedis, SourceCassandra
import math
from helpers.external_sources import SourceMongo, SourceMongoURI, SourceHTTP, SourceHTTPS, SourceCassandra
from helpers.external_sources import SourceMySQL, SourceClickHouse, SourceFile, SourceExecutableCache, \
SourceExecutableHashed
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
dict_configs_path = os.path.join(SCRIPT_DIR, 'configs/dictionaries')
@ -103,8 +104,6 @@ VALUES = {
]
}
LAYOUTS = [
Layout("flat"),
Layout("hashed"),
@ -135,6 +134,7 @@ DICTIONARIES = []
cluster = None
node = None
def get_dict(source, layout, fields, suffix_name=''):
global dict_configs_path
@ -173,7 +173,8 @@ def setup_module(module):
for fname in os.listdir(dict_configs_path):
dictionaries.append(os.path.join(dict_configs_path, fname))
node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mysql=True, with_mongo=True, with_redis=True, with_cassandra=True)
node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mysql=True,
with_mongo=True, with_redis=True, with_cassandra=True)
@pytest.fixture(scope="module")
@ -195,7 +196,7 @@ def get_dictionaries(fold, total_folds, all_dicts):
chunk_len = int(math.ceil(len(all_dicts) / float(total_folds)))
if chunk_len * fold >= len(all_dicts):
return []
return all_dicts[fold * chunk_len : (fold + 1) * chunk_len]
return all_dicts[fold * chunk_len: (fold + 1) * chunk_len]
def remove_mysql_dicts():
@ -225,8 +226,8 @@ def remove_mysql_dicts():
TODO remove this when open ssl will be fixed or thread sanitizer will be suppressed
"""
#global DICTIONARIES
#DICTIONARIES = [d for d in DICTIONARIES if not d.name.startswith("MySQL")]
# global DICTIONARIES
# DICTIONARIES = [d for d in DICTIONARIES if not d.name.startswith("MySQL")]
@pytest.mark.parametrize("fold", list(range(10)))
@ -281,7 +282,6 @@ def test_simple_dictionaries(started_cluster, fold):
@pytest.mark.parametrize("fold", list(range(10)))
def test_complex_dictionaries(started_cluster, fold):
if node.is_built_with_thread_sanitizer():
remove_mysql_dicts()

View File

@ -1,30 +1,40 @@
import pytest
import os
import pytest
from helpers.cluster import ClickHouseCluster
@pytest.fixture(scope="function")
def cluster(request):
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
cluster = ClickHouseCluster(__file__)
try:
if request.param == "memory":
node = cluster.add_instance('node', main_configs=['configs/enable_dictionaries.xml', 'configs/dictionaries/complex_key_cache_string.xml'])
node = cluster.add_instance('node', main_configs=['configs/enable_dictionaries.xml',
'configs/dictionaries/complex_key_cache_string.xml'])
if request.param == "ssd":
node = cluster.add_instance('node', main_configs=['configs/enable_dictionaries.xml', 'configs/dictionaries/ssd_complex_key_cache_string.xml'])
node = cluster.add_instance('node', main_configs=['configs/enable_dictionaries.xml',
'configs/dictionaries/ssd_complex_key_cache_string.xml'])
cluster.start()
node.query("create table radars_table (radar_id String, radar_ip String, client_id String) engine=MergeTree() order by radar_id")
node.query(
"create table radars_table (radar_id String, radar_ip String, client_id String) engine=MergeTree() order by radar_id")
yield cluster
finally:
cluster.shutdown()
@pytest.mark.parametrize("cluster", ["memory", "ssd"], indirect=True)
def test_memory_consumption(cluster):
node = cluster.instances['node']
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))
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)")

View File

@ -1,16 +1,24 @@
import pytest
import os
from helpers.cluster import ClickHouseCluster
from helpers.client import QueryRuntimeException
import pymysql
import warnings
import time
import pymysql
import pytest
from helpers.client import QueryRuntimeException
from helpers.cluster import ClickHouseCluster
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', with_mysql=True, dictionaries=['configs/dictionaries/simple_dictionary.xml'], user_configs=['configs/user_admin.xml', 'configs/user_default.xml'])
node2 = cluster.add_instance('node2', with_mysql=True, dictionaries=['configs/dictionaries/simple_dictionary.xml'], main_configs=['configs/dictionaries/lazy_load.xml', 'configs/allow_remote_node.xml'], user_configs=['configs/user_admin.xml', 'configs/user_default.xml'])
node3 = cluster.add_instance('node3', main_configs=['configs/allow_remote_node.xml'], dictionaries=['configs/dictionaries/dictionary_with_conflict_name.xml', 'configs/dictionaries/conflict_name_dictionary.xml'], user_configs=['configs/user_admin.xml'])
node1 = cluster.add_instance('node1', with_mysql=True, dictionaries=['configs/dictionaries/simple_dictionary.xml'],
user_configs=['configs/user_admin.xml', 'configs/user_default.xml'])
node2 = cluster.add_instance('node2', with_mysql=True, dictionaries=['configs/dictionaries/simple_dictionary.xml'],
main_configs=['configs/dictionaries/lazy_load.xml', 'configs/allow_remote_node.xml'],
user_configs=['configs/user_admin.xml', 'configs/user_default.xml'])
node3 = cluster.add_instance('node3', main_configs=['configs/allow_remote_node.xml'],
dictionaries=['configs/dictionaries/dictionary_with_conflict_name.xml',
'configs/dictionaries/conflict_name_dictionary.xml'],
user_configs=['configs/user_admin.xml'])
node4 = cluster.add_instance('node4', user_configs=['configs/user_admin.xml', 'configs/config_password.xml'])
@ -21,6 +29,7 @@ def create_mysql_conn(user, password, hostname, port):
host=hostname,
port=port)
def execute_mysql_query(connection, query):
with warnings.catch_warnings():
warnings.simplefilter("ignore")
@ -35,15 +44,18 @@ def started_cluster():
cluster.start()
for clickhouse in [node1, node2, node3, node4]:
clickhouse.query("CREATE DATABASE test", user="admin")
clickhouse.query("CREATE TABLE test.xml_dictionary_table (id UInt64, SomeValue1 UInt8, SomeValue2 String) ENGINE = MergeTree() ORDER BY id", user="admin")
clickhouse.query("INSERT INTO test.xml_dictionary_table SELECT number, number % 23, hex(number) from numbers(1000)", user="admin")
clickhouse.query(
"CREATE TABLE test.xml_dictionary_table (id UInt64, SomeValue1 UInt8, SomeValue2 String) ENGINE = MergeTree() ORDER BY id",
user="admin")
clickhouse.query(
"INSERT INTO test.xml_dictionary_table SELECT number, number % 23, hex(number) from numbers(1000)",
user="admin")
yield cluster
finally:
cluster.shutdown()
@pytest.mark.parametrize("clickhouse,name,layout", [
(node1, 'complex_node1_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())'),
(node1, 'complex_node1_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'),
@ -53,7 +65,9 @@ def started_cluster():
def test_create_and_select_mysql(started_cluster, clickhouse, name, layout):
mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", 3308)
execute_mysql_query(mysql_conn, "CREATE DATABASE IF NOT EXISTS clickhouse")
execute_mysql_query(mysql_conn, "CREATE TABLE clickhouse.{} (key_field1 int, key_field2 bigint, value1 text, value2 float, PRIMARY KEY (key_field1, key_field2))".format(name))
execute_mysql_query(mysql_conn,
"CREATE TABLE clickhouse.{} (key_field1 int, key_field2 bigint, value1 text, value2 float, PRIMARY KEY (key_field1, key_field2))".format(
name))
values = []
for i in range(1000):
values.append('(' + ','.join([str(i), str(i * i), str(i) * 5, str(i * 3.14)]) + ')')
@ -80,12 +94,16 @@ def test_create_and_select_mysql(started_cluster, clickhouse, name, layout):
""".format(name, name, layout))
for i in range(172, 200):
assert clickhouse.query("SELECT dictGetString('default.{}', 'value1', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)) == str(i) * 5 + '\n'
stroka = clickhouse.query("SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)).strip()
assert clickhouse.query(
"SELECT dictGetString('default.{}', 'value1', tuple(toInt32({}), toInt64({})))".format(name, i,
i * i)) == str(
i) * 5 + '\n'
stroka = clickhouse.query(
"SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(name, i,
i * i)).strip()
value = float(stroka)
assert int(value) == int(i * 3.14)
for i in range(1000):
values.append('(' + ','.join([str(i), str(i * i), str(i) * 3, str(i * 2.718)]) + ')')
execute_mysql_query(mysql_conn, "REPLACE INTO clickhouse.{} VALUES ".format(name) + ','.join(values))
@ -93,8 +111,13 @@ def test_create_and_select_mysql(started_cluster, clickhouse, name, layout):
clickhouse.query("SYSTEM RELOAD DICTIONARY 'default.{}'".format(name))
for i in range(172, 200):
assert clickhouse.query("SELECT dictGetString('default.{}', 'value1', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)) == str(i) * 3 + '\n'
string = clickhouse.query("SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)).strip()
assert clickhouse.query(
"SELECT dictGetString('default.{}', 'value1', tuple(toInt32({}), toInt64({})))".format(name, i,
i * i)) == str(
i) * 3 + '\n'
string = clickhouse.query(
"SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(name, i,
i * i)).strip()
value = float(string)
assert int(value) == int(i * 2.718)
@ -182,6 +205,7 @@ def test_conflicting_name(started_cluster):
# old version still works
node3.query("select dictGetUInt8('test.conflicting_dictionary', 'SomeValue1', toUInt64(17))") == '17\n'
def test_http_dictionary_restrictions(started_cluster):
try:
node3.query("""
@ -198,6 +222,7 @@ def test_http_dictionary_restrictions(started_cluster):
except QueryRuntimeException as ex:
assert 'is not allowed in config.xml' in str(ex)
def test_file_dictionary_restrictions(started_cluster):
try:
node3.query("""
@ -218,7 +243,8 @@ def test_file_dictionary_restrictions(started_cluster):
def test_dictionary_with_where(started_cluster):
mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", 3308)
execute_mysql_query(mysql_conn, "CREATE DATABASE IF NOT EXISTS clickhouse")
execute_mysql_query(mysql_conn, "CREATE TABLE clickhouse.special_table (key_field1 int, value1 text, PRIMARY KEY (key_field1))")
execute_mysql_query(mysql_conn,
"CREATE TABLE clickhouse.special_table (key_field1 int, value1 text, PRIMARY KEY (key_field1))")
execute_mysql_query(mysql_conn, "INSERT INTO clickhouse.special_table VALUES (1, 'abcabc'), (2, 'qweqwe')")
node1.query("""
@ -243,6 +269,7 @@ def test_dictionary_with_where(started_cluster):
assert node1.query("SELECT dictGetString('default.special_dict', 'value1', toUInt64(2))") == 'qweqwe\n'
def test_clickhouse_remote(started_cluster):
with pytest.raises(QueryRuntimeException):
node3.query("""
@ -256,7 +283,9 @@ def test_clickhouse_remote(started_cluster):
SOURCE(CLICKHOUSE(HOST 'node4' PORT 9000 USER 'default' TABLE 'xml_dictionary_table' DB 'test'))
LIFETIME(MIN 1 MAX 10)
""")
node3.query("system reload dictionaries")
for i in range(5):
node3.query("system reload dictionary test.clickhouse_remote")
time.sleep(0.5)
node3.query("detach dictionary if exists test.clickhouse_remote")
node3.query("""
@ -272,4 +301,3 @@ def test_clickhouse_remote(started_cluster):
""")
node3.query("select dictGetUInt8('test.clickhouse_remote', 'SomeValue1', toUInt64(17))") == '17\n'

View File

@ -17,10 +17,10 @@ def start_cluster():
node.query("CREATE DATABASE IF NOT EXISTS ztest")
node.query("CREATE TABLE test.source(x UInt64, y UInt64) ENGINE=Log")
node.query("INSERT INTO test.source VALUES (5,6)")
node.query("CREATE DICTIONARY test.dict(x UInt64, y UInt64) PRIMARY KEY x "\
"SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'source' DB 'test')) "\
"LAYOUT(FLAT()) LIFETIME(0)")
node.query("CREATE DICTIONARY test.dict(x UInt64, y UInt64) PRIMARY KEY x " \
"SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'source' DB 'test')) " \
"LAYOUT(FLAT()) LIFETIME(0)")
yield cluster
finally:
@ -48,14 +48,14 @@ def cleanup_after_test():
def test_dependency_via_implicit_table(node):
d_names = ["test.adict", "test.zdict", "atest.dict", "ztest.dict"]
for d_name in d_names:
node.query("CREATE DICTIONARY {}(x UInt64, y UInt64) PRIMARY KEY x "\
"SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'dict' DB 'test')) "\
node.query("CREATE DICTIONARY {}(x UInt64, y UInt64) PRIMARY KEY x " \
"SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'dict' DB 'test')) " \
"LAYOUT(FLAT()) LIFETIME(0)".format(d_name))
def check():
for d_name in d_names:
assert node.query("SELECT dictGet({}, 'y', toUInt64(5))".format(d_name)) == "6\n"
check()
# Restart must not break anything.
@ -72,14 +72,14 @@ def test_dependency_via_explicit_table(node):
tbl_database, tbl_shortname = tbl_name.split('.')
d_name = d_names[i]
node.query("CREATE TABLE {}(x UInt64, y UInt64) ENGINE=Dictionary('test.dict')".format(tbl_name))
node.query("CREATE DICTIONARY {}(x UInt64, y UInt64) PRIMARY KEY x "\
"SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE '{}' DB '{}')) "\
node.query("CREATE DICTIONARY {}(x UInt64, y UInt64) PRIMARY KEY x " \
"SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE '{}' DB '{}')) " \
"LAYOUT(FLAT()) LIFETIME(0)".format(d_name, tbl_shortname, tbl_database))
def check():
for d_name in d_names:
assert node.query("SELECT dictGet({}, 'y', toUInt64(5))".format(d_name)) == "6\n"
check()
# Restart must not break anything.
@ -93,14 +93,14 @@ def test_dependency_via_dictionary_database(node):
d_names = ["test.adict", "test.zdict", "atest.dict", "ztest.dict"]
for d_name in d_names:
node.query("CREATE DICTIONARY {}(x UInt64, y UInt64) PRIMARY KEY x "\
"SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'test.dict' DB 'dict_db')) "\
node.query("CREATE DICTIONARY {}(x UInt64, y UInt64) PRIMARY KEY x " \
"SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'test.dict' DB 'dict_db')) " \
"LAYOUT(FLAT()) LIFETIME(0)".format(d_name))
def check():
for d_name in d_names:
assert node.query("SELECT dictGet({}, 'y', toUInt64(5))".format(d_name)) == "6\n"
check()
# Restart must not break anything.

View File

@ -1,13 +1,13 @@
import pytest
import os
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry
ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml']
DICTIONARY_FILES = ['configs/dictionaries/dep_x.xml', 'configs/dictionaries/dep_y.xml', 'configs/dictionaries/dep_z.xml']
DICTIONARY_FILES = ['configs/dictionaries/dep_x.xml', 'configs/dictionaries/dep_y.xml',
'configs/dictionaries/dep_z.xml']
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES,)
instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG + DICTIONARY_FILES, )
@pytest.fixture(scope="module")
@ -60,14 +60,14 @@ def test_get_data(started_cluster):
query("INSERT INTO test.elements VALUES (3, 'fire', 30, 8)")
# Wait for dictionaries to be reloaded.
assert_eq_with_retry(instance, "SELECT dictHas('dep_y', toUInt64(3))", "1", sleep_time = 2, retry_count = 10)
assert_eq_with_retry(instance, "SELECT dictHas('dep_y', toUInt64(3))", "1", sleep_time=2, retry_count=10)
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "XX\n"
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "fire\n"
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "ZZ\n"
# dep_x and dep_z are updated only when there `intDiv(count(), 4)` is changed.
query("INSERT INTO test.elements VALUES (4, 'ether', 404, 0.001)")
assert_eq_with_retry(instance, "SELECT dictHas('dep_x', toUInt64(4))", "1", sleep_time = 2, retry_count = 10)
assert_eq_with_retry(instance, "SELECT dictHas('dep_x', toUInt64(4))", "1", sleep_time=2, retry_count=10)
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "fire\n"
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "fire\n"
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "fire\n"

View File

@ -1,17 +1,13 @@
import pytest
import os
import time
## sudo -H pip install PyMySQL
import pymysql.cursors
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry
CONFIG_FILES = ['configs/dictionaries/mysql_dict1.xml', 'configs/dictionaries/mysql_dict2.xml', 'configs/remote_servers.xml']
CONFIG_FILES = ['configs/dictionaries/mysql_dict1.xml', 'configs/dictionaries/mysql_dict2.xml',
'configs/remote_servers.xml']
CONFIG_FILES += ['configs/enable_dictionaries.xml']
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance', main_configs=CONFIG_FILES, with_mysql = True)
instance = cluster.add_instance('instance', main_configs=CONFIG_FILES, with_mysql=True)
create_table_mysql_template = """
CREATE TABLE IF NOT EXISTS `test`.`{}` (
@ -25,10 +21,11 @@ create_clickhouse_dictionary_table_template = """
CREATE TABLE IF NOT EXISTS `test`.`dict_table_{}` (`id` UInt64, `value` String) ENGINE = Dictionary({})
"""
@pytest.fixture(scope="module")
def started_cluster():
try:
#time.sleep(30)
# time.sleep(30)
cluster.start()
# Create a MySQL database
@ -66,10 +63,12 @@ def test_load_mysql_dictionaries(started_cluster):
# Check number of row
assert query("SELECT count() FROM `test`.`dict_table_{}`".format('test' + str(n % 5))).rstrip() == '10000'
def create_mysql_db(mysql_connection, name):
with mysql_connection.cursor() as cursor:
cursor.execute("CREATE DATABASE IF NOT EXISTS {} DEFAULT CHARACTER SET 'utf8'".format(name))
def prepare_mysql_table(table_name, index):
mysql_connection = get_mysql_conn()
@ -78,17 +77,21 @@ def prepare_mysql_table(table_name, index):
# Insert rows using CH
query = instance.query
query("INSERT INTO `clickhouse_mysql`.{}(id, value) select number, concat('{} value ', toString(number)) from numbers(10000) ".format(table_name + str(index), table_name + str(index)))
query(
"INSERT INTO `clickhouse_mysql`.{}(id, value) select number, concat('{} value ', toString(number)) from numbers(10000) ".format(
table_name + str(index), table_name + str(index)))
assert query("SELECT count() FROM `clickhouse_mysql`.{}".format(table_name + str(index))).rstrip() == '10000'
mysql_connection.close()
#Create CH Dictionary tables based on MySQL tables
# Create CH Dictionary tables based on MySQL tables
query(create_clickhouse_dictionary_table_template.format(table_name + str(index), 'dict' + str(index)))
def get_mysql_conn():
conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.10', port=3308)
return conn
def create_mysql_table(conn, table_name):
with conn.cursor() as cursor:
cursor.execute(create_table_mysql_template.format(table_name))

View File

@ -1,13 +1,11 @@
import pytest
import os
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV, assert_eq_with_retry
ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml']
DICTIONARY_FILES = ['configs/dictionaries/cache.xml']
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES)
instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG + DICTIONARY_FILES)
@pytest.fixture(scope="module")
@ -42,4 +40,5 @@ def test_null_value(started_cluster):
# Check, that empty null_value interprets as default value
assert query("select dictGetUInt64('cache', 'UInt64_', toUInt64(12121212))") == "0\n"
assert query("select toTimeZone(dictGetDateTime('cache', 'DateTime_', toUInt64(12121212)), 'UTC')") == "1970-01-01 00:00:00\n"
assert query(
"select toTimeZone(dictGetDateTime('cache', 'DateTime_', toUInt64(12121212)), 'UTC')") == "1970-01-01 00:00:00\n"

View File

@ -1,7 +1,6 @@
import os
import pytest
import redis
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout
from helpers.external_sources import SourceRedis
@ -22,10 +21,10 @@ KEY_FIELDS = {
}
KEY_VALUES = {
"simple" : [
"simple": [
[1], [2]
],
"complex" : [
"complex": [
[1, 'world'], [2, 'qwerty2']
]
}
@ -76,6 +75,7 @@ LAYOUTS = [
DICTIONARIES = []
def get_dict(source, layout, fields, suffix_name=''):
global dict_configs_path
@ -99,8 +99,10 @@ def setup_module(module):
for i, field in enumerate(FIELDS):
DICTIONARIES.append([])
sources = []
sources.append(SourceRedis("RedisSimple", "localhost", "6380", "redis1", "6379", "", "clickhouse", i * 2, storage_type="simple"))
sources.append(SourceRedis("RedisHash", "localhost", "6380", "redis1", "6379", "", "clickhouse", i * 2 + 1, storage_type="hash_map"))
sources.append(SourceRedis("RedisSimple", "localhost", "6380", "redis1", "6379", "", "clickhouse", i * 2,
storage_type="simple"))
sources.append(SourceRedis("RedisHash", "localhost", "6380", "redis1", "6379", "", "clickhouse", i * 2 + 1,
storage_type="hash_map"))
for source in sources:
for layout in LAYOUTS:
if not source.compatible_with_layout(layout):
@ -118,6 +120,7 @@ def setup_module(module):
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_redis=True)
@pytest.fixture(scope="module", autouse=True)
def started_cluster():
try:
@ -134,6 +137,7 @@ def started_cluster():
finally:
cluster.shutdown()
@pytest.mark.parametrize("id", range(len(FIELDS)))
def test_redis_dictionaries(started_cluster, id):
print 'id:', id

View File

@ -1,6 +1,5 @@
import os
import glob
import difflib
import os
files = ['key_simple.tsv', 'key_complex_integers.tsv', 'key_complex_mixed.tsv']
@ -12,7 +11,6 @@ types = [
'Date', 'DateTime'
]
implicit_defaults = [
'1', '1', '1', '',
'-1', '-1', '-1', '-1',

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