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

This commit is contained in:
proller 2019-09-06 21:38:17 +03:00
commit f8b184787c
60 changed files with 583 additions and 241 deletions

View File

@ -1,11 +1,11 @@
# This strings autochanged from release_lib.sh:
set(VERSION_REVISION 54425)
set(VERSION_REVISION 54426)
set(VERSION_MAJOR 19)
set(VERSION_MINOR 14)
set(VERSION_MINOR 15)
set(VERSION_PATCH 1)
set(VERSION_GITHASH adfc36917222bdb03eba069f0cad0f4f5b8f1c94)
set(VERSION_DESCRIBE v19.14.1.1-prestable)
set(VERSION_STRING 19.14.1.1)
set(VERSION_GITHASH 6f1a8c37abe6ee4e7ee74c0b5cb9c05a87417b61)
set(VERSION_DESCRIBE v19.15.1.1-prestable)
set(VERSION_STRING 19.15.1.1)
# end of autochange
set(VERSION_EXTRA "" CACHE STRING "")

View File

@ -431,8 +431,14 @@ private:
/// Load command history if present.
if (config().has("history_file"))
history_file = config().getString("history_file");
else if (!home_path.empty())
history_file = home_path + "/.clickhouse-client-history";
else
{
auto history_file_from_env = getenv("CLICKHOUSE_HISTORY_FILE");
if (history_file_from_env)
history_file = history_file_from_env;
else if (!home_path.empty())
history_file = home_path + "/.clickhouse-client-history";
}
if (!history_file.empty())
{

View File

@ -15,6 +15,7 @@
#include <common/logger_useful.h>
#include <ext/scope_guard.h>
#include <ext/range.h>
#include <Common/SensitiveDataMasker.h>
namespace DB
{
@ -165,8 +166,7 @@ int ODBCBridge::main(const std::vector<std::string> & /*args*/)
if (config().has("query_masking_rules"))
{
context->setSensitiveDataMasker(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
setLoggerSensitiveDataMasker(logger(), context->getSensitiveDataMasker());
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
}
auto server = Poco::Net::HTTPServer(

View File

@ -55,6 +55,7 @@
#include "TCPHandlerFactory.h"
#include "Common/config_version.h"
#include "MySQLHandlerFactory.h"
#include <Common/SensitiveDataMasker.h>
#if defined(__linux__)
@ -279,8 +280,6 @@ int Server::main(const std::vector<std::string> & /*args*/)
*/
LOG_INFO(log, "Shutting down storages.");
// global_context is the owner of sensitive_data_masker, which will be destoyed after global_context->shutdown() call
setLoggerSensitiveDataMasker(logger(), nullptr);
global_context->shutdown();
LOG_DEBUG(log, "Shutted down storages.");
@ -414,7 +413,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
if (config().has("query_masking_rules"))
{
global_context->setSensitiveDataMasker(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
}
auto main_config_reloader = std::make_unique<ConfigReloader>(config_path,
@ -426,10 +425,6 @@ int Server::main(const std::vector<std::string> & /*args*/)
{
setTextLog(global_context->getTextLog());
buildLoggers(*config, logger());
if (auto masker = global_context->getSensitiveDataMasker())
{
setLoggerSensitiveDataMasker(logger(), masker);
}
global_context->setClustersConfig(config);
global_context->setMacros(std::make_unique<Macros>(*config, "macros"));
},

View File

@ -447,7 +447,7 @@
<query_masking_rules>
<rule>
<name>hide SSN</name>
<regexp>(^|\D)\d{3}-\d{2}-\d{4}($|\D)</regexp>
<regexp>\b\d{3}-\d{2}-\d{4}\b</regexp>
<replace>000-00-0000</replace>
</rule>
</query_masking_rules>

View File

@ -262,8 +262,9 @@ public:
iterator end() { return iterator(this, buf + BUFFER_SIZE); }
protected:
void ALWAYS_INLINE emplaceImpl(Key x, iterator & it, bool & inserted)
public:
/// The last parameter is unused but exists for compatibility with HashTable interface.
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted, size_t /* hash */ = 0)
{
it = iterator(this, &buf[x]);
@ -278,22 +279,16 @@ protected:
++m_size;
}
public:
std::pair<iterator, bool> ALWAYS_INLINE insert(const value_type & x)
{
std::pair<iterator, bool> res;
emplaceImpl(Cell::getKey(x), res.first, res.second);
emplace(Cell::getKey(x), res.first, res.second);
if (res.second)
res.first.ptr->setMapped(x);
return res;
}
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted) { emplaceImpl(x, it, inserted); }
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted, size_t) { emplaceImpl(x, it, inserted); }
iterator ALWAYS_INLINE find(Key x)
{
return !buf[x].isZero(*this) ? iterator(this, &buf[x]) : end();

View File

@ -24,6 +24,7 @@ namespace DB
namespace ErrorCodes
{
extern const int CANNOT_COMPILE_REGEXP;
extern const int LOGICAL_ERROR;
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int INVALID_CONFIG_PARAMETER;
}
@ -38,7 +39,9 @@ private:
const RE2 regexp;
const re2::StringPiece replacement;
#ifndef NDEBUG
mutable std::atomic<std::uint64_t> matches_count = 0;
#endif
public:
//* TODO: option with hyperscan? https://software.intel.com/en-us/articles/why-and-how-to-replace-pcre-with-hyperscan
@ -61,15 +64,37 @@ public:
uint64_t apply(std::string & data) const
{
auto m = RE2::GlobalReplace(&data, regexp, replacement);
#ifndef NDEBUG
matches_count += m;
#endif
return m;
}
const std::string & getName() const { return name; }
const std::string & getReplacementString() const { return replacement_string; }
#ifndef NDEBUG
uint64_t getMatchesCount() const { return matches_count; }
#endif
};
std::unique_ptr<SensitiveDataMasker> SensitiveDataMasker::sensitive_data_masker = nullptr;
void SensitiveDataMasker::setInstance(std::unique_ptr<SensitiveDataMasker> sensitive_data_masker_)
{
if (!sensitive_data_masker_)
throw Exception("Logical error: the 'sensitive_data_masker' is not set", ErrorCodes::LOGICAL_ERROR);
if (sensitive_data_masker_->rulesCount() > 0)
{
sensitive_data_masker = std::move(sensitive_data_masker_);
}
}
SensitiveDataMasker * SensitiveDataMasker::getInstance()
{
return sensitive_data_masker.get();
}
SensitiveDataMasker::SensitiveDataMasker(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{

View File

@ -4,7 +4,6 @@
#include <vector>
#include <cstdint>
namespace Poco
{
namespace Util
@ -13,6 +12,32 @@ namespace Util
}
}
/// SensitiveDataMasker allows to remove sensitive data from queries using set of regexp-based rules
/// It's used as a singelton via getInstance method
/// Initially it's empty (nullptr) and after manual initialization
/// (one-time, done by setInstance call) it takes the proper value which
/// is stored in unique_ptr.
/// It looks like the singelton is the best option here, as
/// two users of that object (OwnSplitChannel & Interpreters/executeQuery)
/// can't own/share that Masker properly without syncronization & locks,
/// and we can't afford setting global locks for each logged line.
/// I've considered singleton alternatives, but it's unclear who should own the object,
/// and it introduce unnecessary complexity in implementation (passing references back and forward):
///
/// context can't own, as Context is destroyed before logger,
/// and logger lives longer and logging can still happen after Context destruction.
/// resetting masker in the logger at the moment of
/// context destruction can't be done w/o synchronization / locks in a safe manner.
///
/// logger is Poco derived and i didn't want to brake it's interface,
/// also logger can be dynamically reconfigured without server restart,
/// and it actually recreates OwnSplitChannel when reconfiguration happen,
/// so that makes it's quite tricky. So it a bad candidate for owning masker too.
namespace DB
{
class SensitiveDataMasker
@ -20,6 +45,7 @@ class SensitiveDataMasker
private:
class MaskingRule;
std::vector<std::unique_ptr<MaskingRule>> all_masking_rules;
static std::unique_ptr<SensitiveDataMasker> sensitive_data_masker;
public:
SensitiveDataMasker(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
@ -28,6 +54,11 @@ public:
/// Returns the number of matched rules.
size_t wipeSensitiveData(std::string & data) const;
/// setInstance is not thread-safe and should be called once in single-thread mode.
/// https://github.com/yandex/ClickHouse/pull/6810#discussion_r321183367
static void setInstance(std::unique_ptr<SensitiveDataMasker> sensitive_data_masker_);
static SensitiveDataMasker * getInstance();
/// Used in tests.
void addMaskingRule(const std::string & name, const std::string & regexp_string, const std::string & replacement_string);

View File

@ -16,4 +16,7 @@ using NameOrderedSet = std::set<std::string>;
using NameToNameMap = std::unordered_map<std::string, std::string>;
using NameToNameSetMap = std::unordered_map<std::string, NameSet>;
using NameWithAlias = std::pair<std::string, std::string>;
using NamesWithAliases = std::vector<NameWithAlias>;
}

View File

@ -858,7 +858,7 @@ void FunctionArrayElement::perform(Block & block, const ColumnNumbers & argument
builder.initSink(input_rows_count);
if (index == 0u)
throw Exception("Array indices is 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
throw Exception("Array indices are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
if (!(executeNumberConst<UInt8>(block, arguments, result, index, builder)
|| executeNumberConst<UInt16>(block, arguments, result, index, builder)

View File

@ -93,14 +93,14 @@ NameSet AnalyzedJoin::getOriginalColumnsSet() const
return out;
}
std::unordered_map<String, String> AnalyzedJoin::getOriginalColumnsMap(const NameSet & required_columns) const
NamesWithAliases AnalyzedJoin::getNamesWithAliases(const NameSet & required_columns) const
{
std::unordered_map<String, String> out;
NamesWithAliases out;
for (const auto & column : required_columns)
{
auto it = original_names.find(column);
if (it != original_names.end())
out.insert(*it);
out.emplace_back(it->second, it->first); /// {original_name, name}
}
return out;
}
@ -129,15 +129,15 @@ Names AnalyzedJoin::requiredJoinedNames() const
return Names(required_columns_set.begin(), required_columns_set.end());
}
void AnalyzedJoin::appendRequiredColumns(const Block & sample, NameSet & required_columns) const
NamesWithAliases AnalyzedJoin::getRequiredColumns(const Block & sample, const Names & action_required_columns) const
{
for (auto & column : key_names_right)
NameSet required_columns(action_required_columns.begin(), action_required_columns.end());
for (auto & column : requiredJoinedNames())
if (!sample.has(column))
required_columns.insert(column);
for (auto & column : columns_added_by_join)
if (!sample.has(column.name))
required_columns.insert(column.name);
return getNamesWithAliases(required_columns);
}
void AnalyzedJoin::addJoinedColumn(const NameAndTypePair & joined_column)

View File

@ -64,12 +64,12 @@ public:
NameSet getQualifiedColumnsSet() const;
NameSet getOriginalColumnsSet() const;
std::unordered_map<String, String> getOriginalColumnsMap(const NameSet & required_columns) const;
NamesWithAliases getNamesWithAliases(const NameSet & required_columns) const;
NamesWithAliases getRequiredColumns(const Block & sample, const Names & action_columns) const;
void deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix);
size_t rightKeyInclusion(const String & name) const;
void appendRequiredColumns(const Block & sample, NameSet & required_columns) const;
void addJoinedColumn(const NameAndTypePair & joined_column);
void addJoinedColumnsAndCorrectNullability(Block & sample_block) const;

View File

@ -30,7 +30,7 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin
/// Set as unchanged to avoid sending to remote server.
new_settings.max_concurrent_queries_for_user.changed = false;
new_settings.max_memory_usage_for_user.changed = false;
new_settings.max_memory_usage_for_all_queries = false;
new_settings.max_memory_usage_for_all_queries.changed = false;
Context new_context(context);
new_context.setSettings(new_settings);

View File

@ -143,8 +143,6 @@ struct ContextShared
std::unique_ptr<DDLWorker> ddl_worker; /// Process ddl commands from zk.
/// Rules for selecting the compression settings, depending on the size of the part.
mutable std::unique_ptr<CompressionCodecSelector> compression_codec_selector;
/// Allows to remove sensitive data from queries using set of regexp-based rules
std::unique_ptr<SensitiveDataMasker> sensitive_data_masker;
std::optional<MergeTreeSettings> merge_tree_settings; /// Settings of MergeTree* engines.
size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default)
size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default)
@ -287,8 +285,6 @@ struct ContextShared
/// Stop trace collector if any
trace_collector.reset();
sensitive_data_masker.reset();
}
bool hasTraceCollector()
@ -538,23 +534,6 @@ String Context::getUserFilesPath() const
return shared->user_files_path;
}
void Context::setSensitiveDataMasker(std::unique_ptr<SensitiveDataMasker> sensitive_data_masker)
{
if (!sensitive_data_masker)
throw Exception("Logical error: the 'sensitive_data_masker' is not set", ErrorCodes::LOGICAL_ERROR);
if (sensitive_data_masker->rulesCount() > 0)
{
auto lock = getLock();
shared->sensitive_data_masker = std::move(sensitive_data_masker);
}
}
SensitiveDataMasker * Context::getSensitiveDataMasker() const
{
return shared->sensitive_data_masker.get();
}
void Context::setPath(const String & path)
{
auto lock = getLock();
@ -1222,8 +1201,8 @@ void Context::setCurrentQueryId(const String & query_id)
} words;
} random;
random.words.a = thread_local_rng();
random.words.b = thread_local_rng();
random.words.a = thread_local_rng(); //-V656
random.words.b = thread_local_rng(); //-V656
/// Use protected constructor.
struct qUUID : Poco::UUID

View File

@ -12,7 +12,6 @@
#include <Common/MultiVersion.h>
#include <Common/ThreadPool.h>
#include "config_core.h"
#include <Common/SensitiveDataMasker.h>
#include <Storages/IStorage_fwd.h>
#include <atomic>
#include <chrono>
@ -178,9 +177,6 @@ public:
String getFlagsPath() const;
String getUserFilesPath() const;
void setSensitiveDataMasker(std::unique_ptr<SensitiveDataMasker> sensitive_data_masker);
SensitiveDataMasker * getSensitiveDataMasker() const;
void setPath(const String & path);
void setTemporaryPath(const String & path);
void setFlagsPath(const String & path);

View File

@ -1,6 +1,5 @@
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/AnalyzedJoin.h> /// for getNamesAndTypeListFromTableExpression
#include <Interpreters/Context.h>
#include <Common/typeid_cast.h>
@ -15,6 +14,8 @@ namespace DB
NameSet removeDuplicateColumns(NamesAndTypesList & columns);
struct ASTTableExpression;
NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context);
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database)
{

View File

@ -20,9 +20,6 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
using NameWithAlias = std::pair<std::string, std::string>;
using NamesWithAliases = std::vector<NameWithAlias>;
class AnalyzedJoin;
class IPreparedFunction;

View File

@ -29,6 +29,7 @@
#include <Interpreters/PredicateExpressionsOptimizer.h>
#include <Interpreters/ExternalDictionaries.h>
#include <Interpreters/Set.h>
#include <Interpreters/AnalyzedJoin.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
@ -417,8 +418,7 @@ bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, b
if (!ast_join)
return false;
SubqueryForSet & subquery_for_set = getSubqueryForJoin(*ast_join);
syntax->analyzed_join->setHashJoin(subquery_for_set.join);
makeTableJoin(*ast_join);
initChain(chain, sourceColumns());
ExpressionActionsChain::Step & step = chain.steps.back();
@ -456,7 +456,15 @@ static JoinPtr tryGetStorageJoin(const ASTTablesInSelectQueryElement & join_elem
return {};
}
SubqueryForSet & SelectQueryExpressionAnalyzer::getSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element)
static ExpressionActionsPtr createJoinedBlockActions(const Context & context, const AnalyzedJoin & analyzed_join)
{
ASTPtr expression_list = analyzed_join.rightKeysList();
auto syntax_result = SyntaxAnalyzer(context).analyze(expression_list,
analyzed_join.columnsFromJoinedTable(), analyzed_join.requiredJoinedNames());
return ExpressionAnalyzer(expression_list, syntax_result, context).getActions(true, false);
}
void SelectQueryExpressionAnalyzer::makeTableJoin(const ASTTablesInSelectQueryElement & join_element)
{
/// Two JOINs are not supported with the same subquery, but different USINGs.
auto join_hash = join_element.getTreeHash();
@ -469,64 +477,45 @@ SubqueryForSet & SelectQueryExpressionAnalyzer::getSubqueryForJoin(const ASTTabl
subquery_for_set.join = tryGetStorageJoin(join_element, context);
if (!subquery_for_set.join)
makeHashJoin(join_element, subquery_for_set);
{
/// Actions which need to be calculated on joined block.
ExpressionActionsPtr joined_block_actions = createJoinedBlockActions(context, analyzedJoin());
return subquery_for_set;
if (!subquery_for_set.source)
makeSubqueryForJoin(join_element, joined_block_actions, subquery_for_set);
/// Test actions on sample block (early error detection)
Block sample_block = subquery_for_set.renamedSampleBlock();
joined_block_actions->execute(sample_block);
/// TODO You do not need to set this up when JOIN is only needed on remote servers.
subquery_for_set.join = analyzedJoin().makeHashJoin(sample_block, settings.size_limits_for_join);
subquery_for_set.joined_block_actions = joined_block_actions;
}
syntax->analyzed_join->setHashJoin(subquery_for_set.join);
}
void SelectQueryExpressionAnalyzer::makeHashJoin(const ASTTablesInSelectQueryElement & join_element,
SubqueryForSet & subquery_for_set) const
void SelectQueryExpressionAnalyzer::makeSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element,
const ExpressionActionsPtr & joined_block_actions,
SubqueryForSet & subquery_for_set) const
{
/// Actions which need to be calculated on joined block.
ExpressionActionsPtr joined_block_actions = createJoinedBlockActions();
/** For GLOBAL JOINs (in the case, for example, of the push method for executing GLOBAL subqueries), the following occurs
* - in the addExternalStorage function, the JOIN (SELECT ...) subquery is replaced with JOIN _data1,
* in the subquery_for_set object this subquery is exposed as source and the temporary table _data1 as the `table`.
* - this function shows the expression JOIN _data1.
*/
if (!subquery_for_set.source)
{
ASTPtr table;
auto & table_to_join = join_element.table_expression->as<ASTTableExpression &>();
if (table_to_join.subquery)
table = table_to_join.subquery;
else if (table_to_join.table_function)
table = table_to_join.table_function;
else if (table_to_join.database_and_table_name)
table = table_to_join.database_and_table_name;
NamesWithAliases required_columns_with_aliases =
analyzedJoin().getRequiredColumns(joined_block_actions->getSampleBlock(), joined_block_actions->getRequiredColumns());
Names action_columns = joined_block_actions->getRequiredColumns();
NameSet required_columns(action_columns.begin(), action_columns.end());
Names original_columns;
for (auto & pr : required_columns_with_aliases)
original_columns.push_back(pr.first);
analyzedJoin().appendRequiredColumns(joined_block_actions->getSampleBlock(), required_columns);
auto interpreter = interpretSubquery(join_element.table_expression, context, subquery_depth, original_columns);
auto original_map = analyzedJoin().getOriginalColumnsMap(required_columns);
Names original_columns;
for (auto & pr : original_map)
original_columns.push_back(pr.second);
auto interpreter = interpretSubquery(table, context, subquery_depth, original_columns);
subquery_for_set.makeSource(interpreter, original_map);
}
Block sample_block = subquery_for_set.renamedSampleBlock();
joined_block_actions->execute(sample_block);
/// TODO You do not need to set this up when JOIN is only needed on remote servers.
subquery_for_set.join = analyzedJoin().makeHashJoin(sample_block, settings.size_limits_for_join);
subquery_for_set.joined_block_actions = joined_block_actions;
}
ExpressionActionsPtr SelectQueryExpressionAnalyzer::createJoinedBlockActions() const
{
ASTPtr expression_list = analyzedJoin().rightKeysList();
Names required_columns = analyzedJoin().requiredJoinedNames();
auto syntax_result = SyntaxAnalyzer(context).analyze(expression_list, analyzedJoin().columnsFromJoinedTable(), required_columns);
return ExpressionAnalyzer(expression_list, syntax_result, context).getActions(true, false);
subquery_for_set.makeSource(interpreter, std::move(required_columns_with_aliases));
}
bool SelectQueryExpressionAnalyzer::appendPrewhere(

View File

@ -26,9 +26,6 @@ class ASTExpressionList;
class ASTSelectQuery;
struct ASTTablesInSelectQueryElement;
struct SyntaxAnalyzerResult;
using SyntaxAnalyzerResultPtr = std::shared_ptr<const SyntaxAnalyzerResult>;
/// ExpressionAnalyzer sources, intermediates and results. It splits data and logic, allows to test them separately.
struct ExpressionAnalyzerData
{
@ -222,9 +219,9 @@ private:
*/
void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name);
SubqueryForSet & getSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element);
ExpressionActionsPtr createJoinedBlockActions() const;
void makeHashJoin(const ASTTablesInSelectQueryElement & join_element, SubqueryForSet & subquery_for_set) const;
void makeTableJoin(const ASTTablesInSelectQueryElement & join_element);
void makeSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element, const ExpressionActionsPtr & joined_block_actions,
SubqueryForSet & subquery_for_set) const;
const ASTSelectQuery * getAggregatingQuery() const;
};

