mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge branch 'master' into no-export-dynamic
This commit is contained in:
commit
2a88c240ca
@ -11,6 +11,7 @@ RUN apt-get update \
|
||||
pv \
|
||||
ripgrep \
|
||||
zstd \
|
||||
locales \
|
||||
--yes --no-install-recommends
|
||||
|
||||
# Sanitizer options for services (clickhouse-server)
|
||||
@ -28,6 +29,9 @@ ENV TSAN_OPTIONS='halt_on_error=1 history_size=7 memory_limit_mb=46080 second_de
|
||||
ENV UBSAN_OPTIONS='print_stacktrace=1'
|
||||
ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1'
|
||||
|
||||
RUN echo "en_US.UTF-8 UTF-8" > /etc/locale.gen && locale-gen en_US.UTF-8
|
||||
ENV LC_ALL en_US.UTF-8
|
||||
|
||||
ENV TZ=Europe/Moscow
|
||||
RUN ln -snf "/usr/share/zoneinfo/$TZ" /etc/localtime && echo "$TZ" > /etc/timezone
|
||||
|
||||
|
@ -18,9 +18,13 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
|
||||
python3-pip \
|
||||
shellcheck \
|
||||
yamllint \
|
||||
locales \
|
||||
&& pip3 install black==23.1.0 boto3 codespell==2.2.1 mypy==1.3.0 PyGithub unidiff pylint==2.6.2 \
|
||||
&& apt-get clean \
|
||||
&& rm -rf /root/.cache/pip
|
||||
&& rm -rf /root/.cache/pip
|
||||
|
||||
RUN echo "en_US.UTF-8 UTF-8" > /etc/locale.gen && locale-gen en_US.UTF-8
|
||||
ENV LC_ALL en_US.UTF-8
|
||||
|
||||
# Architecture of the image when BuildKit/buildx is used
|
||||
ARG TARGETARCH
|
||||
|
@ -328,7 +328,7 @@ void ConfigProcessor::mergeRecursive(XMLDocumentPtr config, Node * config_root,
|
||||
}
|
||||
}
|
||||
|
||||
void ConfigProcessor::merge(XMLDocumentPtr config, XMLDocumentPtr with)
|
||||
bool ConfigProcessor::merge(XMLDocumentPtr config, XMLDocumentPtr with)
|
||||
{
|
||||
Node * config_root = getRootNode(config.get());
|
||||
Node * with_root = getRootNode(with.get());
|
||||
@ -343,11 +343,15 @@ void ConfigProcessor::merge(XMLDocumentPtr config, XMLDocumentPtr with)
|
||||
&& !((config_root_node_name == "yandex" || config_root_node_name == "clickhouse")
|
||||
&& (merged_root_node_name == "yandex" || merged_root_node_name == "clickhouse")))
|
||||
{
|
||||
if (config_root_node_name != "clickhouse" && config_root_node_name != "yandex")
|
||||
return false;
|
||||
|
||||
throw Poco::Exception("Root element doesn't have the corresponding root element as the config file."
|
||||
" It must be <" + config_root->nodeName() + ">");
|
||||
}
|
||||
|
||||
mergeRecursive(config, config_root, with_root);
|
||||
return true;
|
||||
}
|
||||
|
||||
void ConfigProcessor::doIncludesRecursive(
|
||||
@ -645,7 +649,12 @@ XMLDocumentPtr ConfigProcessor::processConfig(
|
||||
with = dom_parser.parse(merge_file);
|
||||
}
|
||||
|
||||
merge(config, with);
|
||||
if (!merge(config, with))
|
||||
{
|
||||
LOG_DEBUG(log, "Merging bypassed - configuration file '{}' doesn't belong to configuration '{}' - merging root node name '{}' doesn't match '{}'",
|
||||
merge_file, path, getRootNode(with.get())->nodeName(), getRootNode(config.get())->nodeName());
|
||||
continue;
|
||||
}
|
||||
|
||||
contributing_files.push_back(merge_file);
|
||||
}
|
||||
|
@ -144,7 +144,9 @@ private:
|
||||
|
||||
void mergeRecursive(XMLDocumentPtr config, Poco::XML::Node * config_root, const Poco::XML::Node * with_root);
|
||||
|
||||
void merge(XMLDocumentPtr config, XMLDocumentPtr with);
|
||||
/// If config root node name is not 'clickhouse' and merging config's root node names doesn't match, bypasses merging and returns false.
|
||||
/// For compatibility root node 'yandex' considered equal to 'clickhouse'.
|
||||
bool merge(XMLDocumentPtr config, XMLDocumentPtr with);
|
||||
|
||||
void doIncludesRecursive(
|
||||
XMLDocumentPtr config,
|
||||
|
@ -463,7 +463,7 @@ void CompressionCodecEncrypted::Configuration::loadImpl(
|
||||
|
||||
/// If there is only one key with non zero ID, curren_key_id should be defined.
|
||||
if (new_params->keys_storage[method].size() == 1 && !new_params->keys_storage[method].contains(0))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Config has one key with non zero id. сurrent_key_id is required");
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Config has one key with non zero id. current_key_id is required");
|
||||
}
|
||||
|
||||
/// Try to find which key will be used for encryption. If there is no current_key and only one key without id
|
||||
|
@ -433,10 +433,10 @@ void DiskEncrypted::applyNewSettings(
|
||||
{
|
||||
auto new_settings = parseDiskEncryptedSettings(name, config, config_prefix, disk_map);
|
||||
if (new_settings->wrapped_disk != delegate)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Сhanging wrapped disk on the fly is not supported. Disk {}", name);
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Changing wrapped disk on the fly is not supported. Disk {}", name);
|
||||
|
||||
if (new_settings->disk_path != disk_path)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Сhanging disk path on the fly is not supported. Disk {}", name);
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Changing disk path on the fly is not supported. Disk {}", name);
|
||||
|
||||
current_settings.set(std::move(new_settings));
|
||||
IDisk::applyNewSettings(config, context, config_prefix, disk_map);
|
||||
|
@ -40,7 +40,7 @@ int readAndPrint(DB::ReadBuffer & in)
|
||||
int main(int, char **)
|
||||
{
|
||||
{
|
||||
std::string s = "-123456 123.456 вася пе\\tтя\t'\\'xyz\\\\'";
|
||||
std::string s = "-123456 123.456 вася pe\\ttya\t'\\'xyz\\\\'";
|
||||
DB::ReadBufferFromString in(s);
|
||||
if (readAndPrint(in))
|
||||
std::cout << "readAndPrint from ReadBufferFromString failed" << std::endl;
|
||||
@ -49,7 +49,7 @@ int main(int, char **)
|
||||
|
||||
std::shared_ptr<DB::ReadBufferFromOwnString> in;
|
||||
{
|
||||
std::string s = "-123456 123.456 вася пе\\tтя\t'\\'xyz\\\\'";
|
||||
std::string s = "-123456 123.456 вася pe\\ttya\t'\\'xyz\\\\'";
|
||||
in = std::make_shared<DB::ReadBufferFromOwnString>(s);
|
||||
}
|
||||
if (readAndPrint(*in))
|
||||
|
@ -14,7 +14,7 @@ int main(int, char **)
|
||||
{
|
||||
DB::Int64 a = -123456;
|
||||
DB::Float64 b = 123.456;
|
||||
DB::String c = "вася пе\tтя";
|
||||
DB::String c = "вася pe\ttya";
|
||||
DB::String d = "'xyz\\";
|
||||
|
||||
std::stringstream s; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
|
@ -14,7 +14,7 @@ int main(int, char **)
|
||||
{
|
||||
DB::Int64 a = -123456;
|
||||
DB::Float64 b = 123.456;
|
||||
DB::String c = "вася пе\tтя";
|
||||
DB::String c = "вася pe\ttya";
|
||||
DB::String d = "'xyz\\";
|
||||
|
||||
std::ofstream s("test");
|
||||
|
@ -115,7 +115,10 @@
|
||||
#include <re2/re2.h>
|
||||
#include <Storages/StorageView.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/FunctionParameterValuesVisitor.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <base/find_symbols.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
|
||||
#if USE_ROCKSDB
|
||||
#include <rocksdb/table.h>
|
||||
@ -1580,8 +1583,21 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const
|
||||
{
|
||||
if (table.get()->isView() && table->as<StorageView>() && table->as<StorageView>()->isParameterizedView())
|
||||
{
|
||||
auto query = table->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone();
|
||||
NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression);
|
||||
StorageView::replaceQueryParametersIfParametrizedView(query, parameterized_view_values);
|
||||
|
||||
ASTCreateQuery create;
|
||||
create.select = query->as<ASTSelectWithUnionQuery>();
|
||||
auto sample_block = InterpreterSelectWithUnionQuery::getSampleBlock(query, getQueryContext());
|
||||
auto res = std::make_shared<StorageView>(StorageID(database_name, table_name),
|
||||
create,
|
||||
ColumnsDescription(sample_block.getNamesAndTypesList()),
|
||||
/* comment */ "",
|
||||
/* is_parameterized_view */ true);
|
||||
res->startup();
|
||||
function->prefer_subquery_to_function_formatting = true;
|
||||
return table;
|
||||
return res;
|
||||
}
|
||||
}
|
||||
auto hash = table_expression->getTreeHash();
|
||||
|
@ -610,27 +610,10 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
{
|
||||
/// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it.
|
||||
ASTPtr view_table;
|
||||
NameToNameMap parameter_types;
|
||||
if (view)
|
||||
{
|
||||
query_info.is_parameterized_view = view->isParameterizedView();
|
||||
/// We need to fetch the parameters set for SELECT ... FROM parameterized_view(<params>) before the query is replaced.
|
||||
/// replaceWithSubquery replaces the function child and adds the subquery in its place.
|
||||
/// the parameters are children of function child, if function (which corresponds to parametrised view and has
|
||||
/// parameters in its arguments: `parametrised_view(<params>)`) is replaced the parameters are also gone from tree
|
||||
/// So we need to get the parameters before they are removed from the tree
|
||||
/// and after query is replaced, we use these parameters to substitute in the parameterized view query
|
||||
if (query_info.is_parameterized_view)
|
||||
{
|
||||
query_info.parameterized_view_values = analyzeFunctionParamValues(query_ptr);
|
||||
parameter_types = view->getParameterTypes();
|
||||
}
|
||||
view->replaceWithSubquery(getSelectQuery(), view_table, metadata_snapshot, view->isParameterizedView());
|
||||
if (query_info.is_parameterized_view)
|
||||
{
|
||||
view->replaceQueryParametersIfParametrizedView(query_ptr, query_info.parameterized_view_values);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
syntax_analyzer_result = TreeRewriter(context).analyzeSelect(
|
||||
@ -639,10 +622,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
options,
|
||||
joined_tables.tablesWithColumns(),
|
||||
required_result_column_names,
|
||||
table_join,
|
||||
query_info.is_parameterized_view,
|
||||
query_info.parameterized_view_values,
|
||||
parameter_types);
|
||||
table_join);
|
||||
|
||||
|
||||
query_info.syntax_analyzer_result = syntax_analyzer_result;
|
||||
@ -793,7 +773,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
query_info.filter_asts.push_back(parallel_replicas_custom_filter_ast);
|
||||
}
|
||||
|
||||
source_header = storage_snapshot->getSampleBlockForColumns(required_columns, query_info.parameterized_view_values);
|
||||
source_header = storage_snapshot->getSampleBlockForColumns(required_columns);
|
||||
}
|
||||
|
||||
/// Calculate structure of the result.
|
||||
|
@ -249,13 +249,6 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
||||
if (first_table || !data.join_using_columns.contains(column.name))
|
||||
{
|
||||
std::string column_name = column.name;
|
||||
|
||||
/// replaceQueryParameterWithValue is used for parameterized view (which are created using query parameters
|
||||
/// and SELECT is used with substitution of these query parameters )
|
||||
if (!data.parameter_values.empty())
|
||||
column_name
|
||||
= StorageView::replaceQueryParameterWithValue(column_name, data.parameter_values, data.parameter_types);
|
||||
|
||||
addIdentifier(columns, table.table, column_name);
|
||||
}
|
||||
}
|
||||
|
@ -28,15 +28,11 @@ public:
|
||||
const TablesWithColumns & tables;
|
||||
std::unordered_set<String> join_using_columns;
|
||||
bool has_columns;
|
||||
NameToNameMap parameter_values;
|
||||
NameToNameMap parameter_types;
|
||||
|
||||
Data(const NameSet & source_columns_, const TablesWithColumns & tables_, bool has_columns_ = true, const NameToNameMap & parameter_values_ = {}, const NameToNameMap & parameter_types_ = {})
|
||||
Data(const NameSet & source_columns_, const TablesWithColumns & tables_, bool has_columns_ = true)
|
||||
: source_columns(source_columns_)
|
||||
, tables(tables_)
|
||||
, has_columns(has_columns_)
|
||||
, parameter_values(parameter_values_)
|
||||
, parameter_types(parameter_types_)
|
||||
{}
|
||||
|
||||
bool hasColumn(const String & name) const { return source_columns.count(name); }
|
||||
|
@ -299,11 +299,10 @@ using ReplacePositionalArgumentsVisitor = InDepthNodeVisitor<OneTypeMatcher<Repl
|
||||
/// Expand asterisks and qualified asterisks with column names.
|
||||
/// There would be columns in normal form & column aliases after translation. Column & column alias would be normalized in QueryNormalizer.
|
||||
void translateQualifiedNames(ASTPtr & query, const ASTSelectQuery & select_query, const NameSet & source_columns_set,
|
||||
const TablesWithColumns & tables_with_columns, const NameToNameMap & parameter_values = {},
|
||||
const NameToNameMap & parameter_types = {})
|
||||
const TablesWithColumns & tables_with_columns)
|
||||
{
|
||||
LogAST log;
|
||||
TranslateQualifiedNamesVisitor::Data visitor_data(source_columns_set, tables_with_columns, true/* has_columns */, parameter_values, parameter_types);
|
||||
TranslateQualifiedNamesVisitor::Data visitor_data(source_columns_set, tables_with_columns, true/* has_columns */);
|
||||
TranslateQualifiedNamesVisitor visitor(visitor_data, log.stream());
|
||||
visitor.visit(query);
|
||||
|
||||
@ -1157,10 +1156,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
|
||||
const SelectQueryOptions & select_options,
|
||||
const TablesWithColumns & tables_with_columns,
|
||||
const Names & required_result_columns,
|
||||
std::shared_ptr<TableJoin> table_join,
|
||||
bool is_parameterized_view,
|
||||
const NameToNameMap parameter_values,
|
||||
const NameToNameMap parameter_types) const
|
||||
std::shared_ptr<TableJoin> table_join) const
|
||||
{
|
||||
auto * select_query = query->as<ASTSelectQuery>();
|
||||
if (!select_query)
|
||||
@ -1198,7 +1194,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
|
||||
result.analyzed_join->setColumnsFromJoinedTable(std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix());
|
||||
}
|
||||
|
||||
translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns, parameter_values, parameter_types);
|
||||
translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns);
|
||||
|
||||
/// Optimizes logical expressions.
|
||||
LogicalExpressionsOptimizer(select_query, tables_with_columns, settings.optimize_min_equality_disjunction_chain_length.value).perform();
|
||||
@ -1256,15 +1252,6 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
|
||||
result.window_function_asts = getWindowFunctions(query, *select_query);
|
||||
result.expressions_with_window_function = getExpressionsWithWindowFunctions(query);
|
||||
|
||||
/// replaceQueryParameterWithValue is used for parameterized view (which are created using query parameters
|
||||
/// and SELECT is used with substitution of these query parameters )
|
||||
/// the replaced column names will be used in the next steps
|
||||
if (is_parameterized_view)
|
||||
{
|
||||
for (auto & column : result.source_columns)
|
||||
column.name = StorageView::replaceQueryParameterWithValue(column.name, parameter_values, parameter_types);
|
||||
}
|
||||
|
||||
result.collectUsedColumns(query, true, settings.query_plan_optimize_primary_key);
|
||||
|
||||
result.required_source_columns_before_expanding_alias_columns = result.required_source_columns.getNames();
|
||||
|
@ -128,10 +128,7 @@ public:
|
||||
const SelectQueryOptions & select_options = {},
|
||||
const std::vector<TableWithColumnNamesAndTypes> & tables_with_columns = {},
|
||||
const Names & required_result_columns = {},
|
||||
std::shared_ptr<TableJoin> table_join = {},
|
||||
bool is_parameterized_view = false,
|
||||
const NameToNameMap parameter_values = {},
|
||||
const NameToNameMap parameter_types = {}) const;
|
||||
std::shared_ptr<TableJoin> table_join = {}) const;
|
||||
|
||||
private:
|
||||
static void normalize(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, bool ignore_alias, const Settings & settings, bool allow_self_aliases, ContextPtr context_, bool is_create_parameterized_view = false);
|
||||
|
@ -180,7 +180,7 @@ Chunk ParquetMetadataInputFormat::generate()
|
||||
else if (name == names[3])
|
||||
{
|
||||
auto column = types[3]->createColumn();
|
||||
/// Version сan be only PARQUET_1_0 or PARQUET_2_LATEST (which is 2.6).
|
||||
/// Version can be only PARQUET_1_0 or PARQUET_2_LATEST (which is 2.6).
|
||||
String version = metadata->version() == parquet::ParquetVersion::PARQUET_1_0 ? "1.0" : "2.6";
|
||||
assert_cast<ColumnString &>(*column).insertData(version.data(), version.size());
|
||||
res.addColumn(std::move(column));
|
||||
|
@ -258,7 +258,6 @@ struct SelectQueryInfo
|
||||
bool parallel_replicas_disabled = false;
|
||||
|
||||
bool is_parameterized_view = false;
|
||||
NameToNameMap parameterized_view_values;
|
||||
|
||||
// If limit is not 0, that means it's a trivial limit query.
|
||||
UInt64 limit = 0;
|
||||
|
@ -113,22 +113,15 @@ NameAndTypePair StorageSnapshot::getColumn(const GetColumnsOptions & options, co
|
||||
return *column;
|
||||
}
|
||||
|
||||
Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names, const NameToNameMap & parameter_values) const
|
||||
Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) const
|
||||
{
|
||||
Block res;
|
||||
|
||||
const auto & columns = getMetadataForQuery()->getColumns();
|
||||
for (const auto & column_name : column_names)
|
||||
{
|
||||
std::string substituted_column_name = column_name;
|
||||
|
||||
/// substituted_column_name is used for parameterized view (which are created using query parameters
|
||||
/// and SELECT is used with substitution of these query parameters )
|
||||
if (!parameter_values.empty())
|
||||
substituted_column_name = StorageView::replaceValueWithQueryParameter(column_name, parameter_values);
|
||||
|
||||
auto column = columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, substituted_column_name);
|
||||
auto object_column = object_columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, substituted_column_name);
|
||||
auto column = columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, column_name);
|
||||
auto object_column = object_columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, column_name);
|
||||
if (column && !object_column)
|
||||
{
|
||||
res.insert({column->type->createColumn(), column->type, column_name});
|
||||
@ -147,7 +140,7 @@ Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names, cons
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK,
|
||||
"Column {} not found in table {}", backQuote(substituted_column_name), storage.getStorageID().getNameForLogs());
|
||||
"Column {} not found in table {}", backQuote(column_name), storage.getStorageID().getNameForLogs());
|
||||
}
|
||||
}
|
||||
return res;
|
||||
|
@ -71,7 +71,7 @@ struct StorageSnapshot
|
||||
NameAndTypePair getColumn(const GetColumnsOptions & options, const String & column_name) const;
|
||||
|
||||
/// Block with ordinary + materialized + aliases + virtuals + subcolumns.
|
||||
Block getSampleBlockForColumns(const Names & column_names, const NameToNameMap & parameter_values = {}) const;
|
||||
Block getSampleBlockForColumns(const Names & column_names) const;
|
||||
|
||||
ColumnsDescription getDescriptionForColumns(const Names & column_names) const;
|
||||
|
||||
|
@ -107,7 +107,8 @@ StorageView::StorageView(
|
||||
const StorageID & table_id_,
|
||||
const ASTCreateQuery & query,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & comment)
|
||||
const String & comment,
|
||||
const bool is_parameterized_view_)
|
||||
: IStorage(table_id_)
|
||||
{
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
@ -123,8 +124,7 @@ StorageView::StorageView(
|
||||
NormalizeSelectWithUnionQueryVisitor::Data data{SetOperationMode::Unspecified};
|
||||
NormalizeSelectWithUnionQueryVisitor{data}.visit(description.inner_query);
|
||||
|
||||
is_parameterized_view = query.isParameterizedView();
|
||||
view_parameter_types = analyzeReceiveQueryParamsWithType(description.inner_query);
|
||||
is_parameterized_view = is_parameterized_view_ || query.isParameterizedView();
|
||||
storage_metadata.setSelectQuery(description);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
}
|
||||
@ -173,7 +173,7 @@ void StorageView::read(
|
||||
query_plan.addStep(std::move(materializing));
|
||||
|
||||
/// And also convert to expected structure.
|
||||
const auto & expected_header = storage_snapshot->getSampleBlockForColumns(column_names, query_info.parameterized_view_values);
|
||||
const auto & expected_header = storage_snapshot->getSampleBlockForColumns(column_names);
|
||||
const auto & header = query_plan.getCurrentDataStream().header;
|
||||
|
||||
const auto * select_with_union = current_inner_query->as<ASTSelectWithUnionQuery>();
|
||||
@ -258,42 +258,6 @@ void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_
|
||||
child = view_query;
|
||||
}
|
||||
|
||||
String StorageView::replaceQueryParameterWithValue(const String & column_name, const NameToNameMap & parameter_values, const NameToNameMap & parameter_types)
|
||||
{
|
||||
std::string name = column_name;
|
||||
std::string::size_type pos = 0u;
|
||||
for (const auto & parameter : parameter_values)
|
||||
{
|
||||
if ((pos = name.find(parameter.first)) != std::string::npos)
|
||||
{
|
||||
auto parameter_datatype_iterator = parameter_types.find(parameter.first);
|
||||
size_t parameter_end = pos + parameter.first.size();
|
||||
if (parameter_datatype_iterator != parameter_types.end() && name.size() >= parameter_end && (name[parameter_end] == ',' || name[parameter_end] == ')'))
|
||||
{
|
||||
String parameter_name("_CAST(" + parameter.second + ", '" + parameter_datatype_iterator->second + "')");
|
||||
name.replace(pos, parameter.first.size(), parameter_name);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
return name;
|
||||
}
|
||||
|
||||
String StorageView::replaceValueWithQueryParameter(const String & column_name, const NameToNameMap & parameter_values)
|
||||
{
|
||||
String name = column_name;
|
||||
std::string::size_type pos = 0u;
|
||||
for (const auto & parameter : parameter_values)
|
||||
{
|
||||
if ((pos = name.find("_CAST(" + parameter.second)) != std::string::npos)
|
||||
{
|
||||
name = name.substr(0,pos) + parameter.first + ")";
|
||||
break;
|
||||
}
|
||||
}
|
||||
return name;
|
||||
}
|
||||
|
||||
ASTPtr StorageView::restoreViewName(ASTSelectQuery & select_query, const ASTPtr & view_name)
|
||||
{
|
||||
ASTTableExpression * table_expression = getFirstTableExpression(select_query);
|
||||
|
@ -15,7 +15,8 @@ public:
|
||||
const StorageID & table_id_,
|
||||
const ASTCreateQuery & query,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & comment);
|
||||
const String & comment,
|
||||
const bool is_parameterized_view_=false);
|
||||
|
||||
std::string getName() const override { return "View"; }
|
||||
bool isView() const override { return true; }
|
||||
@ -44,17 +45,9 @@ public:
|
||||
|
||||
static void replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_query, ASTPtr & view_name, const bool parameterized_view);
|
||||
static ASTPtr restoreViewName(ASTSelectQuery & select_query, const ASTPtr & view_name);
|
||||
static String replaceQueryParameterWithValue (const String & column_name, const NameToNameMap & parameter_values, const NameToNameMap & parameter_types);
|
||||
static String replaceValueWithQueryParameter (const String & column_name, const NameToNameMap & parameter_values);
|
||||
|
||||
const NameToNameMap & getParameterTypes() const
|
||||
{
|
||||
return view_parameter_types;
|
||||
}
|
||||
|
||||
protected:
|
||||
bool is_parameterized_view;
|
||||
NameToNameMap view_parameter_types;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -12,6 +12,7 @@
|
||||
# (simple regexps) to check if the code is likely to have basic style violations.
|
||||
# and then to run formatter only for the specified files.
|
||||
|
||||
LC_ALL="en_US.UTF-8"
|
||||
ROOT_PATH=$(git rev-parse --show-toplevel)
|
||||
EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|poco/|memcpy/|consistent-hashing|benchmark|tests/.*.cpp|utils/keeper-bench/example.yaml'
|
||||
|
||||
@ -413,3 +414,6 @@ find $ROOT_PATH/tests/queries/1_stateful -name '*.sql' -or -name '*.sh' | grep -
|
||||
|
||||
# Check for bad punctuation: whitespace before comma.
|
||||
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P --line-number '\w ,' | grep -v 'bad punctuation is ok here' && echo "^ There is bad punctuation: whitespace before comma. You should write it like this: 'Hello, world!'"
|
||||
|
||||
# Cyrillic characters hiding inside Latin.
|
||||
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P --line-number '[a-zA-Z][а-яА-ЯёЁ]|[а-яА-ЯёЁ][a-zA-Z]' && echo "^ Cyrillic characters found in unexpected place."
|
||||
|
Loading…
Reference in New Issue
Block a user