Merge branch 'master' into fixes-start-stop-listen

This commit is contained in:
Alexey Milovidov 2023-07-31 08:34:17 +03:00 committed by GitHub
commit a7b4a29ed9
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
93 changed files with 504 additions and 371 deletions

View File

@ -165,8 +165,14 @@ elseif(GLIBC_COMPATIBILITY)
message (${RECONFIGURE_MESSAGE_LEVEL} "Glibc compatibility cannot be enabled in current configuration")
endif ()
# Make sure the final executable has symbols exported
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -rdynamic")
if (OS_LINUX)
# We should not export dynamic symbols, because:
# - The main clickhouse binary does not use dlopen,
# and whatever is poisoning it by LD_PRELOAD should not link to our symbols.
# - The clickhouse-odbc-bridge and clickhouse-library-bridge binaries
# should not expose their symbols to ODBC drivers and libraries.
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic")
endif ()
if (OS_DARWIN)
# The `-all_load` flag forces loading of all symbols from all libraries,

View File

@ -22,8 +22,9 @@ macro(clickhouse_split_debug_symbols)
# Splits debug symbols into separate file, leaves the binary untouched:
COMMAND "${OBJCOPY_PATH}" --only-keep-debug "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug"
COMMAND chmod 0644 "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug"
# Strips binary, sections '.note' & '.comment' are removed in line with Debian's stripping policy: www.debian.org/doc/debian-policy/ch-files.html, section '.clickhouse.hash' is needed for integrity check:
COMMAND "${STRIP_PATH}" --remove-section=.comment --remove-section=.note --keep-section=.clickhouse.hash "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}"
# Strips binary, sections '.note' & '.comment' are removed in line with Debian's stripping policy: www.debian.org/doc/debian-policy/ch-files.html, section '.clickhouse.hash' is needed for integrity check.
# Also, after we disabled the export of symbols for dynamic linking, we still to keep a static symbol table for good stack traces.
COMMAND "${STRIP_PATH}" --strip-debug --remove-section=.comment --remove-section=.note "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}"
# Associate stripped binary with debug symbols:
COMMAND "${OBJCOPY_PATH}" --add-gnu-debuglink "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug" "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}"
COMMENT "Stripping clickhouse binary" VERBATIM

View File

@ -64,7 +64,7 @@ then
ninja $NINJA_FLAGS clickhouse-keeper
ls -la ./programs/
ldd ./programs/clickhouse-keeper
ldd ./programs/clickhouse-keeper ||:
if [ -n "$MAKE_DEB" ]; then
# No quotes because I want it to expand to nothing if empty.
@ -80,19 +80,9 @@ else
cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" ..
fi
if [ "coverity" == "$COMBINED_OUTPUT" ]
then
mkdir -p /workdir/cov-analysis
wget --post-data "token=$COVERITY_TOKEN&project=ClickHouse%2FClickHouse" -qO- https://scan.coverity.com/download/linux64 | tar xz -C /workdir/cov-analysis --strip-components 1
export PATH=$PATH:/workdir/cov-analysis/bin
cov-configure --config ./coverity.config --template --comptype clangcc --compiler "$CC"
SCAN_WRAPPER="cov-build --config ./coverity.config --dir cov-int"
fi
# No quotes because I want it to expand to nothing if empty.
# shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty.
$SCAN_WRAPPER ninja $NINJA_FLAGS $BUILD_TARGET
ninja $NINJA_FLAGS $BUILD_TARGET
ls -la ./programs
@ -175,13 +165,6 @@ then
mv "$COMBINED_OUTPUT.tar.zst" /output
fi
if [ "coverity" == "$COMBINED_OUTPUT" ]
then
# Coverity does not understand ZSTD.
tar -cvz -f "coverity-scan.tar.gz" cov-int
mv "coverity-scan.tar.gz" /output
fi
ccache_status
ccache --evict-older-than 1d

View File