View File

@ -45,6 +45,7 @@
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/JoinToSubqueryTransformVisitor.h>
#include <Interpreters/CrossToInnerJoinVisitor.h>
#include <Interpreters/AnalyzedJoin.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
@ -417,8 +418,6 @@ QueryPipeline InterpreterSelectQuery::executeWithProcessors()
Block InterpreterSelectQuery::getSampleBlockImpl()
{
FilterInfoPtr filter_info;
/// Need to create sets before analyzeExpressions(). Otherwise some sets for index won't be created.
query_analyzer->makeSetsForIndex(getSelectQuery().where());
query_analyzer->makeSetsForIndex(getSelectQuery().prewhere());
@ -430,8 +429,9 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
options.to_stage,
context,
storage,
true,
filter_info);
true, // only_types
{} // filter_info
);
if (options.to_stage == QueryProcessingStage::Enum::FetchColumns)
{
@ -990,6 +990,21 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
pipeline.streams.back() = std::make_shared<FilterBlockInputStream>(
pipeline.streams.back(), expressions.prewhere_info->prewhere_actions,
expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->remove_prewhere_column);
// To remove additional columns in dry run
// For example, sample column which can be removed in this stage
if (expressions.prewhere_info->remove_columns_actions)
{
if constexpr (pipeline_with_processors)
{
pipeline.addSimpleTransform([&](const Block & header)
{
return std::make_shared<ExpressionTransform>(header, expressions.prewhere_info->remove_columns_actions);
});
}
else
pipeline.streams.back() = std::make_shared<ExpressionBlockInputStream>(pipeline.streams.back(), expressions.prewhere_info->remove_columns_actions);
}
}
}
else
@ -1561,12 +1576,22 @@ void InterpreterSelectQuery::executeFetchColumns(
streams = {std::make_shared<NullBlockInputStream>(storage->getSampleBlockForColumns(required_columns))};
if (query_info.prewhere_info)
{
streams.back() = std::make_shared<FilterBlockInputStream>(
streams.back(),
prewhere_info->prewhere_actions,
prewhere_info->prewhere_column_name,
prewhere_info->remove_prewhere_column);
// To remove additional columns
// In some cases, we did not read any marks so that the pipeline.streams is empty
// Thus, some columns in prewhere are not removed as expected
// This leads to mismatched header in distributed table
if (query_info.prewhere_info->remove_columns_actions)
{
streams.back() = std::make_shared<ExpressionBlockInputStream>(streams.back(), query_info.prewhere_info->remove_columns_actions);
}
}
}
for (auto & stream : streams)

