mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 10:02:01 +00:00
Merge remote-tracking branch 'origin/less-flaky-test_start_stop_moves' into many-fixes-3
This commit is contained in:
commit
1fa21700ad
@ -2297,7 +2297,9 @@ void ClientBase::runInteractive()
|
||||
catch (const ErrnoException & e)
|
||||
{
|
||||
if (e.getErrno() != EEXIST)
|
||||
throw;
|
||||
{
|
||||
std::cerr << getCurrentExceptionMessage(false) << '\n';
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -60,7 +60,15 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati
|
||||
quota_key = config.getString("quota_key", "");
|
||||
|
||||
/// By default compression is disabled if address looks like localhost.
|
||||
compression = config.getBool("compression", !isLocalAddress(DNSResolver::instance().resolveHost(host)))
|
||||
|
||||
/// Avoid DNS request if the host is "localhost".
|
||||
/// If ClickHouse is run under QEMU-user with a binary for a different architecture,
|
||||
/// and there are all listed startup dependency shared libraries available, but not the runtime dependencies of glibc,
|
||||
/// the glibc cannot open "plugins" for DNS resolving, and the DNS resolution does not work.
|
||||
/// At the same time, I want clickhouse-local to always work, regardless.
|
||||
/// TODO: get rid of glibc, or replace getaddrinfo to c-ares.
|
||||
|
||||
compression = config.getBool("compression", host != "localhost" && !isLocalAddress(DNSResolver::instance().resolveHost(host)))
|
||||
? Protocol::Compression::Enable : Protocol::Compression::Disable;
|
||||
|
||||
timeouts = ConnectionTimeouts(
|
||||
|
@ -540,7 +540,7 @@ bool OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject, si
|
||||
}
|
||||
}
|
||||
|
||||
return re2->Match(StringPieceType(subject, subject_size), 0, subject_size, RegexType::UNANCHORED, nullptr, 0);
|
||||
return re2->Match({subject, subject_size}, 0, subject_size, RegexType::UNANCHORED, nullptr, 0);
|
||||
}
|
||||
}
|
||||
|
||||
@ -585,9 +585,9 @@ bool OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject, si
|
||||
return false;
|
||||
}
|
||||
|
||||
StringPieceType piece;
|
||||
std::string_view piece;
|
||||
|
||||
if (!RegexType::PartialMatch(StringPieceType(subject, subject_size), *re2, &piece))
|
||||
if (!RegexType::PartialMatch({subject, subject_size}, *re2, &piece))
|
||||
return false;
|
||||
else
|
||||
{
|
||||
@ -652,10 +652,10 @@ unsigned OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject
|
||||
return 0;
|
||||
}
|
||||
|
||||
DB::PODArrayWithStackMemory<StringPieceType, 128> pieces(limit);
|
||||
DB::PODArrayWithStackMemory<std::string_view, 128> pieces(limit);
|
||||
|
||||
if (!re2->Match(
|
||||
StringPieceType(subject, subject_size),
|
||||
{subject, subject_size},
|
||||
0,
|
||||
subject_size,
|
||||
RegexType::UNANCHORED,
|
||||
|
@ -52,7 +52,6 @@ public:
|
||||
using MatchVec = std::vector<Match>;
|
||||
|
||||
using RegexType = std::conditional_t<thread_safe, re2::RE2, re2_st::RE2>;
|
||||
using StringPieceType = std::conditional_t<thread_safe, re2::StringPiece, re2_st::StringPiece>;
|
||||
|
||||
OptimizedRegularExpressionImpl(const std::string & regexp_, int options = 0); /// NOLINT
|
||||
/// StringSearcher store pointers to required_substring, it must be updated on move.
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <atomic>
|
||||
|
||||
#include <re2/re2.h>
|
||||
#include <re2/stringpiece.h>
|
||||
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
@ -44,7 +43,7 @@ private:
|
||||
const std::string regexp_string;
|
||||
|
||||
const RE2 regexp;
|
||||
const re2::StringPiece replacement;
|
||||
const std::string_view replacement;
|
||||
|
||||
#ifndef NDEBUG
|
||||
mutable std::atomic<std::uint64_t> matches_count = 0;
|
||||
|
@ -27,7 +27,7 @@ static thread_local size_t max_stack_size = 0;
|
||||
* @param out_address - if not nullptr, here the address of the stack will be written.
|
||||
* @return stack size
|
||||
*/
|
||||
size_t getStackSize(void ** out_address)
|
||||
static size_t getStackSize(void ** out_address)
|
||||
{
|
||||
using namespace DB;
|
||||
|
||||
@ -54,7 +54,15 @@ size_t getStackSize(void ** out_address)
|
||||
throwFromErrno("Cannot pthread_attr_get_np", ErrorCodes::CANNOT_PTHREAD_ATTR);
|
||||
# else
|
||||
if (0 != pthread_getattr_np(pthread_self(), &attr))
|
||||
throwFromErrno("Cannot pthread_getattr_np", ErrorCodes::CANNOT_PTHREAD_ATTR);
|
||||
{
|
||||
if (errno == ENOENT)
|
||||
{
|
||||
/// Most likely procfs is not mounted.
|
||||
return 0;
|
||||
}
|
||||
else
|
||||
throwFromErrno("Cannot pthread_getattr_np", ErrorCodes::CANNOT_PTHREAD_ATTR);
|
||||
}
|
||||
# endif
|
||||
|
||||
SCOPE_EXIT({ pthread_attr_destroy(&attr); });
|
||||
@ -83,6 +91,10 @@ __attribute__((__weak__)) void checkStackSize()
|
||||
if (!stack_address)
|
||||
max_stack_size = getStackSize(&stack_address);
|
||||
|
||||
/// The check is impossible.
|
||||
if (!max_stack_size)
|
||||
return;
|
||||
|
||||
const void * frame_address = __builtin_frame_address(0);
|
||||
uintptr_t int_frame_address = reinterpret_cast<uintptr_t>(frame_address);
|
||||
uintptr_t int_stack_address = reinterpret_cast<uintptr_t>(stack_address);
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <re2/re2.h>
|
||||
#include <re2/stringpiece.h>
|
||||
#include <algorithm>
|
||||
#include <sstream>
|
||||
#include <iomanip>
|
||||
@ -33,14 +32,14 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob
|
||||
std::string escaped_with_globs = buf_for_escaping.str();
|
||||
|
||||
static const re2::RE2 enum_or_range(R"({([\d]+\.\.[\d]+|[^{}*,]+,[^{}*]*[^{}*,])})"); /// regexp for {expr1,expr2,expr3} or {M..N}, where M and N - non-negative integers, expr's should be without "{", "}", "*" and ","
|
||||
re2::StringPiece input(escaped_with_globs);
|
||||
re2::StringPiece matched;
|
||||
std::string_view input(escaped_with_globs);
|
||||
std::string_view matched;
|
||||
std::ostringstream oss_for_replacing; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
oss_for_replacing.exceptions(std::ios::failbit);
|
||||
size_t current_index = 0;
|
||||
while (RE2::FindAndConsume(&input, enum_or_range, &matched))
|
||||
{
|
||||
std::string buffer{matched};
|
||||
std::string buffer(matched);
|
||||
oss_for_replacing << escaped_with_globs.substr(current_index, matched.data() - escaped_with_globs.data() - current_index - 1) << '(';
|
||||
|
||||
if (buffer.find(',') == std::string::npos)
|
||||
|
@ -30,8 +30,6 @@
|
||||
#include <Dictionaries/RegExpTreeDictionary.h>
|
||||
#include <Dictionaries/YAMLRegExpTreeDictionarySource.h>
|
||||
|
||||
#include <re2_st/stringpiece.h>
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#if USE_VECTORSCAN
|
||||
@ -469,10 +467,9 @@ public:
|
||||
|
||||
std::pair<String, bool> processBackRefs(const String & data, const re2_st::RE2 & searcher, const std::vector<StringPiece> & pieces)
|
||||
{
|
||||
re2_st::StringPiece haystack(data.data(), data.size());
|
||||
re2_st::StringPiece matches[10];
|
||||
std::string_view matches[10];
|
||||
String result;
|
||||
searcher.Match(haystack, 0, data.size(), re2_st::RE2::Anchor::UNANCHORED, matches, 10);
|
||||
searcher.Match({data.data(), data.size()}, 0, data.size(), re2_st::RE2::Anchor::UNANCHORED, matches, 10);
|
||||
/// if the pattern is a single '$1' but fails to match, we would use the default value.
|
||||
if (pieces.size() == 1 && pieces[0].ref_num >= 0 && pieces[0].ref_num < 10 && matches[pieces[0].ref_num].empty())
|
||||
return std::make_pair(result, true);
|
||||
|
@ -99,8 +99,8 @@ struct ReplaceRegexpImpl
|
||||
int num_captures,
|
||||
const Instructions & instructions)
|
||||
{
|
||||
re2_st::StringPiece haystack(haystack_data, haystack_length);
|
||||
re2_st::StringPiece matches[max_captures];
|
||||
std::string_view haystack(haystack_data, haystack_length);
|
||||
std::string_view matches[max_captures];
|
||||
|
||||
size_t copy_pos = 0;
|
||||
size_t match_pos = 0;
|
||||
|
@ -45,8 +45,8 @@ bool isLargerThanFifty(std::string_view str)
|
||||
/// Check for sub-patterns of the form x{n} or x{n,} can be expensive. Ignore spaces before/after n and m.
|
||||
bool SlowWithHyperscanChecker::isSlowOneRepeat(std::string_view regexp)
|
||||
{
|
||||
re2_st::StringPiece haystack(regexp.data(), regexp.size());
|
||||
re2_st::StringPiece matches[2];
|
||||
std::string_view haystack(regexp.data(), regexp.size());
|
||||
std::string_view matches[2];
|
||||
size_t start_pos = 0;
|
||||
while (start_pos < haystack.size())
|
||||
{
|
||||
@ -67,8 +67,8 @@ bool SlowWithHyperscanChecker::isSlowOneRepeat(std::string_view regexp)
|
||||
/// Check if sub-patterns of the form x{n,m} can be expensive. Ignore spaces before/after n and m.
|
||||
bool SlowWithHyperscanChecker::isSlowTwoRepeats(std::string_view regexp)
|
||||
{
|
||||
re2_st::StringPiece haystack(regexp.data(), regexp.size());
|
||||
re2_st::StringPiece matches[3];
|
||||
std::string_view haystack(regexp.data(), regexp.size());
|
||||
std::string_view matches[3];
|
||||
size_t start_pos = 0;
|
||||
while (start_pos < haystack.size())
|
||||
{
|
||||
|
@ -94,7 +94,6 @@ public:
|
||||
if (needle.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Length of 'needle' argument must be greater than 0.");
|
||||
|
||||
using StringPiece = typename Regexps::Regexp::StringPieceType;
|
||||
const Regexps::Regexp holder = Regexps::createRegexp<false, false, false>(needle);
|
||||
const auto & regexp = holder.getRE2();
|
||||
|
||||
@ -111,7 +110,7 @@ public:
|
||||
groups_count, std::to_string(MAX_GROUPS_COUNT - 1));
|
||||
|
||||
// Including 0-group, which is the whole regexp.
|
||||
PODArrayWithStackMemory<StringPiece, MAX_GROUPS_COUNT> matched_groups(groups_count + 1);
|
||||
PODArrayWithStackMemory<std::string_view, MAX_GROUPS_COUNT> matched_groups(groups_count + 1);
|
||||
|
||||
ColumnArray::ColumnOffsets::MutablePtr root_offsets_col = ColumnArray::ColumnOffsets::create();
|
||||
ColumnArray::ColumnOffsets::MutablePtr nested_offsets_col = ColumnArray::ColumnOffsets::create();
|
||||
@ -160,7 +159,7 @@ public:
|
||||
/// Additional limit to fail fast on supposedly incorrect usage.
|
||||
const auto max_matches_per_row = context->getSettingsRef().regexp_max_matches_per_row;
|
||||
|
||||
PODArray<StringPiece, 0> all_matches;
|
||||
PODArray<std::string_view, 0> all_matches;
|
||||
/// Number of times RE matched on each row of haystack column.
|
||||
PODArray<size_t, 0> number_of_matches_per_row;
|
||||
|
||||
|
@ -75,7 +75,7 @@ public:
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "There are no groups in regexp: {}", needle);
|
||||
|
||||
// Including 0-group, which is the whole regexp.
|
||||
PODArrayWithStackMemory<re2_st::StringPiece, 128> matched_groups(groups_count + 1);
|
||||
PODArrayWithStackMemory<std::string_view, 128> matched_groups(groups_count + 1);
|
||||
|
||||
ColumnArray::ColumnOffsets::MutablePtr offsets_col = ColumnArray::ColumnOffsets::create();
|
||||
ColumnString::MutablePtr data_col = ColumnString::create();
|
||||
@ -89,7 +89,7 @@ public:
|
||||
{
|
||||
std::string_view current_row = column_haystack->getDataAt(i).toView();
|
||||
|
||||
if (re2->Match(re2_st::StringPiece(current_row.data(), current_row.size()),
|
||||
if (re2->Match({current_row.data(), current_row.size()},
|
||||
0, current_row.size(), re2_st::RE2::UNANCHORED, matched_groups.data(),
|
||||
static_cast<int>(matched_groups.size())))
|
||||
{
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <re2_st/re2.h>
|
||||
#include <re2_st/stringpiece.h>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
#include <Core/Block.h>
|
||||
@ -28,14 +27,14 @@ public:
|
||||
/// Return true if row was successfully parsed and row fields were extracted.
|
||||
bool parseRow(PeekableReadBuffer & buf);
|
||||
|
||||
re2_st::StringPiece getField(size_t index) { return matched_fields[index]; }
|
||||
std::string_view getField(size_t index) { return matched_fields[index]; }
|
||||
size_t getMatchedFieldsSize() const { return matched_fields.size(); }
|
||||
size_t getNumberOfGroups() const { return regexp.NumberOfCapturingGroups(); }
|
||||
|
||||
private:
|
||||
const re2_st::RE2 regexp;
|
||||
// The vector of fields extracted from line using regexp.
|
||||
std::vector<re2_st::StringPiece> matched_fields;
|
||||
std::vector<std::string_view> matched_fields;
|
||||
// These two vectors are needed to use RE2::FullMatchN (function for extracting fields).
|
||||
std::vector<re2_st::RE2::Arg> re2_arguments;
|
||||
std::vector<re2_st::RE2::Arg *> re2_arguments_ptrs;
|
||||
|
@ -44,6 +44,8 @@
|
||||
#include <Poco/String.h>
|
||||
#include <Poco/Net/SocketAddress.h>
|
||||
|
||||
#include <re2/re2.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <sstream>
|
||||
|
||||
@ -1163,8 +1165,8 @@ void PredefinedQueryHandler::customizeContext(HTTPServerRequest & request, Conte
|
||||
{
|
||||
int num_captures = compiled_regex->NumberOfCapturingGroups() + 1;
|
||||
|
||||
re2::StringPiece matches[num_captures];
|
||||
re2::StringPiece input(begin, end - begin);
|
||||
std::string_view matches[num_captures];
|
||||
std::string_view input(begin, end - begin);
|
||||
if (compiled_regex->Match(input, 0, end - begin, re2::RE2::Anchor::ANCHOR_BOTH, matches, num_captures))
|
||||
{
|
||||
for (const auto & [capturing_name, capturing_index] : compiled_regex->NamedCapturingGroups())
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <base/find_symbols.h>
|
||||
|
||||
#include <re2/re2.h>
|
||||
#include <re2/stringpiece.h>
|
||||
#include <Poco/StringTokenizer.h>
|
||||
#include <Poco/Util/LayeredConfiguration.h>
|
||||
|
||||
@ -26,9 +25,8 @@ static inline bool checkRegexExpression(std::string_view match_str, const Compil
|
||||
{
|
||||
int num_captures = compiled_regex->NumberOfCapturingGroups() + 1;
|
||||
|
||||
re2::StringPiece matches[num_captures];
|
||||
re2::StringPiece match_input(match_str.data(), match_str.size());
|
||||
return compiled_regex->Match(match_input, 0, match_str.size(), re2::RE2::Anchor::ANCHOR_BOTH, matches, num_captures);
|
||||
std::string_view matches[num_captures];
|
||||
return compiled_regex->Match({match_str.data(), match_str.size()}, 0, match_str.size(), re2::RE2::Anchor::ANCHOR_BOTH, matches, num_captures);
|
||||
}
|
||||
|
||||
static inline bool checkExpression(std::string_view match_str, const std::pair<String, CompiledRegexPtr> & expression)
|
||||
|
@ -7196,7 +7196,10 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage(
|
||||
if (query_context->canUseParallelReplicasOnInitiator() && to_stage >= QueryProcessingStage::WithMergeableState)
|
||||
{
|
||||
if (!canUseParallelReplicasBasedOnPKAnalysis(query_context, storage_snapshot, query_info))
|
||||
{
|
||||
query_info.parallel_replicas_disabled = true;
|
||||
return QueryProcessingStage::Enum::FetchColumns;
|
||||
}
|
||||
|
||||
/// ReplicatedMergeTree
|
||||
if (supportsReplication())
|
||||
|
@ -255,6 +255,8 @@ struct SelectQueryInfo
|
||||
Block minmax_count_projection_block;
|
||||
MergeTreeDataSelectAnalysisResultPtr merge_tree_select_result_ptr;
|
||||
|
||||
bool parallel_replicas_disabled = false;
|
||||
|
||||
bool is_parameterized_view = false;
|
||||
NameToNameMap parameterized_view_values;
|
||||
|
||||
|
@ -209,7 +209,9 @@ void StorageMergeTree::read(
|
||||
size_t max_block_size,
|
||||
size_t num_streams)
|
||||
{
|
||||
if (local_context->canUseParallelReplicasOnInitiator() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree)
|
||||
if (!query_info.parallel_replicas_disabled &&
|
||||
local_context->canUseParallelReplicasOnInitiator() &&
|
||||
local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree)
|
||||
{
|
||||
auto table_id = getStorageID();
|
||||
|
||||
@ -240,7 +242,10 @@ void StorageMergeTree::read(
|
||||
}
|
||||
else
|
||||
{
|
||||
const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree;
|
||||
const bool enable_parallel_reading =
|
||||
!query_info.parallel_replicas_disabled &&
|
||||
local_context->canUseParallelReplicasOnFollower() &&
|
||||
local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree;
|
||||
|
||||
if (auto plan = reader.read(
|
||||
column_names, storage_snapshot, query_info,
|
||||
|
@ -711,15 +711,16 @@ def test_background_move(start_cluster, name, engine):
|
||||
s1 String
|
||||
) ENGINE = {engine}
|
||||
ORDER BY tuple()
|
||||
SETTINGS storage_policy='moving_jbod_with_external'
|
||||
SETTINGS storage_policy='moving_jbod_with_external', max_replicated_merges_in_queue=0
|
||||
"""
|
||||
)
|
||||
|
||||
node1.query(f"SYSTEM STOP MERGES {name}")
|
||||
|
||||
first_part = None
|
||||
for i in range(5):
|
||||
data = [] # 5MB in total
|
||||
for i in range(5):
|
||||
for _ in range(5):
|
||||
data.append(get_random_string(1024 * 1024)) # 1MB row
|
||||
# small jbod size is 40MB, so lets insert 5MB batch 5 times
|
||||
node1.query_with_retry(
|
||||
@ -728,9 +729,11 @@ def test_background_move(start_cluster, name, engine):
|
||||
)
|
||||
)
|
||||
|
||||
first_part = get_oldest_part(node1, name)
|
||||
# we are doing moves in parallel so we need to fetch the name of first part before we add new parts
|
||||
if i == 0:
|
||||
first_part = get_oldest_part(node1, name)
|
||||
|
||||
used_disks = get_used_disks_for_table(node1, name)
|
||||
assert first_part is not None
|
||||
|
||||
retry = 20
|
||||
i = 0
|
||||
@ -740,9 +743,6 @@ def test_background_move(start_cluster, name, engine):
|
||||
time.sleep(0.5)
|
||||
i += 1
|
||||
|
||||
used_disks = get_used_disks_for_table(node1, name)
|
||||
assert sum(1 for x in used_disks if x == "jbod1") <= 2
|
||||
|
||||
# first (oldest) part was moved to external
|
||||
assert get_disk_for_part(node1, name, first_part) == "external"
|
||||
|
||||
@ -779,7 +779,7 @@ def test_start_stop_moves(start_cluster, name, engine):
|
||||
s1 String
|
||||
) ENGINE = {engine}
|
||||
ORDER BY tuple()
|
||||
SETTINGS storage_policy='moving_jbod_with_external'
|
||||
SETTINGS storage_policy='moving_jbod_with_external', max_replicated_merges_in_queue=0
|
||||
"""
|
||||
)
|
||||
|
||||
@ -856,9 +856,6 @@ def test_start_stop_moves(start_cluster, name, engine):
|
||||
# first (oldest) part moved to external
|
||||
assert get_disk_for_part(node1, name, first_part) == "external"
|
||||
|
||||
used_disks = get_used_disks_for_table(node1, name)
|
||||
assert sum(1 for x in used_disks if x == "jbod1") <= 2
|
||||
|
||||
node1.query(f"SYSTEM START MERGES {name}")
|
||||
finally:
|
||||
node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC")
|
||||
|
@ -111,6 +111,23 @@ cat > /usr/local/hadoop/etc/hadoop/hdfs-site.xml << EOF
|
||||
<name>dfs.datanode.http.address</name>
|
||||
<value>0.0.0.0:1006</value>
|
||||
</property>
|
||||
<!-- If the port is 0 then the server will start on a free port. -->
|
||||
<property>
|
||||
<name>dfs.datanode.ipc.address</name>
|
||||
<value>0.0.0.0:0</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.namenode.secondary.http-address</name>
|
||||
<value>0.0.0.0:0</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.namenode.backup.address</name>
|
||||
<value>0.0.0.0:0</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.namenode.backup.http-address</name>
|
||||
<value>0.0.0.0:0</value>
|
||||
</property>
|
||||
<!--
|
||||
<property>
|
||||
<name>dfs.http.policy</name>
|
||||
|
@ -1,6 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
# Tags: no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
@ -12,12 +11,8 @@ $CLICKHOUSE_CLIENT --query="select toUInt64(pow(2, 62)) as value format JSON" --
|
||||
$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=1 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL"
|
||||
$CLICKHOUSE_CLIENT --readonly=1 --multiquery --query="set output_format_json_quote_64bit_integers=0 ; select toUInt64(pow(2, 63)) as value format JSON" --server_logs_file=/dev/null 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL"
|
||||
|
||||
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" | grep value
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" | grep value
|
||||
|
||||
#${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&session_timeout=3600" -d 'SET readonly = 1'
|
||||
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode.' && echo "OK" || echo "FAIL"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=1" 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=readonly&query=SELECT+toUInt64(pow(2,+63))+as+value+format+JSON&output_format_json_quote_64bit_integers=0" 2>&1 | grep -o -q 'value\|Cannot modify .* setting in readonly mode' && echo "OK" || echo "FAIL"
|
||||
|
||||
|
@ -3,8 +3,6 @@
|
||||
|
||||
set -ue
|
||||
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -1,8 +1,6 @@
|
||||
#!/usr/bin/env bash
|
||||
set -ue
|
||||
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -1,8 +1,6 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: race
|
||||
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -3,8 +3,6 @@ set -ue
|
||||
|
||||
# this test doesn't need 'current_database = currentDatabase()',
|
||||
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -1,8 +1,6 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: race
|
||||
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -3,8 +3,6 @@
|
||||
|
||||
# shellcheck disable=SC2154
|
||||
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -5,8 +5,6 @@
|
||||
|
||||
# shellcheck disable=SC2154
|
||||
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -2,8 +2,6 @@
|
||||
|
||||
# shellcheck disable=SC2154
|
||||
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -1,8 +1,6 @@
|
||||
#!/usr/bin/env bash
|
||||
set -ue
|
||||
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -1,7 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
@ -11,4 +9,4 @@ $CLICKHOUSE_CLIENT --query="SELECT * from format('TSV', '123')"
|
||||
|
||||
$CLICKHOUSE_CLIENT --readonly=1 --query="SELECT * from numbers(1)"
|
||||
$CLICKHOUSE_CLIENT --readonly=1 --query="SELECT * from format('TSV', '123')" 2>&1 | grep -Fq "Cannot execute query in readonly mode. (READONLY)" && echo 'ERROR' || echo 'OK'
|
||||
$CLICKHOUSE_CLIENT --readonly=1 --query="INSERT INTO FUNCTION null('x String') (x) FORMAT TSV '123'" 2>&1 | grep -Fq "Cannot execute query in readonly mode. (READONLY)" && echo 'ERROR' || echo 'OK'
|
||||
$CLICKHOUSE_CLIENT --readonly=1 --query="INSERT INTO FUNCTION null('x String') (x) FORMAT TSV '123'" 2>&1 | grep -Fq "Cannot execute query in readonly mode. (READONLY)" && echo 'ERROR' || echo 'OK'
|
||||
|
@ -3,8 +3,6 @@
|
||||
|
||||
# shellcheck disable=SC2154
|
||||
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -3,8 +3,6 @@
|
||||
|
||||
# shellcheck disable=SC2154
|
||||
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
@ -0,0 +1,4 @@
|
||||
-- count() ------------------------------
|
||||
2
|
||||
-- count() with parallel replicas -------
|
||||
2
|
@ -0,0 +1,24 @@
|
||||
DROP TABLE IF EXISTS users;
|
||||
CREATE TABLE users (uid Int16, name String, age Int16) ENGINE=MergeTree() ORDER BY uid;
|
||||
|
||||
INSERT INTO users VALUES (111, 'JFK', 33);
|
||||
INSERT INTO users VALUES (6666, 'KLM', 48);
|
||||
INSERT INTO users VALUES (88888, 'AMS', 50);
|
||||
|
||||
SELECT '-- count() ------------------------------';
|
||||
SELECT count() FROM users PREWHERE uid > 2000;
|
||||
|
||||
-- enable parallel replicas but with high granules threshold
|
||||
SET
|
||||
skip_unavailable_shards=1,
|
||||
allow_experimental_parallel_reading_from_replicas=1,
|
||||
max_parallel_replicas=3,
|
||||
use_hedged_requests=0,
|
||||
cluster_for_parallel_replicas='parallel_replicas',
|
||||
parallel_replicas_for_non_replicated_merge_tree=1,
|
||||
parallel_replicas_min_number_of_granules_to_enable=1000;
|
||||
|
||||
SELECT '-- count() with parallel replicas -------';
|
||||
SELECT count() FROM users PREWHERE uid > 2000;
|
||||
|
||||
DROP TABLE users;
|
@ -2,8 +2,6 @@
|
||||
|
||||
# shellcheck disable=SC2154
|
||||
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
Loading…
Reference in New Issue
Block a user