@ -253,11 +253,6 @@ def parse_env_variables(
cmake_flags.append(f"-DCMAKE_C_COMPILER={cc}")
cmake_flags.append(f"-DCMAKE_CXX_COMPILER={cxx}")
# Create combined output archive for performance tests.
if package_type == "coverity":
result.append("COMBINED_OUTPUT=coverity")
result.append('COVERITY_TOKEN="$COVERITY_TOKEN"')
if sanitizer:
result.append(f"SANITIZER={sanitizer}")
if build_type:
@ -356,7 +351,7 @@ def parse_args() -> argparse.Namespace:
)
parser.add_argument(
"--package-type",
choices=["deb", "binary", "coverity"],
choices=["deb", "binary"],
required=True,
)
parser.add_argument(

View File

@ -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

View File

@ -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

View File

@ -10,6 +10,7 @@ Columns:
- `event` ([String](../../sql-reference/data-types/string.md)) — Event name.
- `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of events occurred.
- `description` ([String](../../sql-reference/data-types/string.md)) — Event description.
- `name` ([String](../../sql-reference/data-types/string.md)) — Alias for `event`.
You can find all supported events in source file [src/Common/ProfileEvents.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/ProfileEvents.cpp).

View File

@ -10,6 +10,7 @@ Columns:
- `metric` ([String](../../sql-reference/data-types/string.md)) — Metric name.
- `value` ([Int64](../../sql-reference/data-types/int-uint.md)) — Metric value.
- `description` ([String](../../sql-reference/data-types/string.md)) — Metric description.
- `name` ([String](../../sql-reference/data-types/string.md)) — Alias for `metric`.
You can find all supported metrics in source file [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp).

View File

@ -13,10 +13,6 @@ set (CLICKHOUSE_LIBRARY_BRIDGE_SOURCES
library-bridge.cpp
)
if (OS_LINUX)
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic")
endif ()
clickhouse_add_executable(clickhouse-library-bridge ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES})
target_link_libraries(clickhouse-library-bridge PRIVATE

View File

@ -15,12 +15,6 @@ set (CLICKHOUSE_ODBC_BRIDGE_SOURCES
validateODBCConnectionString.cpp
)
if (OS_LINUX)
# clickhouse-odbc-bridge is always a separate binary.
# Reason: it must not export symbols from SSL, mariadb-client, etc. to not break ABI compatibility with ODBC drivers.
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic")
endif ()
clickhouse_add_executable(clickhouse-odbc-bridge ${CLICKHOUSE_ODBC_BRIDGE_SOURCES})
target_link_libraries(clickhouse-odbc-bridge PRIVATE

View File

@ -1691,17 +1691,26 @@ try
global_context->initializeTraceCollector();
/// Set up server-wide memory profiler (for total memory tracker).
UInt64 total_memory_profiler_step = config().getUInt64("total_memory_profiler_step", 0);
if (total_memory_profiler_step)
if (server_settings.total_memory_profiler_step)
{
total_memory_tracker.setProfilerStep(total_memory_profiler_step);
total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step);
}
double total_memory_tracker_sample_probability = config().getDouble("total_memory_tracker_sample_probability", 0);
if (total_memory_tracker_sample_probability > 0.0)
if (server_settings.total_memory_tracker_sample_probability > 0.0)
{
total_memory_tracker.setSampleProbability(total_memory_tracker_sample_probability);
total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability);
}
if (server_settings.total_memory_profiler_sample_min_allocation_size)
{
total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size);
}
if (server_settings.total_memory_profiler_sample_max_allocation_size)
{
total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size);
}
}
#endif

View File

@ -7,6 +7,7 @@
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/QueryNode.h>
#include <Analyzer/TableFunctionNode.h>
#include <Analyzer/UnionNode.h>
#include <Interpreters/Context.h>
@ -90,26 +91,25 @@ private:
template <typename Derived>
using ConstInDepthQueryTreeVisitor = InDepthQueryTreeVisitor<Derived, true /*const_visitor*/>;
/** Same as InDepthQueryTreeVisitor and additionally keeps track of current scope context.
/** Same as InDepthQueryTreeVisitor (but has a different interface) and additionally keeps track of current scope context.
* This can be useful if your visitor has special logic that depends on current scope context.
*
* To specify behavior of the visitor you can implement following methods in derived class:
* 1. needChildVisit This methods allows to skip subtree.
* 2. enterImpl This method is called before children are processed.
* 3. leaveImpl This method is called after children are processed.
*/
template <typename Derived, bool const_visitor = false>
class InDepthQueryTreeVisitorWithContext
{
public:
using VisitQueryTreeNodeType = std::conditional_t<const_visitor, const QueryTreeNodePtr, QueryTreeNodePtr>;
using VisitQueryTreeNodeType = QueryTreeNodePtr;
explicit InDepthQueryTreeVisitorWithContext(ContextPtr context, size_t initial_subquery_depth = 0)
: current_context(std::move(context))
, subquery_depth(initial_subquery_depth)
{}
/// Return true if visitor should traverse tree top to bottom, false otherwise
bool shouldTraverseTopToBottom() const
{
return true;
}
/// Return true if visitor should visit child, false otherwise
bool needChildVisit(VisitQueryTreeNodeType & parent [[maybe_unused]], VisitQueryTreeNodeType & child [[maybe_unused]])
{
@ -146,18 +146,16 @@ public:
++subquery_depth;
bool traverse_top_to_bottom = getDerived().shouldTraverseTopToBottom();
if (!traverse_top_to_bottom)
visitChildren(query_tree_node);
getDerived().enterImpl(query_tree_node);
getDerived().visitImpl(query_tree_node);
if (traverse_top_to_bottom)
visitChildren(query_tree_node);
visitChildren(query_tree_node);
getDerived().leaveImpl(query_tree_node);
}
void enterImpl(VisitQueryTreeNodeType & node [[maybe_unused]])
{}
void leaveImpl(VisitQueryTreeNodeType & node [[maybe_unused]])
{}
private:
@ -171,17 +169,31 @@ private:
return *static_cast<Derived *>(this);
}
bool shouldSkipSubtree(
VisitQueryTreeNodeType & parent,
VisitQueryTreeNodeType & child,
size_t subtree_index)
{
bool need_visit_child = getDerived().needChildVisit(parent, child);
if (!need_visit_child)
return true;
if (auto * table_function_node = parent->as<TableFunctionNode>())
{
const auto & unresolved_indexes = table_function_node->getUnresolvedArgumentIndexes();
return std::find(unresolved_indexes.begin(), unresolved_indexes.end(), subtree_index) != unresolved_indexes.end();
}
return false;
}
void visitChildren(VisitQueryTreeNodeType & expression)
{
size_t index = 0;
for (auto & child : expression->getChildren())
{
if (!child)
continue;
bool need_visit_child = getDerived().needChildVisit(expression, child);
if (need_visit_child)
if (child && !shouldSkipSubtree(expression, child, index))
visit(child);
++index;
}
}
@ -189,50 +201,4 @@ private:
size_t subquery_depth = 0;
};
template <typename Derived>
using ConstInDepthQueryTreeVisitorWithContext = InDepthQueryTreeVisitorWithContext<Derived, true /*const_visitor*/>;
/** Visitor that use another visitor to visit node only if condition for visiting node is true.
* For example, your visitor need to visit only query tree nodes or union nodes.
*
* Condition interface:
* struct Condition
* {
* bool operator()(VisitQueryTreeNodeType & node)
* {
* return shouldNestedVisitorVisitNode(node);
* }
* }
*/
template <typename Visitor, typename Condition, bool const_visitor = false>
class InDepthQueryTreeConditionalVisitor : public InDepthQueryTreeVisitor<InDepthQueryTreeConditionalVisitor<Visitor, Condition, const_visitor>, const_visitor>
{
public:
using Base = InDepthQueryTreeVisitor<InDepthQueryTreeConditionalVisitor<Visitor, Condition, const_visitor>, const_visitor>;
using VisitQueryTreeNodeType = typename Base::VisitQueryTreeNodeType;
explicit InDepthQueryTreeConditionalVisitor(Visitor & visitor_, Condition & condition_)
: visitor(visitor_)
, condition(condition_)
{
}
bool shouldTraverseTopToBottom() const
{
return visitor.shouldTraverseTopToBottom();
}
void visitImpl(VisitQueryTreeNodeType & query_tree_node)
{
if (condition(query_tree_node))
visitor.visit(query_tree_node);
}
Visitor & visitor;
Condition & condition;
};
template <typename Visitor, typename Condition>
using ConstInDepthQueryTreeConditionalVisitor = InDepthQueryTreeConditionalVisitor<Visitor, Condition, true /*const_visitor*/>;
}

View File

@ -51,13 +51,7 @@ public:
using Base = InDepthQueryTreeVisitorWithContext<AggregateFunctionsArithmericOperationsVisitor>;
using Base::Base;
/// Traverse tree bottom to top
static bool shouldTraverseTopToBottom()
{
return false;
}
void visitImpl(QueryTreeNodePtr & node)
void leaveImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_arithmetic_operations_in_aggregate_functions)
return;

View File

@ -22,7 +22,7 @@ public:
using Base = InDepthQueryTreeVisitorWithContext<RewriteArrayExistsToHasVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_rewrite_array_exists_to_has)
return;

View File

@ -20,7 +20,7 @@ public:
using Base = InDepthQueryTreeVisitorWithContext<AutoFinalOnQueryPassVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().final)
return;

View File

@ -50,7 +50,7 @@ public:
&& settings.max_hyperscan_regexp_total_length == 0;
}
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
auto * function_node = node->as<FunctionNode>();
if (!function_node || function_node->getFunctionName() != "or")

View File

@ -688,7 +688,7 @@ public:
using Base = InDepthQueryTreeVisitorWithContext<ConvertQueryToCNFVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
auto * query_node = node->as<QueryNode>();
if (!query_node)

View File

@ -22,7 +22,7 @@ public:
using Base = InDepthQueryTreeVisitorWithContext<CountDistinctVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().count_distinct_optimization)
return;

View File

@ -193,7 +193,7 @@ public:
return true;
}
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
if (!isEnabled())
return;

View File

@ -29,7 +29,7 @@ public:
using Base = InDepthQueryTreeVisitorWithContext<FunctionToSubcolumnsVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node) const
void enterImpl(QueryTreeNodePtr & node) const
{
if (!getSettings().optimize_functions_to_subcolumns)
return;

View File

@ -37,7 +37,7 @@ public:
, names_to_collect(names_to_collect_)
{}
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_syntax_fuse_functions)
return;

View File

@ -46,7 +46,7 @@ public:
{
}
void visitImpl(const QueryTreeNodePtr & node)
void enterImpl(const QueryTreeNodePtr & node)
{
auto * function_node = node->as<FunctionNode>();
if (!function_node || function_node->getFunctionName() != "grouping")

View File

@ -23,7 +23,7 @@ public:
, multi_if_function_ptr(std::move(multi_if_function_ptr_))
{}
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_if_chain_to_multiif)
return;

View File

@ -113,7 +113,7 @@ public:
using Base = InDepthQueryTreeVisitorWithContext<ConvertStringsToEnumVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_if_transform_strings_to_enum)
return;

View File

@ -19,7 +19,7 @@ public:
: Base(std::move(context))
{}
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
auto * function_node = node->as<FunctionNode>();

View File

@ -21,7 +21,7 @@ public:
, if_function_ptr(std::move(if_function_ptr_))
{}
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_multiif_to_if)
return;

View File

@ -20,7 +20,7 @@ public:
using Base = InDepthQueryTreeVisitorWithContext<NormalizeCountVariantsVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_normalize_count_variants)
return;

View File

@ -26,7 +26,7 @@ public:
return !child->as<FunctionNode>();
}
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_group_by_function_keys)
return;

View File

@ -28,7 +28,7 @@ public:
return true;
}
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_redundant_functions_in_order_by)
return;

View File

@ -6451,7 +6451,7 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node,
table_function_ptr->parseArguments(table_function_ast, scope_context);
auto table_function_storage = scope_context->getQueryContext()->executeTableFunction(table_function_ast, table_function_ptr);
table_function_node_typed.resolve(std::move(table_function_ptr), std::move(table_function_storage), scope_context);
table_function_node_typed.resolve(std::move(table_function_ptr), std::move(table_function_storage), scope_context, std::move(skip_analysis_arguments_indexes));
}
/// Resolve array join node in scope

View File

@ -26,7 +26,7 @@ public:
using Base = InDepthQueryTreeVisitorWithContext<RewriteAggregateFunctionWithIfVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_rewrite_aggregate_function_with_if)
return;

View File

@ -24,7 +24,7 @@ public:
using Base = InDepthQueryTreeVisitorWithContext<ShardNumColumnToFunctionVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node) const
void enterImpl(QueryTreeNodePtr & node) const
{
auto * column_node = node->as<ColumnNode>();
if (!column_node)

View File

@ -26,7 +26,7 @@ public:
using Base = InDepthQueryTreeVisitorWithContext<SumIfToCountIfVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_rewrite_sum_if_to_count_if)
return;

View File

@ -31,7 +31,7 @@ public:
using Base = InDepthQueryTreeVisitorWithContext<UniqInjectiveFunctionsEliminationVisitor>;
using Base::Base;
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_injective_functions_inside_uniq)
return;

View File

@ -27,12 +27,13 @@ TableFunctionNode::TableFunctionNode(String table_function_name_)
children[arguments_child_index] = std::make_shared<ListNode>();
}
void TableFunctionNode::resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context)
void TableFunctionNode::resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context, std::vector<size_t> unresolved_arguments_indexes_)
{
table_function = std::move(table_function_value);
storage = std::move(storage_value);
storage_id = storage->getStorageID();
storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context);
unresolved_arguments_indexes = std::move(unresolved_arguments_indexes_);
}
const StorageID & TableFunctionNode::getStorageID() const
@ -132,6 +133,7 @@ QueryTreeNodePtr TableFunctionNode::cloneImpl() const
result->storage_snapshot = storage_snapshot;
result->table_expression_modifiers = table_expression_modifiers;
result->settings_changes = settings_changes;
result->unresolved_arguments_indexes = unresolved_arguments_indexes;
return result;
}

View File

@ -98,7 +98,7 @@ public:
}
/// Resolve table function with table function, storage and context
void resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context);
void resolve(TableFunctionPtr table_function_value, StoragePtr storage_value, ContextPtr context, std::vector<size_t> unresolved_arguments_indexes_);
/// Get storage id, throws exception if function node is not resolved
const StorageID & getStorageID() const;
@ -106,6 +106,11 @@ public:
/// Get storage snapshot, throws exception if function node is not resolved
const StorageSnapshotPtr & getStorageSnapshot() const;
const std::vector<size_t> & getUnresolvedArgumentIndexes() const
{
return unresolved_arguments_indexes;
}
/// Return true if table function node has table expression modifiers, false otherwise
bool hasTableExpressionModifiers() const
{
@ -164,6 +169,7 @@ private:
StoragePtr storage;
StorageID storage_id;
StorageSnapshotPtr storage_snapshot;
std::vector<size_t> unresolved_arguments_indexes;
std::optional<TableExpressionModifiers> table_expression_modifiers;
SettingsChanges settings_changes;

View File

@ -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);
}

View 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,

View File

@ -229,7 +229,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT
}
std::bernoulli_distribution sample(sample_probability);
if (unlikely(sample_probability > 0.0 && sample(thread_local_rng)))
if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng)))
{
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = size});
@ -413,7 +413,7 @@ void MemoryTracker::free(Int64 size)
}
std::bernoulli_distribution sample(sample_probability);
if (unlikely(sample_probability > 0.0 && sample(thread_local_rng)))
if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng)))
{
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = -size});
@ -534,6 +534,12 @@ void MemoryTracker::setOrRaiseProfilerLimit(Int64 value)
;
}
bool MemoryTracker::isSizeOkForSampling(UInt64 size) const
{
/// We can avoid comparison min_allocation_size_bytes with zero, because we cannot have 0 bytes allocation/deallocation
return ((max_allocation_size_bytes == 0 || size <= max_allocation_size_bytes) && size >= min_allocation_size_bytes);
}
bool canEnqueueBackgroundTask()
{
auto limit = background_memory_tracker.getSoftLimit();

View File

@ -67,6 +67,12 @@ private:
/// To randomly sample allocations and deallocations in trace_log.
double sample_probability = 0;
/// Randomly sample allocations only larger or equal to this size
UInt64 min_allocation_size_bytes = 0;
/// Randomly sample allocations only smaller or equal to this size
UInt64 max_allocation_size_bytes = 0;
/// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy).
/// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker.
std::atomic<MemoryTracker *> parent {};
@ -88,6 +94,8 @@ private:
void setOrRaiseProfilerLimit(Int64 value);
bool isSizeOkForSampling(UInt64 size) const;
/// allocImpl(...) and free(...) should not be used directly
friend struct CurrentMemoryTracker;
void allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr);
@ -166,6 +174,16 @@ public:
sample_probability = value;
}
void setSampleMinAllocationSize(UInt64 value)
{
min_allocation_size_bytes = value;
}
void setSampleMaxAllocationSize(UInt64 value)
{
max_allocation_size_bytes = value;
}
void setProfilerStep(Int64 value)
{
profiler_step = value;

View File

@ -1,7 +1,6 @@
#if defined(__ELF__) && !defined(OS_FREEBSD)
#include <Common/SymbolIndex.h>
#include <base/hex.h>
#include <algorithm>
#include <optional>
@ -62,9 +61,11 @@ Otherwise you will get only exported symbols from program headers.
#endif
#define __msan_unpoison_string(X) // NOLINT
#define __msan_unpoison(X, Y) // NOLINT
#if defined(ch_has_feature)
# if ch_has_feature(memory_sanitizer)
# undef __msan_unpoison_string
# undef __msan_unpoison
# include <sanitizer/msan_interface.h>
# endif
#endif
@ -98,10 +99,13 @@ void collectSymbolsFromProgramHeaders(
/* Iterate over all headers of the current shared lib
* (first call is for the executable itself)
*/
__msan_unpoison(&info->dlpi_phnum, sizeof(info->dlpi_phnum));
__msan_unpoison(&info->dlpi_phdr, sizeof(info->dlpi_phdr));
for (size_t header_index = 0; header_index < info->dlpi_phnum; ++header_index)
{
/* Further processing is only needed if the dynamic section is reached
*/
__msan_unpoison(&info->dlpi_phdr[header_index], sizeof(info->dlpi_phdr[header_index]));
if (info->dlpi_phdr[header_index].p_type != PT_DYNAMIC)
continue;
@ -109,6 +113,7 @@ void collectSymbolsFromProgramHeaders(
* It's address is the shared lib's address + the virtual address
*/
const ElfW(Dyn) * dyn_begin = reinterpret_cast<const ElfW(Dyn) *>(info->dlpi_addr + info->dlpi_phdr[header_index].p_vaddr);
__msan_unpoison(&dyn_begin, sizeof(dyn_begin));
/// For unknown reason, addresses are sometimes relative sometimes absolute.
auto correct_address = [](ElfW(Addr) base, ElfW(Addr) ptr)
@ -122,44 +127,53 @@ void collectSymbolsFromProgramHeaders(
*/
size_t sym_cnt = 0;
for (const auto * it = dyn_begin; it->d_tag != DT_NULL; ++it)
{
ElfW(Addr) base_address = correct_address(info->dlpi_addr, it->d_un.d_ptr);
// TODO: this branch leads to invalid address of the hash table. Need further investigation.
// if (it->d_tag == DT_HASH)
// {
// const ElfW(Word) * hash = reinterpret_cast<const ElfW(Word) *>(base_address);
// sym_cnt = hash[1];
// break;
// }
if (it->d_tag == DT_GNU_HASH)
const auto * it = dyn_begin;
while (true)
{
/// This code based on Musl-libc.
__msan_unpoison(it, sizeof(*it));
if (it->d_tag != DT_NULL)
break;
const uint32_t * buckets = nullptr;
const uint32_t * hashval = nullptr;
ElfW(Addr) base_address = correct_address(info->dlpi_addr, it->d_un.d_ptr);
const ElfW(Word) * hash = reinterpret_cast<const ElfW(Word) *>(base_address);
buckets = hash + 4 + (hash[2] * sizeof(size_t) / 4);
for (ElfW(Word) i = 0; i < hash[0]; ++i)
if (buckets[i] > sym_cnt)
sym_cnt = buckets[i];
if (sym_cnt)
if (it->d_tag == DT_GNU_HASH)
{
sym_cnt -= hash[1];
hashval = buckets + hash[0] + sym_cnt;
do
/// This code based on Musl-libc.
const uint32_t * buckets = nullptr;
const uint32_t * hashval = nullptr;
const ElfW(Word) * hash = reinterpret_cast<const ElfW(Word) *>(base_address);
__msan_unpoison(&hash[0], sizeof(*hash));
__msan_unpoison(&hash[1], sizeof(*hash));
__msan_unpoison(&hash[2], sizeof(*hash));
buckets = hash + 4 + (hash[2] * sizeof(size_t) / 4);
__msan_unpoison(buckets, hash[0] * sizeof(buckets[0]));
for (ElfW(Word) i = 0; i < hash[0]; ++i)
if (buckets[i] > sym_cnt)
sym_cnt = buckets[i];
if (sym_cnt)
{
++sym_cnt;
sym_cnt -= hash[1];
hashval = buckets + hash[0] + sym_cnt;
__msan_unpoison(&hashval, sizeof(hashval));
do
{
++sym_cnt;
}
while (!(*hashval++ & 1));
}
while (!(*hashval++ & 1));
break;
}
break;
++it;
}
}
@ -190,6 +204,8 @@ void collectSymbolsFromProgramHeaders(
/* Get the pointer to the first entry of the symbol table */
const ElfW(Sym) * elf_sym = reinterpret_cast<const ElfW(Sym) *>(base_address);
__msan_unpoison(elf_sym, sym_cnt * sizeof(*elf_sym));
/* Iterate over the symbol table */
for (ElfW(Word) sym_index = 0; sym_index < ElfW(Word)(sym_cnt); ++sym_index)
{
@ -197,6 +213,7 @@ void collectSymbolsFromProgramHeaders(
* This is located at the address of st_name relative to the beginning of the string table.
*/
const char * sym_name = &strtab[elf_sym[sym_index].st_name];
__msan_unpoison_string(sym_name);
if (!sym_name)
continue;
@ -223,13 +240,18 @@ void collectSymbolsFromProgramHeaders(
#if !defined USE_MUSL
String getBuildIDFromProgramHeaders(dl_phdr_info * info)
{
__msan_unpoison(&info->dlpi_phnum, sizeof(info->dlpi_phnum));
__msan_unpoison(&info->dlpi_phdr, sizeof(info->dlpi_phdr));
for (size_t header_index = 0; header_index < info->dlpi_phnum; ++header_index)
{
const ElfPhdr & phdr = info->dlpi_phdr[header_index];
__msan_unpoison(&phdr, sizeof(phdr));
if (phdr.p_type != PT_NOTE)
continue;
return Elf::getBuildID(reinterpret_cast<const char *>(info->dlpi_addr + phdr.p_vaddr), phdr.p_memsz);
std::string_view view(reinterpret_cast<const char *>(info->dlpi_addr + phdr.p_vaddr), phdr.p_memsz);
__msan_unpoison(view.data(), view.size());
return Elf::getBuildID(view.data(), view.size());
}
return {};
}
@ -318,6 +340,7 @@ void collectSymbolsFromELF(
build_id = our_build_id;
#else
/// MSan does not know that the program segments in memory are initialized.
__msan_unpoison(info, sizeof(*info));
__msan_unpoison_string(info->dlpi_name);
object_name = info->dlpi_name;

View File

@ -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

View File

@ -83,8 +83,12 @@ namespace DB
M(UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \
M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \
M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \
M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0)
M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \
\
M(UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0) \
M(Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
M(UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \
M(UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0)
DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS)

View File

@ -427,7 +427,9 @@ class IColumn;
M(UInt64, memory_overcommit_ratio_denominator_for_user, 1_GiB, "It represents soft memory limit on the global level. This value is used to compute query overcommit ratio.", 0) \
M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \
M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \
M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
M(UInt64, memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \
M(UInt64, memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \
M(Bool, trace_profile_events, false, "Send to system.trace_log profile event and value of increment on each increment with 'ProfileEvent' trace_type", 0) \
\
M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown.", 0) \

View File

@ -37,6 +37,7 @@ public:
bool canBeInsideNullable() const override { return false; }
bool supportsSparseSerialization() const override { return true; }
bool canBeInsideSparseColumns() const override { return false; }
MutableColumnPtr createColumn() const override;
MutableColumnPtr createColumn(const ISerialization & serialization) const override;

View File

@ -110,6 +110,7 @@ public:
/// TODO: support more types.
virtual bool supportsSparseSerialization() const { return !haveSubtypes(); }
virtual bool canBeInsideSparseColumns() const { return supportsSparseSerialization(); }
SerializationPtr getDefaultSerialization() const;
SerializationPtr getSparseSerialization() const;

View File

@ -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);

View File

@ -16,11 +16,6 @@ namespace fs = std::filesystem;
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
CachedObjectStorage::CachedObjectStorage(
ObjectStoragePtr object_storage_,
FileCachePtr cache_,
@ -79,8 +74,6 @@ std::unique_ptr<ReadBufferFromFileBase> CachedObjectStorage::readObjects( /// NO
std::optional<size_t> read_hint,
std::optional<size_t> file_size) const
{
if (objects.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Received empty list of objects to read");
return object_storage->readObjects(objects, patchSettings(read_settings), read_hint, file_size);
}

View File

@ -361,7 +361,7 @@ ColumnPtr IExecutableFunction::execute(const ColumnsWithTypeAndName & arguments,
/// If default of sparse column is changed after execution of function, convert to full column.
/// If there are any default in non-zero position after execution of function, convert to full column.
/// Currently there is no easy way to rebuild sparse column with new offsets.
if (!result_type->supportsSparseSerialization() || !res->isDefaultAt(0) || res->getNumberOfDefaultRows() != 1)
if (!result_type->canBeInsideSparseColumns() || !res->isDefaultAt(0) || res->getNumberOfDefaultRows() != 1)
{
const auto & offsets_data = assert_cast<const ColumnVector<UInt64> &>(*sparse_offsets).getData();
return res->createWithOffsets(offsets_data, (*res)[0], input_rows_count, /*shift=*/ 1);

View File

@ -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))

View File

@ -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

View File

@ -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");

View File

@ -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();

View File

@ -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.

View File

@ -223,7 +223,10 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q
{
/// Set up memory profiling
thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step);
thread_group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability);
thread_group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size);
thread_group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size);
thread_group->performance_counters.setTraceProfileEvents(settings.trace_profile_events);
}

View File

@ -83,6 +83,8 @@ ThreadGroupPtr ThreadGroup::createForBackgroundProcess(ContextPtr storage_contex
const Settings & settings = storage_context->getSettingsRef();
group->memory_tracker.setProfilerStep(settings.memory_profiler_step);
group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability);
group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size);
group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size);
group->memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator);
group->memory_tracker.setParent(&background_memory_tracker);
if (settings.memory_tracker_fault_probability > 0.0)

View File

@ -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);
}
}

View File

@ -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); }

View File

@ -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();

View File

@ -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);

View File

@ -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));

View File

@ -340,8 +340,10 @@ MergeTreeReadTaskColumns getReadTaskColumns(
if (!columns_from_previous_steps.contains(name))
step_column_names.push_back(name);
injectRequiredColumns(
data_part_info_for_reader, storage_snapshot, with_subcolumns, step_column_names);
if (!step_column_names.empty())
injectRequiredColumns(
data_part_info_for_reader, storage_snapshot,
with_subcolumns, step_column_names);
/// More columns could have been added, filter them as well by the list of columns from previous steps.
Names columns_to_read_in_step;

View File

@ -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;

View File

@ -3311,7 +3311,7 @@ bool StorageReplicatedMergeTree::scheduleDataProcessingJob(BackgroundJobsAssigne
auto job_type = selected_entry->log_entry->type;
/// Depending on entry type execute in fetches (small) pool or big merge_mutate pool
if (job_type == LogEntry::GET_PART)
if (job_type == LogEntry::GET_PART || job_type == LogEntry::ATTACH_PART)
{
assignee.scheduleFetchTask(std::make_shared<ExecutableLambdaAdapter>(
[this, selected_entry] () mutable

View File

@ -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;

View File

@ -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;

View File

@ -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);

View File

@ -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;
};
}

View File

@ -16,6 +16,13 @@ NamesAndTypesList StorageSystemEvents::getNamesAndTypes()
};
}
NamesAndAliases StorageSystemEvents::getNamesAndAliases()
{
return {
{"name", std::make_shared<DataTypeString>(), "event"}
};
}
void StorageSystemEvents::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i)

View File

@ -17,6 +17,8 @@ public:
static NamesAndTypesList getNamesAndTypes();
static NamesAndAliases getNamesAndAliases();
protected:
using IStorageSystemOneBlock::IStorageSystemOneBlock;

View File

@ -17,6 +17,13 @@ NamesAndTypesList StorageSystemMetrics::getNamesAndTypes()
};
}
NamesAndAliases StorageSystemMetrics::getNamesAndAliases()
{
return {
{"name", std::make_shared<DataTypeString>(), "metric"}
};
}
void StorageSystemMetrics::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const
{
for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i)

View File

@ -18,6 +18,8 @@ public:
static NamesAndTypesList getNamesAndTypes();
static NamesAndAliases getNamesAndAliases();
protected:
using IStorageSystemOneBlock::IStorageSystemOneBlock;

View File

@ -130,7 +130,7 @@ public:
return true;
}
void visitImpl(QueryTreeNodePtr & node)
void enterImpl(QueryTreeNodePtr & node)
{
auto * function_node = node->as<FunctionNode>();
auto * join_node = node->as<JoinNode>();

View File

@ -35,6 +35,11 @@ from version_helper import (
get_version_from_repo,
update_version_local,
)
from clickhouse_helper import (
ClickHouseHelper,
prepare_tests_results_for_clickhouse,
)
from stopwatch import Stopwatch
IMAGE_NAME = "clickhouse/binary-builder"
BUILD_LOG_NAME = "build_log.log"
@ -268,6 +273,7 @@ def mark_failed_reports_pending(build_name: str, pr_info: PRInfo) -> None:
def main():
logging.basicConfig(level=logging.INFO)
stopwatch = Stopwatch()
build_name = sys.argv[1]
build_config = CI_CONFIG["build_config"][build_name]
@ -394,7 +400,20 @@ def main():
)
upload_master_static_binaries(pr_info, build_config, s3_helper, build_output_path)
# Fail build job if not successeded
ch_helper = ClickHouseHelper()
prepared_events = prepare_tests_results_for_clickhouse(
pr_info,
[],
"success" if success else "failure",
stopwatch.duration_seconds,
stopwatch.start_time_str,
log_url,
f"Build ({build_name})",
)
ch_helper.insert_events_into(db="default", table="checks", events=prepared_events)
# Fail the build job if it didn't succeed
if not success:
sys.exit(1)

View File

@ -19,16 +19,6 @@ CI_CONFIG = {
"with_coverage": False,
"comment": "",
},
"coverity": {
"compiler": "clang-16",
"build_type": "",
"sanitizer": "",
"package_type": "coverity",
"tidy": "disable",
"with_coverage": False,
"official": False,
"comment": "A special build for coverity",
},
"package_aarch64": {
"compiler": "clang-16-aarch64",
"build_type": "",
@ -187,7 +177,6 @@ CI_CONFIG = {
"builds_report_config": {
"ClickHouse build check": [
"package_release",
"coverity",
"package_aarch64",
"package_asan",
"package_ubsan",

View File

@ -132,6 +132,23 @@ class ClickHouseHelper:
return result
# Obtain the machine type from IMDS:
def get_instance_type():
url = "http://169.254.169.254/latest/meta-data/instance-type"
for i in range(5):
try:
response = requests.get(url, timeout=1)
if response.status_code == 200:
return response.text
except Exception as e:
error = (
f"Received exception while sending data to {url} on {i} attempt: {e}"
)
logging.warning(error)
continue
return ""
def prepare_tests_results_for_clickhouse(
pr_info: PRInfo,
test_results: TestResults,
@ -168,6 +185,7 @@ def prepare_tests_results_for_clickhouse(
head_ref=head_ref,
head_repo=head_repo,
task_url=pr_info.task_url,
instance_type=get_instance_type(),
)
# Always publish a total record for all checks. For checks with individual

View File

@ -129,7 +129,7 @@ position с конца строки.
Атомарно удаляет таблицу перед созданием новой, если такая была.
## * Приведение типов для IN (subquery).
## + Приведение типов для IN (subquery).
`SELECT 1 IN (SELECT -1 UNION ALL SELECT 1)`
@ -205,12 +205,12 @@ https://clickhouse.com/docs/en/operations/table_engines/external_data/
## Возможность задавать параметры соединений для табличных функций, движков таблиц и для реплик из отдельных разделов конфигурации.
## Настройка rollup_use_nulls.
## + Настройка rollup_use_nulls.
Upd: it is named "group_by_use_nulls".
## + Настройка cast_keep_nullable.
## Функция bitEquals для сравнения произвольных типов данных побитово.
## Функция bitEquals для сравнения произвольных типов данных побитово
## Функция serialize для implementation specific non portable non backwards compatible сериализации любого типа данных в набор байт.
## Функция bitEquals и оператор <=>.

View File

@ -218,22 +218,32 @@ def test_delete_race_leftovers(cluster):
time.sleep(5)
# Check that we correctly deleted all outdated parts and no leftovers on s3
known_remote_paths = set(
node.query(
f"SELECT remote_path FROM system.remote_data_paths WHERE disk_name = 's32'"
).splitlines()
)
all_remote_paths = set(
obj.object_name
for obj in cluster.minio_client.list_objects(
cluster.minio_bucket, "data2/", recursive=True
# Do it with retries because we delete blobs in the background
# and it can be race condition between removing from remote_data_paths and deleting blobs
all_remote_paths = set()
known_remote_paths = set()
for i in range(3):
known_remote_paths = set(
node.query(
f"SELECT remote_path FROM system.remote_data_paths WHERE disk_name = 's32'"
).splitlines()
)
)
# Some blobs can be deleted after we listed remote_data_paths
# It's alright, thus we check only that all remote paths are known
# (in other words, all remote paths is subset of known paths)
all_remote_paths = set(
obj.object_name
for obj in cluster.minio_client.list_objects(
cluster.minio_bucket, "data2/", recursive=True
)
)
# Some blobs can be deleted after we listed remote_data_paths
# It's alright, thus we check only that all remote paths are known
# (in other words, all remote paths is subset of known paths)
if all_remote_paths == {p for p in known_remote_paths if p in all_remote_paths}:
break
time.sleep(1)
assert all_remote_paths == {p for p in known_remote_paths if p in all_remote_paths}
# Check that we have all data

View File

@ -0,0 +1 @@
#!/usr/bin/env python3

View File

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

View File

@ -0,0 +1,5 @@
<clickhouse>
<total_memory_tracker_sample_probability>1</total_memory_tracker_sample_probability>
<total_memory_profiler_sample_min_allocation_size>4096</total_memory_profiler_sample_min_allocation_size>
<total_memory_profiler_sample_max_allocation_size>8192</total_memory_profiler_sample_max_allocation_size>
</clickhouse>

View File

@ -0,0 +1,40 @@
from helpers.cluster import ClickHouseCluster
import pytest
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
"node",
main_configs=["configs/memory_profiler.xml"],
user_configs=["configs/max_untracked_memory.xml"],
)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_trace_boundaries_work(started_cluster):
if node.is_built_with_sanitizer():
pytest.skip("Disabled for sanitizers")
node.query("select randomPrintableASCII(number) from numbers(1000) FORMAT Null")
node.query("SYSTEM FLUSH LOGS")
assert (
node.query(
"SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where trace_type = 'MemorySample'"
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM system.trace_log where trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)"
)
== "0\n"
)

View File

@ -262,7 +262,8 @@ CREATE TABLE system.events
(
`event` String,
`value` UInt64,
`description` String
`description` String,
`name` String
)
ENGINE = SystemEvents
COMMENT 'SYSTEM TABLE is built on the fly.'
@ -383,7 +384,8 @@ CREATE TABLE system.metrics
(
`metric` String,
`value` Int64,
`description` String
`description` String,
`name` String
)
ENGINE = SystemMetrics
COMMENT 'SYSTEM TABLE is built on the fly.'

View File

@ -20,7 +20,9 @@ function insert()
-H "tracestate: $4" \
"${CLICKHOUSE_URL}" \
--data @-
${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH DISTRIBUTED ${CLICKHOUSE_DATABASE}.dist_opentelemetry"
# disable probabilistic tracing to avoid stealing the trace context
${CLICKHOUSE_CLIENT} --opentelemetry_start_trace_probability=0 -q "SYSTEM FLUSH DISTRIBUTED ${CLICKHOUSE_DATABASE}.dist_opentelemetry"
}
function check_span()
@ -69,6 +71,8 @@ DROP TABLE IF EXISTS ${CLICKHOUSE_DATABASE}.local_opentelemetry;
CREATE TABLE ${CLICKHOUSE_DATABASE}.dist_opentelemetry (key UInt64) Engine=Distributed('test_cluster_two_shards_localhost', ${CLICKHOUSE_DATABASE}, local_opentelemetry, key % 2);
CREATE TABLE ${CLICKHOUSE_DATABASE}.local_opentelemetry (key UInt64) Engine=MergeTree ORDER BY key;
SYSTEM STOP DISTRIBUTED SENDS ${CLICKHOUSE_DATABASE}.dist_opentelemetry;
"
#

View File

@ -1,3 +1,6 @@
1 []
1 []
[]
1 []
1 []
[]

View File

@ -5,23 +5,26 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} -n --query "
DROP TABLE IF EXISTS test;
CREATE TABLE test (id Int32, empty Array(Int32))
ENGINE=MergeTree ORDER BY id
SETTINGS min_rows_for_wide_part=0, min_bytes_for_wide_part=0, disk='s3_disk';
for DISK in s3_disk s3_cache
do
${CLICKHOUSE_CLIENT} -n --query "
DROP TABLE IF EXISTS test;
CREATE TABLE test (id Int32, empty Array(Int32))
ENGINE=MergeTree ORDER BY id
SETTINGS min_rows_for_wide_part=0, min_bytes_for_wide_part=0, disk='$DISK';
INSERT INTO test (id) VALUES (1);
SELECT * FROM test;
"
INSERT INTO test (id) VALUES (1);
SELECT * FROM test;
"
${CLICKHOUSE_CLIENT} -n --query "
BACKUP TABLE test TO Disk('backups', 'test_s3_backup');
DROP TABLE test;
RESTORE TABLE test FROM Disk('backups', 'test_s3_backup');
" &>/dev/null
${CLICKHOUSE_CLIENT} -n --query "
BACKUP TABLE test TO Disk('backups', 'test_s3_backup');
DROP TABLE test;
RESTORE TABLE test FROM Disk('backups', 'test_s3_backup');
" &>/dev/null
${CLICKHOUSE_CLIENT} -n --query "
SELECT * FROM test;
SELECT empty FROM test;
"
${CLICKHOUSE_CLIENT} -n --query "
SELECT * FROM test;
SELECT empty FROM test;
"
done

View File

@ -0,0 +1,18 @@
#!/usr/bin/env bash
# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-cpu-aarch64, no-random-settings
# requires TraceCollector, does not available under sanitizers and aarch64
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
query_id="${CLICKHOUSE_DATABASE}_min_max_allocation_size_$RANDOM$RANDOM"
${CLICKHOUSE_CLIENT} --query_id="$query_id" --memory_profiler_sample_min_allocation_size=4096 --memory_profiler_sample_max_allocation_size=8192 --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 --query "select randomPrintableASCII(number) from numbers(1000) FORMAT Null"
${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS"
# at least something allocated
${CLICKHOUSE_CLIENT} --query "SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample'"
# show wrong allocations
${CLICKHOUSE_CLIENT} --query "SELECT abs(size) FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)"

View File

@ -0,0 +1,2 @@
10496500
4

View File

@ -0,0 +1,25 @@
-- Tags: no-parallel, no-random-settings, no-random-merge-tree-settings, no-s3-storage
drop table if exists t_multi_prewhere;
drop row policy if exists policy_02834 on t_multi_prewhere;
create table t_multi_prewhere (a UInt64, b UInt64, c UInt8)
engine = MergeTree order by tuple()
settings min_bytes_for_wide_part = 0;
create row policy policy_02834 on t_multi_prewhere using a > 2000 as permissive to all;
insert into t_multi_prewhere select number, number, number from numbers(10000);
system drop mark cache;
select sum(b) from t_multi_prewhere prewhere a < 5000;
system flush logs;
select ProfileEvents['FileOpen'] from system.query_log
where
type = 'QueryFinish'
and current_database = currentDatabase()
and query ilike '%select sum(b) from t_multi_prewhere prewhere a < 5000%';
drop table if exists t_multi_prewhere;
drop row policy if exists policy_02834 on t_multi_prewhere;

View File

@ -0,0 +1,4 @@
(0,0)
(0,0)
(0,1)
(0,NULL)

View File

@ -0,0 +1,14 @@
drop table if exists t_tuple_sparse;
create table t_tuple_sparse (a UInt64, b UInt64)
ENGINE = MergeTree ORDER BY tuple()
SETTINGS ratio_of_defaults_for_sparse_serialization = 0.0;
insert into t_tuple_sparse values (0, 0);
select (a, b) from t_tuple_sparse;
select (a, 0) from t_tuple_sparse;
select (a, 1) from t_tuple_sparse;
select (a, NULL) from t_tuple_sparse;
drop table if exists t_tuple_sparse;

View File

@ -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."