View File

@ -3,7 +3,6 @@
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/Context.h>
#include <Interpreters/AnalyzedJoin.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSelectQuery.h>

View File

@ -1,5 +1,4 @@
#include <Interpreters/SubqueryForSet.h>
#include <Interpreters/AnalyzedJoin.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <DataStreams/LazyBlockInputStream.h>
@ -7,26 +6,14 @@ namespace DB
{
void SubqueryForSet::makeSource(std::shared_ptr<InterpreterSelectWithUnionQuery> & interpreter,
const std::unordered_map<String, String> & name_to_origin)
NamesWithAliases && joined_block_aliases_)
{
joined_block_aliases = std::move(joined_block_aliases_);
source = std::make_shared<LazyBlockInputStream>(interpreter->getSampleBlock(),
[interpreter]() mutable { return interpreter->execute().in; });
for (const auto & names : name_to_origin)
joined_block_aliases.emplace_back(names.second, names.first);
sample_block = source->getHeader();
for (const auto & name_with_alias : joined_block_aliases)
{
if (sample_block.has(name_with_alias.first))
{
auto pos = sample_block.getPositionByName(name_with_alias.first);
auto column = sample_block.getByPosition(pos);
sample_block.erase(pos);
column.name = name_with_alias.second;
sample_block.insert(std::move(column));
}
}
renameColumns(sample_block);
}
void SubqueryForSet::renameColumns(Block & block)

View File

@ -31,7 +31,7 @@ struct SubqueryForSet
StoragePtr table;
void makeSource(std::shared_ptr<InterpreterSelectWithUnionQuery> & interpreter,
const std::unordered_map<String, String> & name_to_origin);
NamesWithAliases && joined_block_aliases_);
Block renamedSampleBlock() const { return sample_block; }
void renameColumns(Block & block);

View File

@ -17,6 +17,7 @@
#include <Interpreters/OptimizeIfWithConstantConditionVisitor.h>
#include <Interpreters/RequiredSourceColumnsVisitor.h>
#include <Interpreters/GetAggregatesVisitor.h>
#include <Interpreters/AnalyzedJoin.h>
#include <Interpreters/ExpressionActions.h> /// getSmallestColumn()
#include <Parsers/ASTExpressionList.h>

View File

