mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
Merge remote-tracking branch 'refs/remotes/upstream/master' into vfs
This commit is contained in:
commit
e5ffdc0224
@ -677,7 +677,7 @@ fix comments to make obvious that it may throw.
|
||||
|
||||
### Backward Incompatible Change
|
||||
* Removed rarely used table function `catBoostPool` and storage `CatBoostPool`. If you have used this table function, please write email to `clickhouse-feedback@yandex-team.com`. Note that CatBoost integration remains and will be supported. [#6279](https://github.com/ClickHouse/ClickHouse/pull/6279) ([alexey-milovidov](https://github.com/alexey-milovidov))
|
||||
* Disable `ANY RIGHT JOIN` and `ANY FULL JOIN` by default. Set `any_join_get_any_from_right_table` setting to enable them. [#5126](https://github.com/ClickHouse/ClickHouse/issues/5126) [#6351](https://github.com/ClickHouse/ClickHouse/pull/6351) ([Artem Zuikov](https://github.com/4ertus2))
|
||||
* Disable `ANY RIGHT JOIN` and `ANY FULL JOIN` by default. Set `any_join_distinct_right_table_keys` setting to enable them. [#5126](https://github.com/ClickHouse/ClickHouse/issues/5126) [#6351](https://github.com/ClickHouse/ClickHouse/pull/6351) ([Artem Zuikov](https://github.com/4ertus2))
|
||||
|
||||
## ClickHouse release 19.13.6.51, 2019-10-02
|
||||
|
||||
|
@ -182,6 +182,9 @@ set(SRCS
|
||||
${HDFS3_SOURCE_DIR}/common/FileWrapper.h
|
||||
)
|
||||
|
||||
# old kernels (< 3.17) doens't have SYS_getrandom. Always use POSIX implementation to have better compatibility
|
||||
set_source_files_properties(${HDFS3_SOURCE_DIR}/rpc/RpcClient.cpp PROPERTIES COMPILE_FLAGS "-DBOOST_UUID_RANDOM_PROVIDER_FORCE_POSIX=1")
|
||||
|
||||
# target
|
||||
add_library(hdfs3 ${SRCS} ${PROTO_SOURCES} ${PROTO_HEADERS})
|
||||
|
||||
|
@ -97,8 +97,7 @@
|
||||
#define BRACK_PASTE_LAST '~'
|
||||
#define BRACK_PASTE_SLEN 6
|
||||
|
||||
/// Make sure we don't get ^J for the enter character.
|
||||
/// This handler also bypasses some unused macro/event checkings.
|
||||
/// This handler bypasses some unused macro/event checkings.
|
||||
static int clickhouse_rl_bracketed_paste_begin(int /* count */, int /* key */)
|
||||
{
|
||||
std::string buf;
|
||||
@ -106,10 +105,10 @@ static int clickhouse_rl_bracketed_paste_begin(int /* count */, int /* key */)
|
||||
|
||||
RL_SETSTATE(RL_STATE_MOREINPUT);
|
||||
SCOPE_EXIT(RL_UNSETSTATE(RL_STATE_MOREINPUT));
|
||||
char c;
|
||||
int c;
|
||||
while ((c = rl_read_key()) >= 0)
|
||||
{
|
||||
if (c == '\r' || c == '\n')
|
||||
if (c == '\r')
|
||||
c = '\n';
|
||||
buf.push_back(c);
|
||||
if (buf.size() >= BRACK_PASTE_SLEN && c == BRACK_PASTE_LAST && buf.substr(buf.size() - BRACK_PASTE_SLEN) == BRACK_PASTE_SUFF)
|
||||
|
@ -469,7 +469,6 @@ namespace ErrorCodes
|
||||
extern const int POCO_EXCEPTION = 1000;
|
||||
extern const int STD_EXCEPTION = 1001;
|
||||
extern const int UNKNOWN_EXCEPTION = 1002;
|
||||
extern const int METRIKA_OTHER_ERROR = 1003;
|
||||
|
||||
extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND = 2001;
|
||||
extern const int ILLEGAL_PROJECTION_MANIPULATOR = 2002;
|
||||
|
@ -17,7 +17,6 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int POCO_EXCEPTION;
|
||||
extern const int METRIKA_OTHER_ERROR;
|
||||
}
|
||||
|
||||
class Exception : public Poco::Exception
|
||||
|
@ -76,6 +76,7 @@ struct Settings : public SettingsCollection<Settings>
|
||||
M(SettingBool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.", 0) \
|
||||
M(SettingBool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \
|
||||
M(SettingUInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \
|
||||
M(SettingUInt64, background_move_pool_size, 8, "Number of threads performing background moves for tables. Only has meaning at server startup.", 0) \
|
||||
M(SettingUInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables. Only has meaning at server startup.", 0) \
|
||||
\
|
||||
M(SettingMilliseconds, distributed_directory_monitor_sleep_time_ms, 100, "Sleep time for StorageDistributed DirectoryMonitors, in case of any errors delay grows exponentially.", 0) \
|
||||
|
@ -203,8 +203,15 @@ UInt32 TTLBlockInputStream::getTimestampByIndex(const IColumn * column, size_t i
|
||||
return date_lut.fromDayNum(DayNum(column_date->getData()[ind]));
|
||||
else if (const ColumnUInt32 * column_date_time = typeid_cast<const ColumnUInt32 *>(column))
|
||||
return column_date_time->getData()[ind];
|
||||
else
|
||||
throw Exception("Unexpected type of result ttl column", ErrorCodes::LOGICAL_ERROR);
|
||||
else if (const ColumnConst * column_const = typeid_cast<const ColumnConst *>(column))
|
||||
{
|
||||
if (typeid_cast<const ColumnUInt16 *>(&column_const->getDataColumn()))
|
||||
return date_lut.fromDayNum(DayNum(column_const->getValue<UInt16>()));
|
||||
else if (typeid_cast<const ColumnUInt32 *>(&column_const->getDataColumn()))
|
||||
return column_const->getValue<UInt32>();
|
||||
}
|
||||
|
||||
throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -361,9 +361,8 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
throw Exception("Cannot create table from metadata file " + table_metadata_path + ", error: " + e.displayText() +
|
||||
", stack trace:\n" + e.getStackTrace().toString(),
|
||||
ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA);
|
||||
throw Exception("Cannot create table from metadata file " + table_metadata_path + ". Error: " + DB::getCurrentExceptionMessage(true),
|
||||
e, DB::ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -81,9 +81,8 @@ try
|
||||
catch (const Exception & e)
|
||||
{
|
||||
throw Exception(
|
||||
"Cannot create object '" + query.table + "' from query " + serializeAST(query) + ", error: " + e.displayText() + ", stack trace:\n"
|
||||
+ e.getStackTrace().toString(),
|
||||
ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA);
|
||||
"Cannot create object '" + query.table + "' from query " + serializeAST(query) + ". Error: " + DB::getCurrentExceptionMessage(true),
|
||||
e, DB::ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA);
|
||||
}
|
||||
|
||||
|
||||
@ -138,8 +137,7 @@ void DatabaseOrdinary::loadStoredObjects(
|
||||
catch (const Exception & e)
|
||||
{
|
||||
throw Exception(
|
||||
"Cannot parse definition from metadata file " + full_path + ", error: " + e.displayText() + ", stack trace:\n"
|
||||
+ e.getStackTrace().toString(), ErrorCodes::CANNOT_PARSE_TEXT);
|
||||
"Cannot parse definition from metadata file " + full_path + ". Error: " + DB::getCurrentExceptionMessage(true), e, ErrorCodes::CANNOT_PARSE_TEXT);
|
||||
}
|
||||
|
||||
});
|
||||
|
@ -140,6 +140,7 @@ struct ContextShared
|
||||
ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections.
|
||||
InterserverIOHandler interserver_io_handler; /// Handler for interserver communication.
|
||||
std::optional<BackgroundProcessingPool> background_pool; /// The thread pool for the background work performed by the tables.
|
||||
std::optional<BackgroundProcessingPool> background_move_pool; /// The thread pool for the background moves performed by the tables.
|
||||
std::optional<BackgroundSchedulePool> schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables)
|
||||
MultiVersion<Macros> macros; /// Substitutions extracted from config.
|
||||
std::unique_ptr<DDLWorker> ddl_worker; /// Process ddl commands from zk.
|
||||
@ -287,6 +288,7 @@ struct ContextShared
|
||||
external_dictionaries_loader.reset();
|
||||
external_models_loader.reset();
|
||||
background_pool.reset();
|
||||
background_move_pool.reset();
|
||||
schedule_pool.reset();
|
||||
ddl_worker.reset();
|
||||
|
||||
@ -1489,6 +1491,14 @@ BackgroundProcessingPool & Context::getBackgroundPool()
|
||||
return *shared->background_pool;
|
||||
}
|
||||
|
||||
BackgroundProcessingPool & Context::getBackgroundMovePool()
|
||||
{
|
||||
auto lock = getLock();
|
||||
if (!shared->background_move_pool)
|
||||
shared->background_move_pool.emplace(settings.background_move_pool_size, "BackgroundMovePool", "BgMoveProcPool");
|
||||
return *shared->background_move_pool;
|
||||
}
|
||||
|
||||
BackgroundSchedulePool & Context::getSchedulePool()
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
@ -450,6 +450,7 @@ public:
|
||||
void dropCaches() const;
|
||||
|
||||
BackgroundProcessingPool & getBackgroundPool();
|
||||
BackgroundProcessingPool & getBackgroundMovePool();
|
||||
BackgroundSchedulePool & getSchedulePool();
|
||||
|
||||
void setDDLWorker(std::unique_ptr<DDLWorker> ddl_worker);
|
||||
|
@ -34,6 +34,8 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ABORTED;
|
||||
extern const int UNKNOWN_CODEC;
|
||||
extern const int CANNOT_DECOMPRESS;
|
||||
extern const int INCORRECT_FILE_NAME;
|
||||
extern const int CHECKSUM_DOESNT_MATCH;
|
||||
extern const int TOO_LARGE_SIZE_COMPRESSED;
|
||||
@ -577,6 +579,8 @@ bool StorageDistributedDirectoryMonitor::isFileBrokenErrorCode(int code)
|
||||
return code == ErrorCodes::CHECKSUM_DOESNT_MATCH
|
||||
|| code == ErrorCodes::TOO_LARGE_SIZE_COMPRESSED
|
||||
|| code == ErrorCodes::CANNOT_READ_ALL_DATA
|
||||
|| code == ErrorCodes::UNKNOWN_CODEC
|
||||
|| code == ErrorCodes::CANNOT_DECOMPRESS
|
||||
|| code == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF;
|
||||
}
|
||||
|
||||
|
@ -61,9 +61,12 @@ void BackgroundProcessingPoolTaskInfo::wake()
|
||||
}
|
||||
|
||||
|
||||
BackgroundProcessingPool::BackgroundProcessingPool(int size_) : size(size_)
|
||||
BackgroundProcessingPool::BackgroundProcessingPool(int size_, const char * log_name, const char * thread_name_)
|
||||
: size(size_)
|
||||
, thread_name(thread_name_)
|
||||
{
|
||||
LOG_INFO(&Logger::get("BackgroundProcessingPool"), "Create BackgroundProcessingPool with " << size << " threads");
|
||||
logger = &Logger::get(log_name);
|
||||
LOG_INFO(logger, "Create " << log_name << " with " << size << " threads");
|
||||
|
||||
threads.resize(size);
|
||||
for (auto & thread : threads)
|
||||
@ -122,7 +125,7 @@ BackgroundProcessingPool::~BackgroundProcessingPool()
|
||||
|
||||
void BackgroundProcessingPool::threadFunction()
|
||||
{
|
||||
setThreadName("BackgrProcPool");
|
||||
setThreadName(thread_name);
|
||||
|
||||
{
|
||||
std::lock_guard lock(tasks_mutex);
|
||||
|
@ -46,7 +46,9 @@ public:
|
||||
using TaskHandle = std::shared_ptr<TaskInfo>;
|
||||
|
||||
|
||||
BackgroundProcessingPool(int size_);
|
||||
BackgroundProcessingPool(int size_,
|
||||
const char * log_name = "BackgroundProcessingPool",
|
||||
const char * thread_name_ = "BackgrProcPool");
|
||||
|
||||
size_t getNumberOfThreads() const
|
||||
{
|
||||
@ -67,6 +69,8 @@ protected:
|
||||
using Threads = std::vector<ThreadFromGlobalPool>;
|
||||
|
||||
const size_t size;
|
||||
const char * thread_name;
|
||||
Poco::Logger * logger;
|
||||
|
||||
Tasks tasks; /// Ordered in priority.
|
||||
std::mutex tasks_mutex;
|
||||
|
@ -544,19 +544,6 @@ void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const Strin
|
||||
}
|
||||
}
|
||||
|
||||
bool has_date_column = false;
|
||||
for (const auto & elem : ttl_expression->getRequiredColumnsWithTypes())
|
||||
{
|
||||
if (typeid_cast<const DataTypeDateTime *>(elem.type.get()) || typeid_cast<const DataTypeDate *>(elem.type.get()))
|
||||
{
|
||||
has_date_column = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (!has_date_column)
|
||||
throw Exception("TTL expression should use at least one Date or DateTime column", ErrorCodes::BAD_TTL_EXPRESSION);
|
||||
|
||||
const auto & result_column = ttl_expression->getSampleBlock().getByName(result_column_name);
|
||||
|
||||
if (!typeid_cast<const DataTypeDateTime *>(result_column.type.get())
|
||||
@ -3479,6 +3466,11 @@ bool MergeTreeData::selectPartsAndMove()
|
||||
return moveParts(std::move(moving_tagger));
|
||||
}
|
||||
|
||||
bool MergeTreeData::areBackgroundMovesNeeded() const
|
||||
{
|
||||
return storage_policy->getVolumes().size() > 1;
|
||||
}
|
||||
|
||||
bool MergeTreeData::movePartsToSpace(const DataPartsVector & parts, SpacePtr space)
|
||||
{
|
||||
if (parts_mover.moves_blocker.isCancelled())
|
||||
|
@ -939,6 +939,8 @@ protected:
|
||||
/// Selects parts for move and moves them, used in background process
|
||||
bool selectPartsAndMove();
|
||||
|
||||
bool areBackgroundMovesNeeded() const;
|
||||
|
||||
private:
|
||||
/// RAII Wrapper for atomic work with currently moving parts
|
||||
/// Acuire them in constructor and remove them in destructor
|
||||
|
@ -96,8 +96,22 @@ void updateTTL(const MergeTreeData::TTLEntry & ttl_entry, MergeTreeDataPart::TTL
|
||||
for (const auto & val : column_date_time->getData())
|
||||
ttl_info.update(val);
|
||||
}
|
||||
else if (const ColumnConst * column_const = typeid_cast<const ColumnConst *>(column))
|
||||
{
|
||||
if (typeid_cast<const ColumnUInt16 *>(&column_const->getDataColumn()))
|
||||
{
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
ttl_info.update(date_lut.fromDayNum(DayNum(column_const->getValue<UInt16>())));
|
||||
}
|
||||
else if (typeid_cast<const ColumnUInt32 *>(&column_const->getDataColumn()))
|
||||
{
|
||||
ttl_info.update(column_const->getValue<UInt32>());
|
||||
}
|
||||
else
|
||||
throw Exception("Unexpected type of result ttl column", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
else
|
||||
throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
ttl_infos.updatePartMinMaxTTL(ttl_info.min, ttl_info.max);
|
||||
}
|
||||
|
@ -214,7 +214,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup()
|
||||
}
|
||||
catch (const Coordination::Exception & e)
|
||||
{
|
||||
LOG_ERROR(log, "Couldn't start replication: " << e.what() << ", " << e.displayText() << ", stack trace:\n" << e.getStackTrace().toString());
|
||||
LOG_ERROR(log, "Couldn't start replication: " << e.what() << ". " << DB::getCurrentExceptionMessage(true));
|
||||
return false;
|
||||
}
|
||||
catch (const Exception & e)
|
||||
@ -222,7 +222,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup()
|
||||
if (e.code() != ErrorCodes::REPLICA_IS_ALREADY_ACTIVE)
|
||||
throw;
|
||||
|
||||
LOG_ERROR(log, "Couldn't start replication: " << e.what() << ", " << e.displayText() << ", stack trace:\n" << e.getStackTrace().toString());
|
||||
LOG_ERROR(log, "Couldn't start replication: " << e.what() << ". " << DB::getCurrentExceptionMessage(true));
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
@ -99,7 +99,8 @@ void StorageMergeTree::startup()
|
||||
/// NOTE background task will also do the above cleanups periodically.
|
||||
time_after_previous_cleanup.restart();
|
||||
merging_mutating_task_handle = global_context.getBackgroundPool().addTask([this] { return mergeMutateTask(); });
|
||||
moving_task_handle = global_context.getBackgroundPool().addTask([this] { return movePartsTask(); });
|
||||
if (areBackgroundMovesNeeded())
|
||||
moving_task_handle = global_context.getBackgroundMovePool().addTask([this] { return movePartsTask(); });
|
||||
}
|
||||
|
||||
|
||||
@ -115,7 +116,7 @@ void StorageMergeTree::shutdown()
|
||||
global_context.getBackgroundPool().removeTask(merging_mutating_task_handle);
|
||||
|
||||
if (moving_task_handle)
|
||||
global_context.getBackgroundPool().removeTask(moving_task_handle);
|
||||
global_context.getBackgroundMovePool().removeTask(moving_task_handle);
|
||||
}
|
||||
|
||||
|
||||
|
@ -2878,7 +2878,8 @@ void StorageReplicatedMergeTree::startup()
|
||||
data_parts_exchange_endpoint->getId(replica_path), data_parts_exchange_endpoint, global_context.getInterserverIOHandler());
|
||||
|
||||
queue_task_handle = global_context.getBackgroundPool().addTask([this] { return queueTask(); });
|
||||
move_parts_task_handle = global_context.getBackgroundPool().addTask([this] { return movePartsTask(); });
|
||||
if (areBackgroundMovesNeeded())
|
||||
move_parts_task_handle = global_context.getBackgroundMovePool().addTask([this] { return movePartsTask(); });
|
||||
|
||||
/// In this thread replica will be activated.
|
||||
restarting_thread.start();
|
||||
@ -2902,7 +2903,7 @@ void StorageReplicatedMergeTree::shutdown()
|
||||
queue_task_handle.reset();
|
||||
|
||||
if (move_parts_task_handle)
|
||||
global_context.getBackgroundPool().removeTask(move_parts_task_handle);
|
||||
global_context.getBackgroundMovePool().removeTask(move_parts_task_handle);
|
||||
move_parts_task_handle.reset();
|
||||
|
||||
if (data_parts_exchange_endpoint_holder)
|
||||
|
@ -18,7 +18,7 @@ Poco::Path getMarksFile(const std::string & part_path)
|
||||
return p;
|
||||
++it;
|
||||
}
|
||||
throw Exception("Cannot find any mark file in directory " + part_path, DB::ErrorCodes::METRIKA_OTHER_ERROR);
|
||||
throw Exception("Cannot find any mark file in directory " + part_path, DB::ErrorCodes::POCO_EXCEPTION);
|
||||
}
|
||||
|
||||
MergeTreeIndexGranularity readGranularity(const Poco::Path & mrk_file_path, size_t fixed_granularity)
|
||||
|
@ -6,3 +6,11 @@
|
||||
2000-10-10 00:00:00 0
|
||||
2100-10-10 00:00:00 3
|
||||
2100-10-10 2
|
||||
CREATE TABLE default.ttl_00933_1 (`b` Int32, `a` Int32 TTL now() - 1000) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
1 0
|
||||
CREATE TABLE default.ttl_00933_1 (`b` Int32, `a` Int32 TTL now() + 1000) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
1 1
|
||||
CREATE TABLE default.ttl_00933_1 (`b` Int32, `a` Int32 TTL today() - 1) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
1 0
|
||||
CREATE TABLE default.ttl_00933_1 (`b` Int32, `a` Int32 TTL today() + 1) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
1 1
|
||||
|
@ -47,6 +47,42 @@ select sleep(0.7) format Null; -- wait if very fast merge happen
|
||||
optimize table ttl_00933_1 final;
|
||||
select * from ttl_00933_1 order by d;
|
||||
|
||||
-- const DateTime TTL positive
|
||||
drop table if exists ttl_00933_1;
|
||||
create table ttl_00933_1 (b Int, a Int ttl now()-1000) engine = MergeTree order by tuple() partition by tuple();
|
||||
show create table ttl_00933_1;
|
||||
insert into ttl_00933_1 values (1, 1);
|
||||
select sleep(0.7) format Null; -- wait if very fast merge happen
|
||||
optimize table ttl_00933_1 final;
|
||||
select * from ttl_00933_1;
|
||||
|
||||
-- const DateTime TTL negative
|
||||
drop table if exists ttl_00933_1;
|
||||
create table ttl_00933_1 (b Int, a Int ttl now()+1000) engine = MergeTree order by tuple() partition by tuple();
|
||||
show create table ttl_00933_1;
|
||||
insert into ttl_00933_1 values (1, 1);
|
||||
select sleep(0.7) format Null; -- wait if very fast merge happen
|
||||
optimize table ttl_00933_1 final;
|
||||
select * from ttl_00933_1;
|
||||
|
||||
-- const Date TTL positive
|
||||
drop table if exists ttl_00933_1;
|
||||
create table ttl_00933_1 (b Int, a Int ttl today()-1) engine = MergeTree order by tuple() partition by tuple();
|
||||
show create table ttl_00933_1;
|
||||
insert into ttl_00933_1 values (1, 1);
|
||||
select sleep(0.7) format Null; -- wait if very fast merge happen
|
||||
optimize table ttl_00933_1 final;
|
||||
select * from ttl_00933_1;
|
||||
|
||||
-- const Date TTL negative
|
||||
drop table if exists ttl_00933_1;
|
||||
create table ttl_00933_1 (b Int, a Int ttl today()+1) engine = MergeTree order by tuple() partition by tuple();
|
||||
show create table ttl_00933_1;
|
||||
insert into ttl_00933_1 values (1, 1);
|
||||
select sleep(0.7) format Null; -- wait if very fast merge happen
|
||||
optimize table ttl_00933_1 final;
|
||||
select * from ttl_00933_1;
|
||||
|
||||
set send_logs_level = 'none';
|
||||
|
||||
drop table if exists ttl_00933_1;
|
||||
@ -54,7 +90,6 @@ drop table if exists ttl_00933_1;
|
||||
create table ttl_00933_1 (d DateTime ttl d) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 44}
|
||||
create table ttl_00933_1 (d DateTime, a Int ttl d) engine = MergeTree order by a partition by toSecond(d); -- { serverError 44}
|
||||
create table ttl_00933_1 (d DateTime, a Int ttl 2 + 2) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 450 }
|
||||
create table ttl_00933_1 (d DateTime, a Int ttl toDateTime(1)) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 450 }
|
||||
create table ttl_00933_1 (d DateTime, a Int ttl d - d) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 450 }
|
||||
|
||||
create table ttl_00933_1 (d DateTime, a Int ttl d + interval 1 day) engine = Log; -- { serverError 36 }
|
||||
|
@ -131,7 +131,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va
|
||||
|
||||
`clickhouse-client` uses the first existing file of the following:
|
||||
|
||||
- Defined in the `-config-file` parameter.
|
||||
- Defined in the `--config-file` parameter.
|
||||
- `./clickhouse-client.xml`
|
||||
- `~/.clickhouse-client/config.xml`
|
||||
- `/etc/clickhouse-client/config.xml`
|
||||
|
@ -72,7 +72,7 @@ For a description of parameters, see the [CREATE query description](../../query_
|
||||
|
||||
- `TTL` — An expression for setting storage time for rows.
|
||||
|
||||
It must depend on the `Date` or `DateTime` column and have one `Date` or `DateTime` column as a result. Example:
|
||||
It must have one `Date` or `DateTime` column as a result. Example:
|
||||
`TTL date + INTERVAL 1 DAY`
|
||||
|
||||
For more details, see [TTL for columns and tables](#table_engine-mergetree-ttl)
|
||||
@ -373,7 +373,7 @@ Determines the lifetime of values.
|
||||
|
||||
The `TTL` clause can be set for the whole table and for each individual column. If both `TTL` are set, ClickHouse uses that `TTL` which expires earlier.
|
||||
|
||||
The table must have the column in the [Date](../../data_types/date.md) or [DateTime](../../data_types/datetime.md) data type. To define the lifetime of data, use operations on this time column, for example:
|
||||
To define the lifetime of data, use expression evaluating to [Date](../../data_types/date.md) or [DateTime](../../data_types/datetime.md) data type, for example:
|
||||
|
||||
```sql
|
||||
TTL time_column
|
||||
|
@ -253,7 +253,7 @@ ALTER TABLE visits ATTACH PART 201901_2_2_0;
|
||||
|
||||
Read more about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr).
|
||||
|
||||
This query is replicated. Each replica checks whether there is data in the `detached` directory. If the data is in this directory, the query checks the integrity, verifies that it matches the data on the server that initiated the query. If everything is correct, the query adds data to the replica. If not, it downloads data from the query requestor replica, or from another replica where the data has already been added.
|
||||
This query is replicated. The replica-initiator checks whether there is data in the `detached` directory. If data exists, the query checks its integrity. If everything is correct, the query adds the data to the table. All other replicas download the data from the replica-initiator.
|
||||
|
||||
So you can put data to the `detached` directory on one replica, and use the `ALTER ... ATTACH` query to add it to the table on all replicas.
|
||||
|
||||
|
@ -181,7 +181,7 @@ Changes already made by the mutation are not rolled back.
|
||||
## OPTIMIZE {#misc_operations-optimize}
|
||||
|
||||
```sql
|
||||
OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION ID 'partition_id'] [FINAL]
|
||||
OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION ID 'partition_id'] [FINAL] [DEDUPLICATE]
|
||||
```
|
||||
|
||||
This query tries to initialize an unscheduled merge of data parts for tables with a table engine from the [MergeTree](../operations/table_engines/mergetree.md) family. Other kinds of table engines aren't supported.
|
||||
@ -191,6 +191,7 @@ When `OPTIMIZE` is used with the [ReplicatedMergeTree](../operations/table_engin
|
||||
- If `OPTIMIZE` doesn't perform a merge for any reason, it doesn't notify the client. To enable notifications, use the [optimize_throw_if_noop](../operations/settings/settings.md#setting-optimize_throw_if_noop) setting.
|
||||
- If you specify a `PARTITION`, only the specified partition is optimized. [How to set partition expression](alter.md#alter-how-to-specify-part-expr).
|
||||
- If you specify `FINAL`, optimization is performed even when all the data is already in one part.
|
||||
- If you specify `DEDUPLICATE`, then completely identical rows will be deduplicated (all columns are compared), it makes sense only for the MergeTree engine.
|
||||
|
||||
!!! warning "Warning"
|
||||
`OPTIMIZE` can't fix the "Too many parts" error.
|
||||
|
@ -7,7 +7,7 @@ Groups of operators are listed in order of priority (the higher it is in the lis
|
||||
|
||||
`a[N]` Access to an element of an array; ` arrayElement(a, N) function`.
|
||||
|
||||
`a.N` – Access to a tuble element; `tupleElement(a, N)` function.
|
||||
`a.N` – Access to a tuple element; `tupleElement(a, N)` function.
|
||||
|
||||
## Numeric Negation Operator
|
||||
|
||||
|
@ -3,7 +3,7 @@
|
||||
- [RELOAD DICTIONARIES](#query_language-system-reload-dictionaries)
|
||||
- [RELOAD DICTIONARY](#query_language-system-reload-dictionary)
|
||||
- [DROP DNS CACHE](#query_language-system-drop-dns-cache)
|
||||
- [DROP MARKS CACHE](#query_language-system-drop-marks-cache)
|
||||
- [DROP MARK CACHE](#query_language-system-drop-mark-cache)
|
||||
- [FLUSH LOGS](#query_language-system-flush_logs)
|
||||
- [RELOAD CONFIG](#query_language-system-reload-config)
|
||||
- [SHUTDOWN](#query_language-system-shutdown)
|
||||
@ -36,7 +36,7 @@ Resets ClickHouse's internal DNS cache. Sometimes (for old ClickHouse versions)
|
||||
|
||||
For more convenient (automatic) cache management, see disable_internal_dns_cache, dns_cache_update_period parameters.
|
||||
|
||||
## DROP MARKS CACHE {#query_language-system-drop-marks-cache}
|
||||
## DROP MARK CACHE {#query_language-system-drop-mark-cache}
|
||||
|
||||
Resets the mark cache. Used in development of ClickHouse and performance tests.
|
||||
|
||||
|
@ -96,13 +96,13 @@ command line برا پایه 'readline' (و 'history' یا 'libedit'، یه بد
|
||||
- `--vertical, -E` اگر مشخص شود، از فرمت Vertical برای نمایش خروجی استفاده می شود. این گزینه مشابه '--format=Vertical' می باشد. در این فرمت، هر مقدار در یک خط جدید چاپ می شود، که در هنگام نمایش جداول عریض مفید است.
|
||||
- `--time, -t` اگر مشخص شود، در حالت non-interactive زمان اجرای query در 'stderr' جاپ می شود.
|
||||
- `--stacktrace` – اگر مشخص شود stack trase مربوط به اجرای query در هنگام رخ دادن یک exception چاپ می شود.
|
||||
- `-config-file` – نام فایل پیکربندی.
|
||||
- `--config-file` – نام فایل پیکربندی.
|
||||
|
||||
### فایل های پیکربندی
|
||||
|
||||
`clickhouse-client` به ترتیب اولویت زیر از اولین فایل موجود برای ست کردن تنظیمات استفاده می کند:
|
||||
|
||||
- مشخص شده در پارامتر `-config-file`
|
||||
- مشخص شده در پارامتر `--config-file`
|
||||
- `./clickhouse-client.xml`
|
||||
- `\~/.clickhouse-client/config.xml`
|
||||
- `/etc/clickhouse-client/config.xml`
|
||||
|
@ -226,18 +226,24 @@ ClickHouse использует небольшое подмножество фу
|
||||
|
||||
### 4.3. Ограничение числа одновременных скачиваний с реплик.
|
||||
|
||||
Дмитрий Григорьев, ВШЭ.
|
||||
Изначально делал Олег Алексеенков, но пока решение не готово, хотя там не так уж много доделывать.
|
||||
|
||||
### 4.4. Ограничение сетевой полосы при репликации.
|
||||
|
||||
Дмитрий Григорьев, ВШЭ.
|
||||
|
||||
### 4.5. Возможность продолжить передачу куска данных при репликации после сбоя.
|
||||
|
||||
Дмитрий Григорьев, ВШЭ.
|
||||
|
||||
### 4.6. p2p передача для GLOBAL подзапросов.
|
||||
|
||||
### 4.7. Ленивая загрузка множеств для IN и JOIN с помощью k/v запросов.
|
||||
|
||||
### 4.8. Разделить background pool для fetch и merge.
|
||||
|
||||
Дмитрий Григорьев, ВШЭ.
|
||||
В очереди. Исправить проблему, что восстанавливающаяся реплика перестаёт мержить. Частично компенсируется 4.3.
|
||||
|
||||
|
||||
@ -458,6 +464,20 @@ Fuzzing тестирование - это тестирование случай
|
||||
|
||||
1. Добавление в SQL диалект ClickHouse функций для генерации случайных данных (пример - случайные бинарные строки заданной длины, случайные валидные UTF-8 строки) и "порчи" данных (например, поменять значения случайных бит с заданной частотой). Это будет использовано для тестирования SQL-функций ClickHouse.
|
||||
|
||||
Можно добавить функции:
|
||||
`randomString(length)`
|
||||
`randomFixedString(length)`
|
||||
- строка заданной длины с равномерно распределёнными случайными байтами;
|
||||
`randomStringASCII(length)`
|
||||
`randomStringUTF8(length)`
|
||||
|
||||
`fuzzBits(s, inverse_probability)` - изменить каждый бит строки на противоположный с заданной вероятностью;
|
||||
`fuzzBytes(s, inverse_probability)` - изменить каждый байт строки на равномерно случайный с заданной вероятностью;
|
||||
|
||||
У каждой функции опциональный аргумент против склейки одинаковых выражений в запросе.
|
||||
|
||||
Также можно сделать функции с детерминированным генератором случайных чисел (аргументом передаётся seed) для воспроизводимости тестовых кейсов.
|
||||
|
||||
### 7.24. Fuzzing лексера и парсера запросов; кодеков и форматов.
|
||||
|
||||
Андрей Некрашевич, ВШЭ.
|
||||
@ -563,12 +583,20 @@ Fuzzing тестирование - это тестирование случай
|
||||
|
||||
### 8.10. Запись в табличную функцию ODBC.
|
||||
|
||||
Артемий Бобровский, ВШЭ
|
||||
|
||||
### 8.11. Движок таблиц для чтения из Mongo.
|
||||
|
||||
Артемий Бобровский, ВШЭ
|
||||
|
||||
### 8.12. Пропуск столбцов в форматах Parquet, ORC.
|
||||
|
||||
Артемий Бобровский, ВШЭ
|
||||
|
||||
### 8.13. Поддержка массивов в Parquet, ORC.
|
||||
|
||||
Артемий Бобровский, ВШЭ
|
||||
|
||||
### 8.14. Запись данных в ORC.
|
||||
|
||||
Возможно, Андрей Коняев, ArenaData (зависит от желания).
|
||||
|
@ -72,7 +72,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
|
||||
|
||||
- `TTL` — выражение, определяющее длительность хранения строк.
|
||||
|
||||
Должно зависеть от столбца `Date` или `DateTime` и возвращать столбец `Date` или `DateTime`. Пример:`TTL date + INTERVAL 1 DAY`
|
||||
Должно возвращать столбец `Date` или `DateTime`. Пример: `TTL date + INTERVAL 1 DAY`.
|
||||
|
||||
Дополнительные сведения смотрите в разделе [TTL для столбцов и таблиц](#table_engine-mergetree-ttl)
|
||||
|
||||
@ -365,7 +365,7 @@ hasToken | ✗ | ✗ | ✗ | ✔ | ✗
|
||||
|
||||
Секция `TTL` может быть установлена как для всей таблицы, так и для каждого отдельного столбца. Если установлены оба `TTL`, то ClickHouse использует тот, что истекает раньше.
|
||||
|
||||
Таблица должна иметь столбец типа [Date](../../data_types/date.md) или [DateTime](../../data_types/datetime.md). Для установки времени жизни данных, следует использовать операцию со столбцом с временем, например:
|
||||
Для установки времени жизни данных, следует использовать выражение, возвращающее тип [Date](../../data_types/date.md) или [DateTime](../../data_types/datetime.md), например:
|
||||
|
||||
```sql
|
||||
TTL time_column
|
||||
|
@ -252,7 +252,7 @@ ALTER TABLE visits ATTACH PART 201901_2_2_0;
|
||||
|
||||
Как корректно задать имя партиции или куска, см. в разделе [Как задавать имя партиции в запросах ALTER](#alter-how-to-specify-part-expr).
|
||||
|
||||
Этот запрос реплицируется. Каждая реплика проверяет, есть ли данные в директории `detached`. Если данные есть, то запрос проверяет их целостность и соответствие данным на сервере-инициаторе запроса. В случае успеха данные добавляются в таблицу. В противном случае, реплика загружает данные с реплики-инициатора запроса или с другой реплики, на которой эти данные уже добавлены.
|
||||
Этот запрос реплицируется. Реплика-иницатор проверяет, есть ли данные в директории `detached`. Если данные есть, то запрос проверяет их целостность. В случае успеха данные добавляются в таблицу. Все остальные реплики загружают данные с реплики-инициатора запроса.
|
||||
|
||||
Это означает, что вы можете разместить данные в директории `detached` на одной реплике и с помощью запроса `ALTER ... ATTACH` добавить их в таблицу на всех репликах.
|
||||
|
||||
|
@ -173,7 +173,7 @@ KILL MUTATION WHERE database = 'default' AND table = 'table' AND mutation_id = '
|
||||
## OPTIMIZE {#misc_operations-optimize}
|
||||
|
||||
```sql
|
||||
OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION ID 'partition_id'] [FINAL]
|
||||
OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION ID 'partition_id'] [FINAL] [DEDUPLICATE]
|
||||
```
|
||||
|
||||
Запрос пытается запустить внеплановый мёрж кусков данных для таблиц семейства [MergeTree](../operations/table_engines/mergetree.md). Другие движки таблиц не поддерживаются.
|
||||
@ -183,6 +183,7 @@ OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION I
|
||||
- Если `OPTIMIZE` не выполняет мёрж по любой причине, ClickHouse не оповещает об этом клиента. Чтобы включить оповещения, используйте настройку [optimize_throw_if_noop](../operations/settings/settings.md#setting-optimize_throw_if_noop).
|
||||
- Если указать `PARTITION`, то оптимизация выполняется только для указанной партиции. [Как задавать имя партиции в запросах](alter.md#alter-how-to-specify-part-expr).
|
||||
- Если указать `FINAL`, то оптимизация выполняется даже в том случае, если все данные уже лежат в одном куске.
|
||||
- Если указать `DEDUPLICATE`, то произойдет схлопывание полностью одинаковых строк (сравниваются значения во всех колонках), имеет смысл только для движка MergeTree.
|
||||
|
||||
!!! warning "Внимание"
|
||||
Запрос `OPTIMIZE` не может устранить причину появления ошибки "Too many parts".
|
||||
|
@ -1 +0,0 @@
|
||||
../../en/development/build_cross.md
|
53
docs/zh/development/build_cross.md
Normal file
53
docs/zh/development/build_cross.md
Normal file
@ -0,0 +1,53 @@
|
||||
# 如何在Linux中编译Mac OS X ClickHouse
|
||||
|
||||
Linux机器也可以编译运行在OS X系统的`clickhouse`二进制包,这可以用于在Linux上跑持续集成测试。如果要直接在Mac OS X上构建ClickHouse,请参考另外一篇指南: https://clickhouse.yandex/docs/zh/development/build_osx/
|
||||
|
||||
Mac OS X的交叉编译基于以下构建说明,请首先遵循它们。
|
||||
|
||||
# Install Clang-8
|
||||
|
||||
按照https://apt.llvm.org/中的说明进行Ubuntu或Debian安装。
|
||||
例如,按照Bionic的命令如下:
|
||||
|
||||
```bash
|
||||
sudo echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-8 main" >> /etc/apt/sources.list
|
||||
sudo apt-get install clang-8
|
||||
```
|
||||
|
||||
# 安装交叉编译工具集
|
||||
|
||||
我们假设安装 `cctools` 在 ${CCTOOLS} 路径下
|
||||
|
||||
```bash
|
||||
mkdir ${CCTOOLS}
|
||||
|
||||
git clone https://github.com/tpoechtrager/apple-libtapi.git
|
||||
cd apple-libtapi
|
||||
INSTALLPREFIX=${CCTOOLS} ./build.sh
|
||||
./install.sh
|
||||
cd ..
|
||||
|
||||
git clone https://github.com/tpoechtrager/cctools-port.git
|
||||
cd cctools-port/cctools
|
||||
./configure --prefix=${CCTOOLS} --with-libtapi=${CCTOOLS} --target=x86_64-apple-darwin
|
||||
make install
|
||||
|
||||
cd ${CCTOOLS}
|
||||
wget https://github.com/phracker/MacOSX-SDKs/releases/download/10.14-beta4/MacOSX10.14.sdk.tar.xz
|
||||
tar xJf MacOSX10.14.sdk.tar.xz
|
||||
```
|
||||
|
||||
# 编译 ClickHouse
|
||||
|
||||
```bash
|
||||
cd ClickHouse
|
||||
mkdir build-osx
|
||||
CC=clang-8 CXX=clang++-8 cmake . -Bbuild-osx -DCMAKE_SYSTEM_NAME=Darwin \
|
||||
-DCMAKE_AR:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ar \
|
||||
-DCMAKE_RANLIB:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ranlib \
|
||||
-DLINKER_NAME=${CCTOOLS}/bin/x86_64-apple-darwin-ld \
|
||||
-DSDK_PATH=${CCTOOLS}/MacOSX10.14.sdk
|
||||
ninja -C build-osx
|
||||
```
|
||||
|
||||
生成的二进制文件将具有Mach-O可执行格式,并且不能在Linux上运行。
|
@ -1 +0,0 @@
|
||||
../../en/development/contrib.md
|
34
docs/zh/development/contrib.md
Normal file
34
docs/zh/development/contrib.md
Normal file
@ -0,0 +1,34 @@
|
||||
# 使用的三方库
|
||||
|
||||
| Library | License |
|
||||
| ------- | ------- |
|
||||
| base64 | [BSD 2-Clause License](https://github.com/aklomp/base64/blob/a27c565d1b6c676beaf297fe503c4518185666f7/LICENSE) |
|
||||
| boost | [Boost Software License 1.0](https://github.com/ClickHouse-Extras/boost-extra/blob/6883b40449f378019aec792f9983ce3afc7ff16e/LICENSE_1_0.txt) |
|
||||
| brotli | [MIT](https://github.com/google/brotli/blob/master/LICENSE) |
|
||||
| capnproto | [MIT](https://github.com/capnproto/capnproto/blob/master/LICENSE) |
|
||||
| cctz | [Apache License 2.0](https://github.com/google/cctz/blob/4f9776a310f4952454636363def82c2bf6641d5f/LICENSE.txt) |
|
||||
| double-conversion | [BSD 3-Clause License](https://github.com/google/double-conversion/blob/cf2f0f3d547dc73b4612028a155b80536902ba02/LICENSE) |
|
||||
| FastMemcpy | [MIT](https://github.com/yandex/ClickHouse/blob/master/libs/libmemcpy/impl/LICENSE) |
|
||||
| googletest | [BSD 3-Clause License](https://github.com/google/googletest/blob/master/LICENSE) |
|
||||
| hyperscan | [BSD 3-Clause License](https://github.com/intel/hyperscan/blob/master/LICENSE) |
|
||||
| libbtrie | [BSD 2-Clause License](https://github.com/yandex/ClickHouse/blob/master/contrib/libbtrie/LICENSE) |
|
||||
| libcxxabi | [BSD + MIT](https://github.com/yandex/ClickHouse/blob/master/libs/libglibc-compatibility/libcxxabi/LICENSE.TXT) |
|
||||
| libdivide | [Zlib License](https://github.com/yandex/ClickHouse/blob/master/contrib/libdivide/LICENSE.txt) |
|
||||
| libgsasl | [LGPL v2.1](https://github.com/ClickHouse-Extras/libgsasl/blob/3b8948a4042e34fb00b4fb987535dc9e02e39040/LICENSE)
|
||||
| libhdfs3 | [Apache License 2.0](https://github.com/ClickHouse-Extras/libhdfs3/blob/bd6505cbb0c130b0db695305b9a38546fa880e5a/LICENSE.txt) |
|
||||
| libmetrohash | [Apache License 2.0](https://github.com/yandex/ClickHouse/blob/master/contrib/libmetrohash/LICENSE) |
|
||||
| libpcg-random | [Apache License 2.0](https://github.com/yandex/ClickHouse/blob/master/contrib/libpcg-random/LICENSE-APACHE.txt) |
|
||||
| libressl | [OpenSSL License](https://github.com/ClickHouse-Extras/ssl/blob/master/COPYING) |
|
||||
| librdkafka | [BSD 2-Clause License](https://github.com/edenhill/librdkafka/blob/363dcad5a23dc29381cc626620e68ae418b3af19/LICENSE) |
|
||||
| libwidechar\_width | [CC0 1.0 Universal](https://github.com/yandex/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) |
|
||||
| llvm | [BSD 3-Clause License](https://github.com/ClickHouse-Extras/llvm/blob/163def217817c90fb982a6daf384744d8472b92b/llvm/LICENSE.TXT) |
|
||||
| lz4 | [BSD 2-Clause License](https://github.com/lz4/lz4/blob/c10863b98e1503af90616ae99725ecd120265dfb/LICENSE) |
|
||||
| mariadb-connector-c | [LGPL v2.1](https://github.com/ClickHouse-Extras/mariadb-connector-c/blob/3.1/COPYING.LIB) |
|
||||
| murmurhash | [Public Domain](https://github.com/yandex/ClickHouse/blob/master/contrib/murmurhash/LICENSE)
|
||||
| pdqsort | [Zlib License](https://github.com/yandex/ClickHouse/blob/master/contrib/pdqsort/license.txt) |
|
||||
| poco | [Boost Software License - Version 1.0](https://github.com/ClickHouse-Extras/poco/blob/fe5505e56c27b6ecb0dcbc40c49dc2caf4e9637f/LICENSE) |
|
||||
| protobuf | [BSD 3-Clause License](https://github.com/ClickHouse-Extras/protobuf/blob/12735370922a35f03999afff478e1c6d7aa917a4/LICENSE) |
|
||||
| re2 | [BSD 3-Clause License](https://github.com/google/re2/blob/7cf8b88e8f70f97fd4926b56aa87e7f53b2717e0/LICENSE) |
|
||||
| UnixODBC | [LGPL v2.1](https://github.com/ClickHouse-Extras/UnixODBC/tree/b0ad30f7f6289c12b76f04bfb9d466374bb32168) |
|
||||
| zlib-ng | [Zlib License](https://github.com/ClickHouse-Extras/zlib-ng/blob/develop/LICENSE.md) |
|
||||
| zstd | [BSD 3-Clause License](https://github.com/facebook/zstd/blob/dev/LICENSE) |
|
@ -9,7 +9,7 @@ ClickHose支持Linux,FreeBSD 及 Mac OS X 系统。
|
||||
|
||||
您需要(申请)一个GitHub账户来使用ClickHouse。
|
||||
|
||||
如果没有账户,请在https://github.com上注册一个。如果没有SSH密钥,请在本地创建密钥并将其上传到GitHub上。这些交互都是必须的,也可以使用与其他任何SSH服务器相同的密钥。
|
||||
如果没有账户,请在https://github.com上注册一个。如果没有SSH密钥,请在本地创建密钥并将公钥上传到GitHub上。这有助于你提交更新代码。并且在不同的SSH服务端,你也可以使用相同的SSH密钥。
|
||||
|
||||
要创建ClickHouse源码库的分支,请在https://github.com/ClickHouse/ClickHouse页面上点击右上角的"fork"按钮。它会在本账户上创建您个人的ClickHouse/ClickHouse分支。
|
||||
|
||||
|
@ -166,7 +166,7 @@ clickhouse benchmark --concurrency 16 < queries.tsv
|
||||
当我们扩展 ClickHouse 网络协议时,我们手动测试旧的 clickhouse-client 与新的 clickhouse-server 和新的clickhouse-client 一起使用旧的 clickhouse-server (只需从相应的包中运行二进制文件)
|
||||
|
||||
|
||||
## 来自编译器的帮助
|
||||
## 来自编译器的提示
|
||||
|
||||
ClickHouse 主要的代码 (位于`dbms`目录中) 使用 `-Wall -Wextra -Werror` 构建,并带有一些其他已启用的警告。 虽然没有为第三方库启用这些选项。
|
||||
|
||||
|
@ -89,13 +89,13 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA
|
||||
- `--vertical, -E` – 如果指定,默认情况下使用垂直格式输出结果。这与 '--format=Vertical' 相同。在这种格式中,每个值都在单独的行上打印,这种方式对显示宽表很有帮助。
|
||||
- `--time, -t` – 如果指定,非交互模式下会打印查询执行的时间到 'stderr' 中。
|
||||
- `--stacktrace` – 如果指定,如果出现异常,会打印堆栈跟踪信息。
|
||||
- `-config-file` – 配置文件的名称。
|
||||
- `--config-file` – 配置文件的名称。
|
||||
|
||||
### 配置文件
|
||||
|
||||
`clickhouse-client` 使用一下第一个存在的文件:
|
||||
|
||||
- 通过 `-config-file` 参数指定的文件.
|
||||
- 通过 `--config-file` 参数指定的文件.
|
||||
- `./clickhouse-client.xml`
|
||||
- `\~/.clickhouse-client/config.xml`
|
||||
- `/etc/clickhouse-client/config.xml`
|
||||
|
@ -1 +0,0 @@
|
||||
../../en/operations/monitoring.md
|
37
docs/zh/operations/monitoring.md
Normal file
37
docs/zh/operations/monitoring.md
Normal file
@ -0,0 +1,37 @@
|
||||
# 监控
|
||||
|
||||
可以监控到:
|
||||
|
||||
- 硬件资源的利用率。
|
||||
- ClickHouse 服务的指标。
|
||||
|
||||
## 硬件资源利用率
|
||||
|
||||
ClickHouse 本身不会去监控硬件资源的状态。
|
||||
|
||||
强烈推荐监控以下监控项:
|
||||
|
||||
- 处理器上的负载和温度。
|
||||
|
||||
可以使用 [dmesg](https://en.wikipedia.org/wiki/Dmesg), [turbostat](https://www.linux.org/docs/man8/turbostat.html) 或者其他工具。
|
||||
|
||||
- 磁盘存储,RAM和网络的使用率。
|
||||
|
||||
## ClickHouse 服务的指标。
|
||||
|
||||
ClickHouse服务本身具有用于自我状态监视指标。
|
||||
|
||||
要跟踪服务器事件,请观察服务器日志。 请参阅配置文件的[logger](server_settings/settings.md#server_settings-logger)部分。
|
||||
|
||||
ClickHouse 收集的指标项:
|
||||
|
||||
- 服务用于计算的资源占用的各种指标。
|
||||
- 关于查询处理的常见统计信息。
|
||||
|
||||
可以在 [system.metrics](system_tables.md#system_tables-metrics) ,[system.events](system_tables.md#system_tables-events) 以及[system.asynchronous_metrics](system_tables.md#system_tables-asynchronous_metrics) 等系统表查看所有的指标项。
|
||||
|
||||
可以配置ClickHouse 往 [Graphite](https://github.com/graphite-project)导入指标。 参考 [Graphite section](server_settings/settings.md#server_settings-graphite) 配置文件。在配置指标导出之前,需要参考Graphite[官方教程](https://graphite.readthedocs.io/en/latest/install.html)搭建服务。
|
||||
|
||||
此外,您可以通过HTTP API监视服务器可用性。 将HTTP GET请求发送到 `/`。 如果服务器可用,它将以 `200 OK` 响应。
|
||||
|
||||
要监视服务器集群的配置中,应设置[max_replica_delay_for_distributed_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries)参数并使用HTTP资源`/replicas-delay`。 如果副本可用,并且不延迟在其他副本之后,则对`/replicas-delay`的请求将返回200 OK。 如果副本被延迟,它将返回有关延迟信息。
|
@ -1 +0,0 @@
|
||||
../en/roadmap.md
|
16
docs/zh/roadmap.md
Normal file
16
docs/zh/roadmap.md
Normal file
@ -0,0 +1,16 @@
|
||||
# 规划
|
||||
|
||||
## Q3 2019
|
||||
|
||||
- 字典表的DDL
|
||||
- 与类S3对象存储集成
|
||||
- 冷热数据存储分离,支持JBOD
|
||||
|
||||
## Q4 2019
|
||||
|
||||
- JOIN 不受可用内存限制
|
||||
- 更精确的用户资源池,可以在用户之间合理分配集群资源
|
||||
- 细粒度的授权管理
|
||||
- 与外部认证服务集成
|
||||
|
||||
[来源文章](https://clickhouse.yandex/docs/en/roadmap/) <!--hide-->
|
@ -1 +0,0 @@
|
||||
../en/security_changelog.md
|
39
docs/zh/security_changelog.md
Normal file
39
docs/zh/security_changelog.md
Normal file
@ -0,0 +1,39 @@
|
||||
## 修复于 ClickHouse Release 18.12.13, 2018-09-10
|
||||
|
||||
### CVE-2018-14672
|
||||
|
||||
加载CatBoost模型的功能,允许遍历路径并通过错误消息读取任意文件。
|
||||
|
||||
来源: Yandex信息安全团队的Andrey Krasichkov
|
||||
|
||||
## 修复于 ClickHouse Release 18.10.3, 2018-08-13
|
||||
|
||||
### CVE-2018-14671
|
||||
|
||||
unixODBC允许从文件系统加载任意共享对象,从而导致“远程执行代码”漏洞。
|
||||
|
||||
来源:Yandex信息安全团队的Andrey Krasichkov和Evgeny Sidorov
|
||||
|
||||
## 修复于 ClickHouse Release 1.1.54388, 2018-06-28
|
||||
|
||||
### CVE-2018-14668
|
||||
远程表函数功能允许在 "user", "password" 及 "default_database" 字段中使用任意符号,从而导致跨协议请求伪造攻击。
|
||||
|
||||
来源:Yandex信息安全团队的Andrey Krasichkov
|
||||
|
||||
## 修复于 ClickHouse Release 1.1.54390, 2018-07-06
|
||||
|
||||
### CVE-2018-14669
|
||||
ClickHouse MySQL客户端启用了 "LOAD DATA LOCAL INFILE" 功能,该功能允许恶意MySQL数据库从连接的ClickHouse服务器读取任意文件。
|
||||
|
||||
来源:Yandex信息安全团队的Andrey Krasichkov和Evgeny Sidorov
|
||||
|
||||
## 修复于 ClickHouse Release 1.1.54131, 2017-01-10
|
||||
|
||||
### CVE-2018-14670
|
||||
|
||||
deb软件包中的错误配置可能导致使用未经授权的数据库。
|
||||
|
||||
来源:英国国家网络安全中心(NCSC)
|
||||
|
||||
[来源文章](https://clickhouse.yandex/docs/en/security_changelog/) <!--hide-->
|
@ -26,7 +26,7 @@ inline uint32_t HI_32(uint64_t x) { return static_cast<uint32_t>(x >> 32); }
|
||||
return std::numeric_limits<unsigned long long>::digits - __builtin_clzll(value);
|
||||
}
|
||||
#else
|
||||
/// Stupid realization for non GCC-like compilers. Can use BSR from x86 instructions set.
|
||||
/// Stupid implementation for non GCC-like compilers. Can use BSR from x86 instructions set.
|
||||
template <typename T>
|
||||
inline unsigned GetValueBitCountImpl(T value) noexcept {
|
||||
unsigned result = 1; // result == 0 - impossible value, since value cannot be zero
|
||||
|
Loading…
Reference in New Issue
Block a user