@ -1,7 +1,7 @@
#pragma once
#include <Core/NamesAndTypes.h>
#include <Interpreters/Aliases.h>
#include <Interpreters/AnalyzedJoin.h>
#include <Interpreters/SelectQueryOptions.h>
#include <Storages/IStorage_fwd.h>
@ -11,6 +11,9 @@ namespace DB
NameSet removeDuplicateColumns(NamesAndTypesList & columns);
class ASTFunction;
class AnalyzedJoin;
class Context;
struct SelectQueryOptions;
struct SyntaxAnalyzerResult
{

View File

@ -31,6 +31,7 @@
#include <Common/ProfileEvents.h>
#include <Interpreters/DNSCacheUpdater.h>
#include <Common/SensitiveDataMasker.h>
#include <Processors/Transforms/LimitsCheckingTransform.h>
#include <Processors/Transforms/MaterializingTransform.h>
@ -76,7 +77,7 @@ static String prepareQueryForLogging(const String & query, Context & context)
// wiping sensitive data before cropping query by log_queries_cut_to_length,
// otherwise something like credit card without last digit can go to log
if (auto masker = context.getSensitiveDataMasker())
if (auto masker = SensitiveDataMasker::getInstance())
{
auto matches = masker->wipeSensitiveData(res);
if (matches > 0)

View File

@ -18,6 +18,19 @@ namespace DB
std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
const ASTPtr & table_expression, const Context & context, size_t subquery_depth, const Names & required_source_columns)
{
if (auto * expr = table_expression->as<ASTTableExpression>())
{
ASTPtr table;
if (expr->subquery)
table = expr->subquery;
else if (expr->table_function)
table = expr->table_function;
else if (expr->database_and_table_name)
table = expr->database_and_table_name;
return interpretSubquery(table, context, subquery_depth, required_source_columns);
}
/// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`.
const auto * subquery = table_expression->as<ASTSubquery>();
const auto * function = table_expression->as<ASTFunction>();

View File

@ -7,4 +7,6 @@
6
7
8
9
text_log non empty
finish

View File

@ -11,6 +11,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
cur_name=$(basename "${BASH_SOURCE[0]}")
tmp_file=${CLICKHOUSE_TMP}/$cur_name"_server.logs"
tmp_file2=${CLICKHOUSE_TMP}/$cur_name"_server.2.logs"
rm -f $tmp_file >/dev/null 2>&1
echo 1
@ -55,9 +56,10 @@ grep 'TOPSECRET' $tmp_file && echo 'fail 4b'
echo 5
# run in background
rm -f $tmp_file2 >/dev/null 2>&1
bash -c "$CLICKHOUSE_CLIENT \
--query=\"select sleepEachRow(0.5) from numbers(4) where ignore('find_me_TOPSECRET=TOPSECRET')=0 and ignore('fwerkh_that_magic_string_make_me_unique') = 0 FORMAT Null\" \
--log_queries=1 --ignore-error --multiquery 2>&1 | grep TOPSECRET" &
--log_queries=1 --ignore-error --multiquery >$tmp_file2 2>&1" &
sleep 0.1
@ -67,12 +69,14 @@ rm -f $tmp_file >/dev/null 2>&1
echo '5.1'
# check that executing query doesn't expose secrets in processlist
$CLICKHOUSE_CLIENT --query="SHOW PROCESSLIST" --log_queries=0 >$tmp_file 2>&1
grep 'fwerkh_that_magic_string_make_me_unique' $tmp_file >/dev/null || echo 'fail 5a'
( grep 'fwerkh_that_magic_string_make_me_unique' $tmp_file | grep 'find_me_\[hidden\]' $tmp_file >/dev/null ) || echo 'fail 5b'
grep 'TOPSECRET' $tmp_file && echo 'fail 5c'
wait
grep 'TOPSECRET' $tmp_file2 && echo 'fail 5d'
rm -f $tmp_file2 >/dev/null 2>&1
grep 'fwerkh_that_magic_string_make_me_unique' $tmp_file >$tmp_file2 || echo 'fail 5a'
grep 'find_me_\[hidden\]' $tmp_file2 >/dev/null || echo 'fail 5b'
grep 'TOPSECRET' $tmp_file && echo 'fail 5c'
# instead of disabling send_logs_level=trace (enabled globally for that test) - redir it's output to /dev/null
@ -107,4 +111,15 @@ drop table sensitive;" --log_queries=1 --ignore-error --multiquery >$tmp_file 2>
grep 'find_me_\[hidden\]' $tmp_file >/dev/null || echo 'fail 8a'
grep 'TOPSECRET' $tmp_file && echo 'fail 8b'
echo 'finish'
$CLICKHOUSE_CLIENT --query="SYSTEM FLUSH LOGS" --server_logs_file=/dev/null
sleep 0.1;
echo 9
$CLICKHOUSE_CLIENT \
--server_logs_file=/dev/null \
--query="SELECT if( count() > 0, 'text_log non empty', 'text_log empty') FROM system.text_log WHERE event_time>now() - 60 and message like '%find_me%';
select * from system.text_log where event_time>now() - 60 and message like '%TOPSECRET=TOPSECRET%';" --ignore-error --multiquery
echo 'finish'
rm -f $tmp_file >/dev/null 2>&1
rm -f $tmp_file2 >/dev/null 2>&1

View File

@ -0,0 +1,7 @@
create table if not exists sample_prewhere (date Date, id Int32, time Int64) engine = MergeTree partition by date order by (id, time, intHash64(time)) sample by intHash64(time);
insert into sample_prewhere values ('2019-01-01', 2, toDateTime('2019-07-20 00:00:01'));
insert into sample_prewhere values ('2019-01-01', 1, toDateTime('2019-07-20 00:00:02'));
insert into sample_prewhere values ('2019-01-02', 3, toDateTime('2019-07-20 00:00:03'));
select id from remote('127.0.0.{1,3}', currentDatabase(), sample_prewhere) SAMPLE 1 where toDateTime(time) = '2019-07-20 00:00:00';

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (19.13.1.1) unstable; urgency=low
clickhouse (19.15.1.1) unstable; urgency=low
* Modified source code
-- clickhouse-release <clickhouse-release@yandex-team.ru> Tue, 23 Jul 2019 11:20:49 +0300
-- clickhouse-release <clickhouse-release@yandex-team.ru> Fri, 06 Sep 2019 17:58:30 +0300

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
ARG version=19.13.1.*
ARG version=19.15.1.*
RUN apt-get update \
&& apt-get install --yes --no-install-recommends \

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
ARG version=19.13.1.*
ARG version=19.15.1.*
ARG gosu_ver=1.10
RUN apt-get update \

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
ARG version=19.13.1.*
ARG version=19.15.1.*
RUN apt-get update && \
apt-get install -y apt-transport-https dirmngr && \

View File

@ -40,7 +40,7 @@ sudo apt-get install git cmake ninja-build
Or cmake3 instead of cmake on older systems.
## Install GCC 8
## Install GCC 9
There are several ways to do this.
@ -50,18 +50,18 @@ There are several ways to do this.
sudo apt-get install software-properties-common
sudo apt-add-repository ppa:ubuntu-toolchain-r/test
sudo apt-get update
sudo apt-get install gcc-8 g++-8
sudo apt-get install gcc-9 g++-9
```
### Install from Sources
Look at [utils/ci/build-gcc-from-sources.sh](https://github.com/yandex/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh)
## Use GCC 8 for Builds
## Use GCC 9 for Builds
```bash
export CC=gcc-8
export CXX=g++-8
export CC=gcc-9
export CXX=g++-9
```
## Install Required Libraries from Packages

View File

@ -11,7 +11,7 @@ The supported formats are:
| [TabSeparatedRaw](#tabseparatedraw) | ✗ | ✔ |
| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ |
| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ |
| [Template](#template) | ✔ | ✔ |
| [Template](#format-template) | ✔ | ✔ |
| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ |
| [CSV](#csv) | ✔ | ✔ |
| [CSVWithNames](#csvwithnames) | ✔ | ✔ |
@ -121,7 +121,7 @@ During parsing, the first and second rows are completely ignored.
This format is also available under the name `TSVWithNamesAndTypes`.
## Template {#template}
## Template {#format-template}
This format allows to specify a custom format string with placeholders for values with specified escaping rule.

View File

@ -1,10 +1,11 @@
# Configuration Files {#configuration_files}
The main server config file is `config.xml`. It resides in the `/etc/clickhouse-server/` directory.
ClickHouse supports multi-file configuration management. The main server configuration file is `/etc/clickhouse-server/config.xml`. Other files must be in the `/etc/clickhouse-server/config.d` directory.
Individual settings can be overridden in the `*.xml` and `*.conf` files in the `config.d` directory next to the config file.
!!! note "Note"
All the configuration files should be in XML format. Also, they should have the same root element, usually `<yandex>`.
The `replace` or `remove` attributes can be specified for the elements of these config files.
Some settings specified in the main configuration file can be overridden in other configuration files. The `replace` or `remove` attributes can be specified for the elements of these configuration files.
If neither is specified, it combines the contents of elements recursively, replacing values of duplicate children.

View File

@ -74,7 +74,7 @@ If `force_primary_key=1`, ClickHouse checks to see if the query has a primary ke
## format_schema
This parameter is useful when you are using formats that require a schema definition, such as [Cap'n Proto](https://capnproto.org/), [Protobuf](https://developers.google.com/protocol-buffers/) or [Template](https://clickhouse.yandex/docs/en/interfaces/formats/#template). The value depends on the format.
This parameter is useful when you are using formats that require a schema definition, such as [Cap'n Proto](https://capnproto.org/), [Protobuf](https://developers.google.com/protocol-buffers/) or [Template](../../interfaces/formats.md#format-template). The value depends on the format.
## fsync_metadata

View File

@ -313,6 +313,48 @@ INDEX sample_index2 (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100
INDEX sample_index3 (lower(str), str) TYPE ngrambf_v1(3, 256, 2, 0) GRANULARITY 4
```
#### Functions Support
Conditions in the `WHERE` clause contain calls of functions over the columns. If the column is a part of some index, ClickHouse tries to use this index when performing the functions. ClickHouse supports different subset of functions for using indexes.
The `set` index can be used with all functions. Functions subsets for other indexes are in the table below.
Function (operator) / Index | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter
----------------------------|-------------|--------|------------|------------|---------------
[equals (=, ==)](../../query_language/functions/comparison_functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔
[notEquals(!=, <>)](../../query_language/functions/comparison_functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔
[like](../../query_language/functions/string_search_functions.md#function-like) | ✔ | ✔ | ✔ | ✗ | ✗
[notLike](../../query_language/functions/string_search_functions.md#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗
[startsWith](../../query_language/functions/string_functions.md#function-startswith) | ✔ | ✔ | ✔ | ✔ | ✗
[endsWith](../../query_language/functions/string_functions.md#function-endswith) | ✗ | ✗ | ✔ | ✔ |
[multiSearchAny](../../query_language/functions/string_search_functions.md#function-multisearchany) | ✗ | ✗ | ✔ | ✔ | ✗
[in](../../query_language/functions/in_functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔
[notIn](../../query_language/functions/in_functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔
[less (<)](../../query_language/functions/comparison_functions.md#function-less) | ✔ | ✔ | ✗ | ✗ | ✗
[greater (>)](../../query_language/functions/comparison_functions.md#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗
[lessOrEquals (<=)](../../query_language/functions/comparison_functions.md#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗
[greaterOrEquals (>=)](../../query_language/functions/comparison_functions.md#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗
[empty](../../query_language/functions/array_functions.md#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗
[notEmpty](../../query_language/functions/array_functions.md#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗
hasToken | ✗ | ✗ | ✗ | ✔ | ✗
Functions with a constant argument less than ngram size couldn't be used by `ngrambf_v1` for the query optimization.
Bloom filters can have false positive matches, so the `ngrambf_v1`, `tokenbf_v1`, `bloom_filter` indexes couldn't be used for optimizing queries where the result of a function is expected to be false, for example:
- Can be optimized:
- `s LIKE '%test%'`
- `NOT s NOT LIKE '%test%'`
- `s = 1`
- `NOT s != 1`
- `startsWith(s, 'test')`
- Can't be optimized:
- `NOT s LIKE '%test%'`
- `s NOT LIKE '%test%'`
- `NOT s = 1`
- `s != 1`
- `NOT startsWith(s, 'test')`
## Concurrent Data Access
For concurrent table access, we use multi-versioning. In other words, when a table is simultaneously read and updated, data is read from a set of parts that is current at the time of the query. There are no lengthy locks. Inserts do not get in the way of read operations.

View File

@ -1,12 +1,12 @@
# Functions for working with arrays
## empty
## empty {#function-empty}
Returns 1 for an empty array, or 0 for a non-empty array.
The result type is UInt8.
The function also works for strings.
## notEmpty
## notEmpty {#function-notempty}
Returns 0 for an empty array, or 1 for a non-empty array.
The result type is UInt8.
@ -73,7 +73,7 @@ Get the element with the index `n` from the array `arr`. `n` must be any integer
Indexes in an array begin from one.
Negative indexes are supported. In this case, it selects the corresponding element numbered from the end. For example, `arr[-1]` is the last item in the array.
If the index falls outside of the bounds of an array, it returns some default value (0 for numbers, an empty string for strings, etc.).
If the index falls outside of the bounds of an array, it returns some default value (0 for numbers, an empty string for strings, etc.), except for the case with a non-constant array and a constant index 0 (in this case there will be an error `Array indices are 1-based`).
## has(arr, elem)

View File

@ -17,17 +17,17 @@ Strings are compared by bytes. A shorter string is smaller than all strings that
Note. Up until version 1.1.54134, signed and unsigned numbers were compared the same way as in C++. In other words, you could get an incorrect result in cases like SELECT 9223372036854775807 &gt; -1. This behavior changed in version 1.1.54134 and is now mathematically correct.
## equals, a = b and a == b operator
## equals, a = b and a == b operator {#function-equals}
## notEquals, a ! operator= b and a `<>` b
## notEquals, a ! operator= b and a `<>` b {#function-notequals}
## less, `< operator`
## less, `< operator` {#function-less}
## greater, `> operator`
## greater, `> operator` {#function-greater}
## lessOrEquals, `<= operator`
## lessOrEquals, `<= operator` {#function-lessorequals}
## greaterOrEquals, `>= operator`
## greaterOrEquals, `>= operator` {#function-greaterorequals}
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/comparison_functions/) <!--hide-->

View File

@ -1,6 +1,6 @@
# Functions for implementing the IN operator
## in, notIn, globalIn, globalNotIn
## in, notIn, globalIn, globalNotIn {#in-functions}
See the section [IN operators](../select.md#select-in-operators).

View File

@ -145,11 +145,11 @@ Decode base64-encoded string 's' into original string. In case of failure raises
## tryBase64Decode(s)
Similar to base64Decode, but in case of error an empty string would be returned.
## endsWith(s, suffix)
## endsWith(s, suffix) {#function-endswith}
Returns whether to end with the specified suffix. Returns 1 if the string ends with the specified suffix, otherwise it returns 0.
## startsWith(s, prefix)
## startsWith(s, prefix) {#function-startswith}
Returns whether to start with the specified prefix. Returns 1 if the string starts with the specified prefix, otherwise it returns 0.

View File

@ -33,7 +33,7 @@ Returns the index `i` (starting from 1) of the leftmost found needle<sub>i</sub>
For a case-insensitive search or/and in UTF-8 format use functions `multiSearchFirstIndexCaseInsensitive, multiSearchFirstIndexUTF8, multiSearchFirstIndexCaseInsensitiveUTF8`.
## multiSearchAny(haystack, [needle<sub>1</sub>, needle<sub>2</sub>, ..., needle<sub>n</sub>])
## multiSearchAny(haystack, [needle<sub>1</sub>, needle<sub>2</sub>, ..., needle<sub>n</sub>]) {#function-multisearchany}
Returns 1, if at least one string needle<sub>i</sub> matches the string `haystack` and 0 otherwise.
@ -86,7 +86,7 @@ Extracts a fragment of a string using a regular expression. If 'haystack' doesn'
Extracts all the fragments of a string using a regular expression. If 'haystack' doesn't match the 'pattern' regex, an empty string is returned. Returns an array of strings consisting of all matches to the regex. In general, the behavior is the same as the 'extract' function (it takes the first subpattern, or the entire expression if there isn't a subpattern).
## like(haystack, pattern), haystack LIKE pattern operator
## like(haystack, pattern), haystack LIKE pattern operator {#function-like}
Checks whether a string matches a simple regular expression.
The regular expression can contain the metasymbols `%` and `_`.
@ -100,7 +100,7 @@ Use the backslash (`\`) for escaping metasymbols. See the note on escaping in th
For regular expressions like `%needle%`, the code is more optimal and works as fast as the `position` function.
For other regular expressions, the code is the same as for the 'match' function.
## notLike(haystack, pattern), haystack NOT LIKE pattern operator
## notLike(haystack, pattern), haystack NOT LIKE pattern operator {#function-notlike}
The same thing as 'like', but negative.

View File

@ -12,6 +12,8 @@ Format | INSERT | SELECT
[TabSeparatedRaw](formats.md#tabseparatedraw) | ✗ | ✔ |
[TabSeparatedWithNames](formats.md#tabseparatedwithnames) | ✔ | ✔ |
[TabSeparatedWithNamesAndTypes](formats.md#tabseparatedwithnamesandtypes) | ✔ | ✔ |
[Template](#format-template) | ✔ | ✔ |
[TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ |
[CSV](formats.md#csv) | ✔ | ✔ |
[CSVWithNames](formats.md#csvwithnames) | ✔ | ✔ |
[Values](formats.md#data-format-values) | ✔ | ✔ |
@ -115,6 +117,122 @@ SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORD
همچنین این فرمت تحت عنوان ` TSVWithNamesAndTypes`وجود دارد.
## Template {#format-template}
This format allows to specify a custom format string with placeholders for values with specified escaping rule.
It uses settings `format_schema`, `format_schema_rows`, `format_schema_rows_between_delimiter` and some settings of other formats (e.g. `output_format_json_quote_64bit_integers` when using `JSON` escaping, see further)
Format string `format_schema_rows` specifies rows format with the following syntax:
`delimiter_1${column_1:serializeAs_1}delimiter_2${column_2:serializeAs_2} ... delimiter_N`,
where `delimiter_i` is a delimiter between values (`$` symbol can be escaped as `$$`),
`column_i` is a name of a column whose values are to be selected or inserted (if empty, then column will be skipped),
`serializeAs_i` is an escaping rule for the column values. The following escaping rules are supported:
- `CSV`, `JSON`, `XML` (similarly to the formats of the same names)
- `Escaped` (similarly to `TSV`)
- `Quoted` (similarly to `Values`)
- `Raw` (without escaping, similarly to `TSVRaw`)
- `None` (no escaping rule, see further)
If escaping rule is omitted, then`None` will be used. `XML` and `Raw` are suitable only for output.
So, for the following format string:
`Search phrase: ${SearchPhrase:Quoted}, count: ${c:Escaped}, ad price: $$${price:JSON};`
the values of `SearchPhrase`, `c` and `price` columns, which are escaped as `Quoted`, `Escaped` and `JSON` will be printed (for select) or will be expected (for insert) between `Search phrase: `, `, count: `, `, ad price: $` and `;` delimiters respectively. For example:
`Search phrase: 'bathroom interior design', count: 2166, ad price: $3;`
The `format_schema_rows_between_delimiter` setting specifies delimiter between rows, which is printed (or expected) after every row except the last one (`\n` by default)
Format string `format_schema` has the same syntax as `format_schema_rows` and allows to specify a prefix, a suffix and a way to print some additional information. It contains the following placeholders instead of column names:
- `data` is the rows with data in `format_schema_rows` format, separated by `format_schema_rows_between_delimiter`. This placeholder must be the first placeholder in the format string.
- `totals` is the row with total values in `format_schema_rows` format (when using WITH TOTALS)
- `min` is the row with minimum values in `format_schema_rows` format (when extremes is set to 1)
- `max` is the row with maximum values in `format_schema_rows` format (when extremes is set to 1)
- `rows` is the total number of output rows
- `rows_before_limit` is the minimal number of rows there would have been without LIMIT. Output only if the query contains LIMIT. If the query contains GROUP BY, rows_before_limit_at_least is the exact number of rows there would have been without a LIMIT.
- `time` is the request execution time in seconds
- `rows_read` is the number of rows have been read
- `bytes_read` is the number of bytes (uncompressed) have been read
The placeholders `data`, `totals`, `min` and `max` must not have escaping rule specified (or `None` must be specified explicitly). The remaining placeholders may have any escaping rule specified.
If the `format_schema` setting is an empty string, `${data}` is used as default value.
For insert queries format allows to skip some columns or some fields if prefix or suffix (see example).
`Select` example:
```sql
SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase ORDER BY c DESC LIMIT 5
FORMAT Template
SETTINGS format_schema = '<!DOCTYPE HTML>
<html> <head> <title>Search phrases</title> </head>
<body>
<table border="1"> <caption>Search phrases</caption>
<tr> <th>Search phrase</th> <th>Count</th> </tr>
${data}
</table>
<table border="1"> <caption>Max</caption>
${max}
</table>
<b>Processed ${rows_read:XML} rows in ${time:XML} sec</b>
</body>
</html>',
format_schema_rows = '<tr> <td>${SearchPhrase:XML}</td> <td>${с:XML}</td> </tr>',
format_schema_rows_between_delimiter = '\n '
```
```html
<!DOCTYPE HTML>
<html> <head> <title>Search phrases</title> </head>
<body>
<table border="1"> <caption>Search phrases</caption>
<tr> <th>Search phrase</th> <th>Count</th> </tr>
<tr> <td></td> <td>8267016</td> </tr>
<tr> <td>bathroom interior design</td> <td>2166</td> </tr>
<tr> <td>yandex</td> <td>1655</td> </tr>
<tr> <td>spring 2014 fashion</td> <td>1549</td> </tr>
<tr> <td>freeform photos</td> <td>1480</td> </tr>
</table>
<table border="1"> <caption>Max</caption>
<tr> <td></td> <td>8873898</td> </tr>
</table>
<b>Processed 3095973 rows in 0.1569913 sec</b>
</body>
</html>
```
`Insert` example:
```
Some header
Page views: 5, User id: 4324182021466249494, Useless field: hello, Duration: 146, Sign: -1
Page views: 6, User id: 4324182021466249494, Useless field: world, Duration: 185, Sign: 1
Total rows: 2
```
```sql
INSERT INTO UserActivity FORMAT Template SETTINGS
format_schema = 'Some header\n${data}\nTotal rows: ${:CSV}\n',
format_schema_rows = 'Page views: ${PageViews:CSV}, User id: ${UserID:CSV}, Useless field: ${:CSV}, Duration: ${Duration:CSV}, Sign: ${Sign:CSV}'
```
`PageViews`, `UserID`, `Duration` and `Sign` inside placeholders are names of columns in the table. Values after `Useless field` in rows and after `\nTotal rows: ` in suffix will be ignored.
All delimiters in the input data must be strictly equal to delimiters in specified format strings.
## TemplateIgnoreSpaces {#templateignorespaces}
This format is suitable only for input.
Similar to `Template`, but skips whitespace characters between delimiters and values in the input stream. However, if format strings contain whitespace characters, these characters will be expected in the input stream. Also allows to specify empty placeholders (`${}` or `${:None}`) to split some delimiter into separate parts to ignore spaces between them. Such placeholders are used only for skipping whitespace characters.
It's possible to read `JSON` using this format, if values of columns have the same order in all rows. For example, the following request can be used for inserting data from output example of format [JSON](#json):
```sql
INSERT INTO table_name FORMAT TemplateIgnoreSpaces SETTINGS
format_schema = '{${}"meta"${}:${:JSON},${}"data"${}:${}[${data}]${},${}"totals"${}:${:JSON},${}"extremes"${}:${:JSON},${}"rows"${}:${:JSON},${}"rows_before_limit_at_least"${}:${:JSON}${}}',
format_schema_rows = '{${}"SearchPhrase"${}:${}${phrase:JSON}${},${}"c"${}:${}${cnt:JSON}${}}',
format_schema_rows_between_delimiter = ','
```
## TSKV
مشابه فرمت TabSeparated، اما خروجی به صورت name=value می باشد. نام ها مشابه روش TabSeparated، escape می شوند، و همچنین = symbol هم escape می شود.

View File

@ -10,7 +10,7 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT
| [TabSeparatedRaw](#tabseparatedraw) | ✗ | ✔ |
| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ |
| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ |
| [Template](#template) | ✔ | ✔ |
| [Template](#format-template) | ✔ | ✔ |
| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ |
| [CSV](#csv) | ✔ | ✔ |
| [CSVWithNames](#csvwithnames) | ✔ | ✔ |
@ -120,7 +120,7 @@ world
Этот формат также доступен под именем `TSVWithNamesAndTypes`.
## Template {#template}
## Template {#format-template}
Этот формат позволяет указать произвольную форматную строку, в которую подставляются значения, сериализованные выбранным способом.

View File

@ -72,7 +72,7 @@ ClickHouse применяет настройку в тех случаях, ко
## format_schema
Параметр применяется в том случае, когда используются форматы, требующие определения схемы, например [Cap'n Proto](https://capnproto.org/), [Protobuf](https://developers.google.com/protocol-buffers/) или [Template](../../interfaces/formats.md#template). Значение параметра зависит от формата.
Параметр применяется в том случае, когда используются форматы, требующие определения схемы, например [Cap'n Proto](https://capnproto.org/), [Protobuf](https://developers.google.com/protocol-buffers/) или [Template](../../interfaces/formats.md#format-template). Значение параметра зависит от формата.
## fsync_metadata

View File

@ -74,7 +74,7 @@ SELECT arrayConcat([1, 2], [3, 4], [5, 6]) AS res
Индексы в массиве начинаются с единицы.
Поддерживаются отрицательные индексы. В этом случае, будет выбран соответствующий по номеру элемент с конца. Например, arr\[-1\] - последний элемент массива.
Если индекс выходит за границы массива, то возвращается некоторое значение по умолчанию (0 для чисел, пустая строка для строк и т. п.).
Если индекс выходит за границы массива, то возвращается некоторое значение по умолчанию (0 для чисел, пустая строка для строк и т. п.), кроме случая с неконстантным массивом и константным индексом 0 (в этом случае будет ошибка `Array indices are 1-based`).
## has(arr, elem)

View File

@ -18,16 +18,16 @@
Замечание. До версии 1.1.54134 сравнение знаковых и беззнаковых целых чисел производилось также, как в C++. То есть, вы могли получить неверный результат в таких случаях: SELECT 9223372036854775807 &gt; -1. С версии 1.1.54134 поведение изменилось и стало математически корректным.
## equals, оператор a = b и a == b
## equals, оператор a = b и a == b {#function-equals}
## notEquals, оператор a != b и a `<>` b
## notEquals, оператор a != b и a `<>` b {#function-notequals}
## less, оператор `<`
## less, оператор `<` {#function-less}
## greater, оператор `>`
## greater, оператор `>` {#function-greater}
## lessOrEquals, оператор `<=`
## lessOrEquals, оператор `<=` {#function-lessorequals}
## greaterOrEquals, оператор `>=`
## greaterOrEquals, оператор `>=` {#function-greaterorequals}
[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/comparison_functions/) <!--hide-->

View File

@ -1,6 +1,7 @@
# Функции для реализации оператора IN.
## in, notIn, globalIn, globalNotIn
## in, notIn, globalIn, globalNotIn {#in-functions}
Смотрите раздел [Операторы IN](../select.md#select-in-operators).
## tuple(x, y, ...), оператор (x, y, ...)

View File

@ -117,6 +117,14 @@ SELECT format('{} {}', 'Hello', 'World')
## tryBase64Decode(s)
Функционал аналогичен base64Decode, но при невозможности декодирования возвращает пустую строку.
## endsWith(s, suffix) {#function-endswith}
Возвращает 1, если строка завершается указанным суффиксом, и 0 в противном случае.
## startsWith(s, prefix) {#function-startswith}
Возвращает 1, если строка начинается указанным префиксом, и 0 в противном случае.
## CRC32(s)
Возвращает чексумму CRC32 данной строки.
Тип результата - UInt32.

View File

@ -29,7 +29,7 @@
Для поиска без учета регистра и/или в кодировке UTF-8 используйте функции `multiSearchFirstIndexCaseInsensitive, multiSearchFirstIndexUTF8, multiSearchFirstIndexCaseInsensitiveUTF8`.
## multiSearchAny(haystack, [needle<sub>1</sub>, needle<sub>2</sub>, ..., needle<sub>n</sub>])
## multiSearchAny(haystack, [needle<sub>1</sub>, needle<sub>2</sub>, ..., needle<sub>n</sub>]) {#function-multisearchany}
Возвращает 1, если хотя бы одна подстрока needle<sub>i</sub> нашлась в строке `haystack` и 0 иначе.
Для поиска без учета регистра и/или в кодировке UTF-8 используйте функции `multiSearchAnyCaseInsensitive, multiSearchAnyUTF8, multiSearchAnyCaseInsensitiveUTF8`.
@ -77,7 +77,7 @@
## extractAll(haystack, pattern)
Извлечение всех фрагментов строки по регулярному выражению. Если haystack не соответствует регулярному выражению pattern, то возвращается пустая строка. Возвращается массив строк, состоящий из всех соответствий регулярному выражению. В остальном, поведение аналогично функции extract (по прежнему, вынимается первый subpattern, или всё выражение, если subpattern-а нет).
## like(haystack, pattern), оператор haystack LIKE pattern
## like(haystack, pattern), оператор haystack LIKE pattern {#function-like}
Проверка строки на соответствие простому регулярному выражению.
Регулярное выражение может содержать метасимволы `%` и `_`.
@ -90,7 +90,7 @@
Для регулярных выражений вида `%needle%` действует более оптимальный код, который работает также быстро, как функция `position`.
Для остальных регулярных выражений, код аналогичен функции match.
## notLike(haystack, pattern), оператор haystack NOT LIKE pattern
## notLike(haystack, pattern), оператор haystack NOT LIKE pattern {#function-notlike}
То же, что like, но с отрицанием.
## ngramDistance(haystack, needle)

View File

@ -10,6 +10,8 @@ ClickHouse 可以接受多种数据格式,可以在 (`INSERT`) 以及 (`SELECT
| [TabSeparatedRaw](#tabseparatedraw) | ✗ | ✔ |
| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ |
| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ |
| [Template](#format-template) | ✔ | ✔ |
| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ |
| [CSV](#csv) | ✔ | ✔ |
| [CSVWithNames](#csvwithnames) | ✔ | ✔ |
| [Values](#data-format-values) | ✔ | ✔ |
@ -115,6 +117,121 @@ world
这种格式也可以使用名称 ` TSVWithNamesAndTypes` 来表示。
## Template {#format-template}
This format allows to specify a custom format string with placeholders for values with specified escaping rule.
It uses settings `format_schema`, `format_schema_rows`, `format_schema_rows_between_delimiter` and some settings of other formats (e.g. `output_format_json_quote_64bit_integers` when using `JSON` escaping, see further)
Format string `format_schema_rows` specifies rows format with the following syntax:
`delimiter_1${column_1:serializeAs_1}delimiter_2${column_2:serializeAs_2} ... delimiter_N`,
where `delimiter_i` is a delimiter between values (`$` symbol can be escaped as `$$`),
`column_i` is a name of a column whose values are to be selected or inserted (if empty, then column will be skipped),
`serializeAs_i` is an escaping rule for the column values. The following escaping rules are supported:
- `CSV`, `JSON`, `XML` (similarly to the formats of the same names)
- `Escaped` (similarly to `TSV`)
- `Quoted` (similarly to `Values`)
- `Raw` (without escaping, similarly to `TSVRaw`)
- `None` (no escaping rule, see further)
If escaping rule is omitted, then`None` will be used. `XML` and `Raw` are suitable only for output.
So, for the following format string:
`Search phrase: ${SearchPhrase:Quoted}, count: ${c:Escaped}, ad price: $$${price:JSON};`
the values of `SearchPhrase`, `c` and `price` columns, which are escaped as `Quoted`, `Escaped` and `JSON` will be printed (for select) or will be expected (for insert) between `Search phrase: `, `, count: `, `, ad price: $` and `;` delimiters respectively. For example:
`Search phrase: 'bathroom interior design', count: 2166, ad price: $3;`
The `format_schema_rows_between_delimiter` setting specifies delimiter between rows, which is printed (or expected) after every row except the last one (`\n` by default)
Format string `format_schema` has the same syntax as `format_schema_rows` and allows to specify a prefix, a suffix and a way to print some additional information. It contains the following placeholders instead of column names:
- `data` is the rows with data in `format_schema_rows` format, separated by `format_schema_rows_between_delimiter`. This placeholder must be the first placeholder in the format string.
- `totals` is the row with total values in `format_schema_rows` format (when using WITH TOTALS)
- `min` is the row with minimum values in `format_schema_rows` format (when extremes is set to 1)
- `max` is the row with maximum values in `format_schema_rows` format (when extremes is set to 1)
- `rows` is the total number of output rows
- `rows_before_limit` is the minimal number of rows there would have been without LIMIT. Output only if the query contains LIMIT. If the query contains GROUP BY, rows_before_limit_at_least is the exact number of rows there would have been without a LIMIT.
- `time` is the request execution time in seconds
- `rows_read` is the number of rows have been read
- `bytes_read` is the number of bytes (uncompressed) have been read
The placeholders `data`, `totals`, `min` and `max` must not have escaping rule specified (or `None` must be specified explicitly). The remaining placeholders may have any escaping rule specified.
If the `format_schema` setting is an empty string, `${data}` is used as default value.
For insert queries format allows to skip some columns or some fields if prefix or suffix (see example).
`Select` example:
```sql
SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase ORDER BY c DESC LIMIT 5
FORMAT Template
SETTINGS format_schema = '<!DOCTYPE HTML>
<html> <head> <title>Search phrases</title> </head>
<body>
<table border="1"> <caption>Search phrases</caption>
<tr> <th>Search phrase</th> <th>Count</th> </tr>
${data}
</table>
<table border="1"> <caption>Max</caption>
${max}
</table>
<b>Processed ${rows_read:XML} rows in ${time:XML} sec</b>
</body>
</html>',
format_schema_rows = '<tr> <td>${SearchPhrase:XML}</td> <td>${с:XML}</td> </tr>',
format_schema_rows_between_delimiter = '\n '
```
```html
<!DOCTYPE HTML>
<html> <head> <title>Search phrases</title> </head>
<body>
<table border="1"> <caption>Search phrases</caption>
<tr> <th>Search phrase</th> <th>Count</th> </tr>
<tr> <td></td> <td>8267016</td> </tr>
<tr> <td>bathroom interior design</td> <td>2166</td> </tr>
<tr> <td>yandex</td> <td>1655</td> </tr>
<tr> <td>spring 2014 fashion</td> <td>1549</td> </tr>
<tr> <td>freeform photos</td> <td>1480</td> </tr>
</table>
<table border="1"> <caption>Max</caption>
<tr> <td></td> <td>8873898</td> </tr>
</table>
<b>Processed 3095973 rows in 0.1569913 sec</b>
</body>
</html>
```
`Insert` example:
```
Some header
Page views: 5, User id: 4324182021466249494, Useless field: hello, Duration: 146, Sign: -1
Page views: 6, User id: 4324182021466249494, Useless field: world, Duration: 185, Sign: 1
Total rows: 2
```
```sql
INSERT INTO UserActivity FORMAT Template SETTINGS
format_schema = 'Some header\n${data}\nTotal rows: ${:CSV}\n',
format_schema_rows = 'Page views: ${PageViews:CSV}, User id: ${UserID:CSV}, Useless field: ${:CSV}, Duration: ${Duration:CSV}, Sign: ${Sign:CSV}'
```
`PageViews`, `UserID`, `Duration` and `Sign` inside placeholders are names of columns in the table. Values after `Useless field` in rows and after `\nTotal rows: ` in suffix will be ignored.
All delimiters in the input data must be strictly equal to delimiters in specified format strings.
## TemplateIgnoreSpaces {#templateignorespaces}
This format is suitable only for input.
Similar to `Template`, but skips whitespace characters between delimiters and values in the input stream. However, if format strings contain whitespace characters, these characters will be expected in the input stream. Also allows to specify empty placeholders (`${}` or `${:None}`) to split some delimiter into separate parts to ignore spaces between them. Such placeholders are used only for skipping whitespace characters.
It's possible to read `JSON` using this format, if values of columns have the same order in all rows. For example, the following request can be used for inserting data from output example of format [JSON](#json):
```sql
INSERT INTO table_name FORMAT TemplateIgnoreSpaces SETTINGS
format_schema = '{${}"meta"${}:${:JSON},${}"data"${}:${}[${data}]${},${}"totals"${}:${:JSON},${}"extremes"${}:${:JSON},${}"rows"${}:${:JSON},${}"rows_before_limit_at_least"${}:${:JSON}${}}',
format_schema_rows = '{${}"SearchPhrase"${}:${}${phrase:JSON}${},${}"c"${}:${}${cnt:JSON}${}}',
format_schema_rows_between_delimiter = ','
```
## TSKV {#tskv}
`TabSeparated` 格式类似,但它输出的是 `name=value` 的格式。名称会和 `TabSeparated` 格式一样被转义,`=` 字符也会被转义。

View File

@ -167,14 +167,6 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
logger.root().get(level).setLevel(config.getString("logger.levels." + level, "trace"));
}
void Loggers::setLoggerSensitiveDataMasker(Poco::Logger & logger, DB::SensitiveDataMasker * sensitive_data_masker)
{
if (auto split = dynamic_cast<DB::OwnSplitChannel *>(logger.getChannel()))
{
split->setMasker(sensitive_data_masker);
}
}
void Loggers::closeLogs(Poco::Logger & logger)
{
if (log_file)

View File

@ -11,18 +11,10 @@ namespace Poco::Util
class AbstractConfiguration;
}
namespace DB
{
class SensitiveDataMasker;
}
class Loggers
{
public:
void buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger, const std::string & cmd_name = "");
void setLoggerSensitiveDataMasker(Poco::Logger & logger, DB::SensitiveDataMasker * sensitive_data_masker);
/// Close log files. On next log write files will be reopened.
void closeLogs(Poco::Logger & logger);

View File

@ -9,7 +9,7 @@
#include <Common/CurrentThread.h>
#include <Common/DNSResolver.h>
#include <common/getThreadNumber.h>
#include <Common/SensitiveDataMasker.h>
namespace DB
{
@ -20,7 +20,7 @@ void OwnSplitChannel::log(const Poco::Message & msg)
if (channels.empty() && (logs_queue == nullptr || msg.getPriority() > logs_queue->max_priority))
return;
if (auto masker = sensitive_data_masker.load())
if (auto masker = SensitiveDataMasker::getInstance())
{
auto message_text = msg.getText();
auto matches = masker->wipeSensitiveData(message_text);
@ -31,6 +31,7 @@ void OwnSplitChannel::log(const Poco::Message & msg)
}
}
logSplit(msg);
}
@ -100,11 +101,6 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg)
}
void OwnSplitChannel::setMasker(DB::SensitiveDataMasker * _sensitive_data_masker)
{
sensitive_data_masker.store(_sensitive_data_masker);
}
void OwnSplitChannel::addChannel(Poco::AutoPtr<Poco::Channel> channel)
{
channels.emplace_back(std::move(channel), dynamic_cast<ExtendedLogChannel *>(channel.get()));

View File

@ -4,7 +4,6 @@
#include <Poco/AutoPtr.h>
#include <Poco/Channel.h>
#include "ExtendedLogChannel.h"
#include <Common/SensitiveDataMasker.h>
#include <Interpreters/TextLog.h>
@ -18,9 +17,6 @@ class OwnSplitChannel : public Poco::Channel
public:
/// Makes an extended message from msg and passes it to the client logs queue and child (if possible)
void log(const Poco::Message & msg) override;
void setMasker(DB::SensitiveDataMasker * _sensitive_data_masker);
/// Adds a child channel
void addChannel(Poco::AutoPtr<Poco::Channel> channel);
@ -33,9 +29,9 @@ private:
/// Handler and its pointer casted to extended interface
using ExtendedChannelPtrPair = std::pair<ChannelPtr, ExtendedLogChannel *>;
std::vector<ExtendedChannelPtrPair> channels;
std::atomic<DB::SensitiveDataMasker *> sensitive_data_masker = nullptr; // global context owns that object, pointer should be reset before context destroying.
std::mutex text_log_mutex;
std::weak_ptr<DB::TextLog> text_log;
};

View File

@ -47,14 +47,14 @@ parser.add_argument('--login', type=str,
args = parser.parse_args()
github = query.Query(args.token, 50)
github = query.Query(args.token, 30)
repo = local.Local(args.repo, args.remote, github.get_default_branch())
stables = repo.get_stables()[-args.number:] # [(branch name, base)]
if not stables:
sys.exit('No stable branches found!')
sys.exit('No release branches found!')
else:
print('Found stable branches:')
print('Found release branches:')
for stable in stables:
print(f'{CHECK_MARK} {stable[0]} forked from {stable[1]}')
@ -171,3 +171,8 @@ print(f'{CHECK_MARK} - good')
print(f'{CROSS_MARK} - bad')
print(f'{LABEL_MARK} - backport is detected via label')
print(f'{CLOCK_MARK} - backport is waiting to merge')
# print API costs
print('\nGitHub API total costs per query:')
for name, value in github.api_costs.items():
print(f'{name} : {value}')

View File

@ -11,9 +11,9 @@ class Query:
self._token = token
self._max_page_size = max_page_size
self._min_page_size = min_page_size
self.api_costs = {}
_MEMBERS = '''
{{
organization(login: "{organization}") {{
team(slug: "{team}") {{
members(first: {max_page_size} {next}) {{
@ -27,7 +27,6 @@ class Query:
}}
}}
}}
}}
'''
def get_members(self, organization, team):
'''Get all team members for organization
@ -58,7 +57,6 @@ class Query:
return logins
_LABELS = '''
{{
repository(owner: "yandex" name: "ClickHouse") {{
pullRequest(number: {number}) {{
labels(first: {max_page_size} {next}) {{
@ -73,7 +71,6 @@ class Query:
}}
}}
}}
}}
'''
def get_labels(self, pull_request):
'''Fetchs all labels for given pull-request
@ -102,7 +99,6 @@ class Query:
return labels
_TIMELINE = '''
{{
repository(owner: "yandex" name: "ClickHouse") {{
pullRequest(number: {number}) {{
timeline(first: {max_page_size} {next}) {{
@ -140,7 +136,6 @@ class Query:
}}
}}
}}
}}
'''
def get_timeline(self, pull_request):
'''Fetchs all cross-reference events from pull-request's timeline
@ -169,7 +164,6 @@ class Query:
return events
_PULL_REQUESTS = '''
{{
repository(owner: "yandex" name: "ClickHouse") {{
defaultBranchRef {{
name
@ -248,7 +242,6 @@ class Query:
}}
}}
}}
}}
'''
def get_pull_requests(self, before_commit, login):
'''Get all merged pull-requests from the HEAD of default branch to the last commit (excluding)
@ -294,13 +287,11 @@ class Query:
return pull_requests
_DEFAULT = '''
{
repository(owner: "yandex", name: "ClickHouse") {
defaultBranchRef {
name
}
}
}
'''
def get_default_branch(self):
'''Get short name of the default branch
@ -334,11 +325,27 @@ class Query:
return session
headers = {'Authorization': f'bearer {self._token}'}
query = f'''
{{
{query}
rateLimit {{
cost
remaining
}}
}}
'''
request = requests_retry_session().post('https://api.github.com/graphql', json={'query': query}, headers=headers)
if request.status_code == 200:
result = request.json()
if 'errors' in result:
raise Exception(f'Errors occured: {result["errors"]}')
import inspect
caller = inspect.getouterframes(inspect.currentframe(), 2)[1][3]
if caller not in self.api_costs.keys():
self.api_costs[caller] = 0
self.api_costs[caller] += result['data']['rateLimit']['cost']
return result['data']
else:
import json