mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge pull request #11524 from ClickHouse/stack-frams-size-limit
Avoid too large stack frames
This commit is contained in:
commit
85f28db714
File diff suppressed because it is too large
Load Diff
@ -20,6 +20,12 @@ endif ()
|
|||||||
|
|
||||||
option (WEVERYTHING "Enables -Weverything option with some exceptions. This is intended for exploration of new compiler warnings that may be found to be useful. Only makes sense for clang." ON)
|
option (WEVERYTHING "Enables -Weverything option with some exceptions. This is intended for exploration of new compiler warnings that may be found to be useful. Only makes sense for clang." ON)
|
||||||
|
|
||||||
|
# Control maximum size of stack frames. It can be important if the code is run in fibers with small stack size.
|
||||||
|
# Only in release build because debug has too large stack frames.
|
||||||
|
if ((NOT CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") AND (NOT SANITIZE))
|
||||||
|
add_warning(frame-larger-than=16384)
|
||||||
|
endif ()
|
||||||
|
|
||||||
if (COMPILER_CLANG)
|
if (COMPILER_CLANG)
|
||||||
add_warning(pedantic)
|
add_warning(pedantic)
|
||||||
no_warning(vla-extension)
|
no_warning(vla-extension)
|
||||||
|
@ -157,7 +157,7 @@ private:
|
|||||||
std::string query_id;
|
std::string query_id;
|
||||||
bool continue_on_errors;
|
bool continue_on_errors;
|
||||||
bool print_stacktrace;
|
bool print_stacktrace;
|
||||||
Settings settings;
|
const Settings & settings;
|
||||||
SharedContextHolder shared_context;
|
SharedContextHolder shared_context;
|
||||||
Context global_context;
|
Context global_context;
|
||||||
QueryProcessingStage::Enum query_processing_stage;
|
QueryProcessingStage::Enum query_processing_stage;
|
||||||
|
@ -1360,7 +1360,8 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
|||||||
|
|
||||||
LOG_DEBUG(log, "Create destination tables. Query: {}", query);
|
LOG_DEBUG(log, "Create destination tables. Query: {}", query);
|
||||||
UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, PoolMode::GET_MANY);
|
UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, PoolMode::GET_MANY);
|
||||||
LOG_DEBUG(log, "Destination tables {} have been created on {} shards of {}", getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount());
|
LOG_DEBUG(log, "Destination tables {} have been created on {} shards of {}",
|
||||||
|
getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount());
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Do the copying
|
/// Do the copying
|
||||||
@ -1391,18 +1392,18 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
|||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
|
std::unique_ptr<Context> context_select = std::make_unique<Context>(context);
|
||||||
|
context_select->setSettings(task_cluster->settings_pull);
|
||||||
|
|
||||||
|
std::unique_ptr<Context> context_insert = std::make_unique<Context>(context);
|
||||||
|
context_insert->setSettings(task_cluster->settings_push);
|
||||||
|
|
||||||
/// Custom INSERT SELECT implementation
|
/// Custom INSERT SELECT implementation
|
||||||
Context context_select = context;
|
|
||||||
context_select.setSettings(task_cluster->settings_pull);
|
|
||||||
|
|
||||||
Context context_insert = context;
|
|
||||||
context_insert.setSettings(task_cluster->settings_push);
|
|
||||||
|
|
||||||
BlockInputStreamPtr input;
|
BlockInputStreamPtr input;
|
||||||
BlockOutputStreamPtr output;
|
BlockOutputStreamPtr output;
|
||||||
{
|
{
|
||||||
BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute();
|
BlockIO io_select = InterpreterFactory::get(query_select_ast, *context_select)->execute();
|
||||||
BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute();
|
BlockIO io_insert = InterpreterFactory::get(query_insert_ast, *context_insert)->execute();
|
||||||
|
|
||||||
input = io_select.getInputStream();
|
input = io_select.getInputStream();
|
||||||
output = io_insert.out;
|
output = io_insert.out;
|
||||||
|
@ -149,7 +149,7 @@ static void getNotEnoughMemoryMessage(std::string & msg)
|
|||||||
#if defined(__linux__)
|
#if defined(__linux__)
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
static constexpr size_t buf_size = 4096;
|
static constexpr size_t buf_size = 1024;
|
||||||
char buf[buf_size];
|
char buf[buf_size];
|
||||||
|
|
||||||
UInt64 max_map_count = 0;
|
UInt64 max_map_count = 0;
|
||||||
|
@ -252,7 +252,7 @@ private:
|
|||||||
/// There are loops of NUM_PASSES. It is very important that they are unfolded at compile-time.
|
/// There are loops of NUM_PASSES. It is very important that they are unfolded at compile-time.
|
||||||
|
|
||||||
/// For each of the NUM_PASSES bit ranges of the key, consider how many times each value of this bit range met.
|
/// For each of the NUM_PASSES bit ranges of the key, consider how many times each value of this bit range met.
|
||||||
CountType histograms[HISTOGRAM_SIZE * NUM_PASSES] = {0};
|
std::unique_ptr<CountType[]> histograms{new CountType[HISTOGRAM_SIZE * NUM_PASSES]{}};
|
||||||
|
|
||||||
typename Traits::Allocator allocator;
|
typename Traits::Allocator allocator;
|
||||||
|
|
||||||
@ -358,7 +358,7 @@ private:
|
|||||||
|
|
||||||
/// The beginning of every i-1-th bucket. 0th element will be equal to 1st.
|
/// The beginning of every i-1-th bucket. 0th element will be equal to 1st.
|
||||||
/// Last element will point to array end.
|
/// Last element will point to array end.
|
||||||
Element * prev_buckets[HISTOGRAM_SIZE + 1];
|
std::unique_ptr<Element *[]> prev_buckets{new Element*[HISTOGRAM_SIZE + 1]};
|
||||||
/// The beginning of every i-th bucket (the same array shifted by one).
|
/// The beginning of every i-th bucket (the same array shifted by one).
|
||||||
Element ** buckets = &prev_buckets[1];
|
Element ** buckets = &prev_buckets[1];
|
||||||
|
|
||||||
@ -375,7 +375,7 @@ private:
|
|||||||
/// also it corresponds with the results from https://github.com/powturbo/TurboHist
|
/// also it corresponds with the results from https://github.com/powturbo/TurboHist
|
||||||
|
|
||||||
static constexpr size_t UNROLL_COUNT = 8;
|
static constexpr size_t UNROLL_COUNT = 8;
|
||||||
CountType count[HISTOGRAM_SIZE * UNROLL_COUNT]{};
|
std::unique_ptr<CountType[]> count{new CountType[HISTOGRAM_SIZE * UNROLL_COUNT]{}};
|
||||||
size_t unrolled_size = size / UNROLL_COUNT * UNROLL_COUNT;
|
size_t unrolled_size = size / UNROLL_COUNT * UNROLL_COUNT;
|
||||||
|
|
||||||
for (Element * elem = arr; elem < arr + unrolled_size; elem += UNROLL_COUNT)
|
for (Element * elem = arr; elem < arr + unrolled_size; elem += UNROLL_COUNT)
|
||||||
|
@ -318,7 +318,7 @@ protected:
|
|||||||
|
|
||||||
/** max needle length is 255, max distinct ngrams for case-sensitive is (255 - 1), case-insensitive is 4 * (255 - 1)
|
/** max needle length is 255, max distinct ngrams for case-sensitive is (255 - 1), case-insensitive is 4 * (255 - 1)
|
||||||
* storage of 64K ngrams (n = 2, 128 KB) should be large enough for both cases */
|
* storage of 64K ngrams (n = 2, 128 KB) should be large enough for both cases */
|
||||||
VolnitskyTraits::Offset hash[VolnitskyTraits::hash_size]; /// Hash table.
|
std::unique_ptr<VolnitskyTraits::Offset[]> hash; /// Hash table.
|
||||||
|
|
||||||
const bool fallback; /// Do we need to use the fallback algorithm.
|
const bool fallback; /// Do we need to use the fallback algorithm.
|
||||||
|
|
||||||
@ -340,7 +340,7 @@ public:
|
|||||||
if (fallback)
|
if (fallback)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
memset(hash, 0, sizeof(hash));
|
hash = std::unique_ptr<VolnitskyTraits::Offset[]>(new VolnitskyTraits::Offset[VolnitskyTraits::hash_size]{});
|
||||||
|
|
||||||
auto callback = [this](const VolnitskyTraits::Ngram ngram, const int offset) { return this->putNGramBase(ngram, offset); };
|
auto callback = [this](const VolnitskyTraits::Ngram ngram, const int offset) { return this->putNGramBase(ngram, offset); };
|
||||||
/// ssize_t is used here because unsigned can't be used with condition like `i >= 0`, unsigned always >= 0
|
/// ssize_t is used here because unsigned can't be used with condition like `i >= 0`, unsigned always >= 0
|
||||||
@ -419,7 +419,7 @@ private:
|
|||||||
VolnitskyTraits::Offset off;
|
VolnitskyTraits::Offset off;
|
||||||
};
|
};
|
||||||
|
|
||||||
OffsetId hash[VolnitskyTraits::hash_size];
|
std::unique_ptr<OffsetId[]> hash;
|
||||||
|
|
||||||
/// step for each bunch of strings
|
/// step for each bunch of strings
|
||||||
size_t step;
|
size_t step;
|
||||||
@ -434,6 +434,7 @@ public:
|
|||||||
MultiVolnitskyBase(const std::vector<StringRef> & needles_) : needles{needles_}, step{0}, last{0}
|
MultiVolnitskyBase(const std::vector<StringRef> & needles_) : needles{needles_}, step{0}, last{0}
|
||||||
{
|
{
|
||||||
fallback_searchers.reserve(needles.size());
|
fallback_searchers.reserve(needles.size());
|
||||||
|
hash = std::unique_ptr<OffsetId[]>(new OffsetId[VolnitskyTraits::hash_size]); /// No zero initialization, it will be done later.
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -454,7 +455,7 @@ public:
|
|||||||
if (last == needles.size())
|
if (last == needles.size())
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
memset(hash, 0, sizeof(hash));
|
memset(hash.get(), 0, VolnitskyTraits::hash_size * sizeof(OffsetId));
|
||||||
fallback_needles.clear();
|
fallback_needles.clear();
|
||||||
step = std::numeric_limits<size_t>::max();
|
step = std::numeric_limits<size_t>::max();
|
||||||
|
|
||||||
|
@ -79,8 +79,8 @@ String getFilesystemName([[maybe_unused]] const String & mount_point)
|
|||||||
throw DB::Exception("Cannot open /etc/mtab to get name of filesystem", ErrorCodes::SYSTEM_ERROR);
|
throw DB::Exception("Cannot open /etc/mtab to get name of filesystem", ErrorCodes::SYSTEM_ERROR);
|
||||||
mntent fs_info;
|
mntent fs_info;
|
||||||
constexpr size_t buf_size = 4096; /// The same as buffer used for getmntent in glibc. It can happen that it's not enough
|
constexpr size_t buf_size = 4096; /// The same as buffer used for getmntent in glibc. It can happen that it's not enough
|
||||||
char buf[buf_size];
|
std::vector<char> buf(buf_size);
|
||||||
while (getmntent_r(mounted_filesystems, &fs_info, buf, buf_size) && fs_info.mnt_dir != mount_point)
|
while (getmntent_r(mounted_filesystems, &fs_info, buf.data(), buf_size) && fs_info.mnt_dir != mount_point)
|
||||||
;
|
;
|
||||||
endmntent(mounted_filesystems);
|
endmntent(mounted_filesystems);
|
||||||
if (fs_info.mnt_dir != mount_point)
|
if (fs_info.mnt_dir != mount_point)
|
||||||
|
@ -235,8 +235,7 @@ void DatabaseOrdinary::alterTable(const Context & context, const StorageID & tab
|
|||||||
String statement;
|
String statement;
|
||||||
|
|
||||||
{
|
{
|
||||||
char in_buf[METADATA_FILE_BUFFER_SIZE];
|
ReadBufferFromFile in(table_metadata_path, METADATA_FILE_BUFFER_SIZE);
|
||||||
ReadBufferFromFile in(table_metadata_path, METADATA_FILE_BUFFER_SIZE, -1, in_buf);
|
|
||||||
readStringUntilEOF(statement, in);
|
readStringUntilEOF(statement, in);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -48,11 +48,11 @@ struct NgramDistanceImpl
|
|||||||
/// Max codepoints to store at once. 16 is for batching usage and PODArray has this padding.
|
/// Max codepoints to store at once. 16 is for batching usage and PODArray has this padding.
|
||||||
static constexpr size_t simultaneously_codepoints_num = default_padding + N - 1;
|
static constexpr size_t simultaneously_codepoints_num = default_padding + N - 1;
|
||||||
|
|
||||||
/** This fits mostly in L2 cache all the time.
|
/** map_size of this fits mostly in L2 cache all the time.
|
||||||
* Actually use UInt16 as addings and subtractions do not UB overflow. But think of it as a signed
|
* Actually use UInt16 as addings and subtractions do not UB overflow. But think of it as a signed
|
||||||
* integer array.
|
* integer array.
|
||||||
*/
|
*/
|
||||||
using NgramStats = UInt16[map_size];
|
using NgramCount = UInt16;
|
||||||
|
|
||||||
static ALWAYS_INLINE UInt16 calculateASCIIHash(const CodePoint * code_points)
|
static ALWAYS_INLINE UInt16 calculateASCIIHash(const CodePoint * code_points)
|
||||||
{
|
{
|
||||||
@ -169,8 +169,8 @@ struct NgramDistanceImpl
|
|||||||
static ALWAYS_INLINE inline size_t calculateNeedleStats(
|
static ALWAYS_INLINE inline size_t calculateNeedleStats(
|
||||||
const char * data,
|
const char * data,
|
||||||
const size_t size,
|
const size_t size,
|
||||||
NgramStats & ngram_stats,
|
NgramCount * ngram_stats,
|
||||||
[[maybe_unused]] UInt16 * ngram_storage,
|
[[maybe_unused]] NgramCount * ngram_storage,
|
||||||
size_t (*read_code_points)(CodePoint *, const char *&, const char *),
|
size_t (*read_code_points)(CodePoint *, const char *&, const char *),
|
||||||
UInt16 (*hash_functor)(const CodePoint *))
|
UInt16 (*hash_functor)(const CodePoint *))
|
||||||
{
|
{
|
||||||
@ -202,7 +202,7 @@ struct NgramDistanceImpl
|
|||||||
static ALWAYS_INLINE inline UInt64 calculateHaystackStatsAndMetric(
|
static ALWAYS_INLINE inline UInt64 calculateHaystackStatsAndMetric(
|
||||||
const char * data,
|
const char * data,
|
||||||
const size_t size,
|
const size_t size,
|
||||||
NgramStats & ngram_stats,
|
NgramCount * ngram_stats,
|
||||||
size_t & distance,
|
size_t & distance,
|
||||||
[[maybe_unused]] UInt16 * ngram_storage,
|
[[maybe_unused]] UInt16 * ngram_storage,
|
||||||
size_t (*read_code_points)(CodePoint *, const char *&, const char *),
|
size_t (*read_code_points)(CodePoint *, const char *&, const char *),
|
||||||
@ -256,7 +256,7 @@ struct NgramDistanceImpl
|
|||||||
|
|
||||||
static void constantConstant(std::string data, std::string needle, Float32 & res)
|
static void constantConstant(std::string data, std::string needle, Float32 & res)
|
||||||
{
|
{
|
||||||
NgramStats common_stats = {};
|
std::unique_ptr<NgramCount[]> common_stats{new NgramCount[map_size]{}};
|
||||||
|
|
||||||
/// We use unsafe versions of getting ngrams, so I decided to use padded strings.
|
/// We use unsafe versions of getting ngrams, so I decided to use padded strings.
|
||||||
const size_t needle_size = needle.size();
|
const size_t needle_size = needle.size();
|
||||||
@ -264,11 +264,11 @@ struct NgramDistanceImpl
|
|||||||
needle.resize(needle_size + default_padding);
|
needle.resize(needle_size + default_padding);
|
||||||
data.resize(data_size + default_padding);
|
data.resize(data_size + default_padding);
|
||||||
|
|
||||||
size_t second_size = dispatchSearcher(calculateNeedleStats<false>, needle.data(), needle_size, common_stats, nullptr);
|
size_t second_size = dispatchSearcher(calculateNeedleStats<false>, needle.data(), needle_size, common_stats.get(), nullptr);
|
||||||
size_t distance = second_size;
|
size_t distance = second_size;
|
||||||
if (data_size <= max_string_size)
|
if (data_size <= max_string_size)
|
||||||
{
|
{
|
||||||
size_t first_size = dispatchSearcher(calculateHaystackStatsAndMetric<false>, data.data(), data_size, common_stats, distance, nullptr);
|
size_t first_size = dispatchSearcher(calculateHaystackStatsAndMetric<false>, data.data(), data_size, common_stats.get(), distance, nullptr);
|
||||||
/// For !symmetric version we should not use first_size.
|
/// For !symmetric version we should not use first_size.
|
||||||
if constexpr (symmetric)
|
if constexpr (symmetric)
|
||||||
res = distance * 1.f / std::max(first_size + second_size, size_t(1));
|
res = distance * 1.f / std::max(first_size + second_size, size_t(1));
|
||||||
@ -295,7 +295,7 @@ struct NgramDistanceImpl
|
|||||||
size_t prev_haystack_offset = 0;
|
size_t prev_haystack_offset = 0;
|
||||||
size_t prev_needle_offset = 0;
|
size_t prev_needle_offset = 0;
|
||||||
|
|
||||||
NgramStats common_stats = {};
|
std::unique_ptr<NgramCount[]> common_stats{new NgramCount[map_size]{}};
|
||||||
|
|
||||||
/// The main motivation is to not allocate more on stack because we have already allocated a lot (128Kb).
|
/// The main motivation is to not allocate more on stack because we have already allocated a lot (128Kb).
|
||||||
/// And we can reuse these storages in one thread because we care only about what was written to first places.
|
/// And we can reuse these storages in one thread because we care only about what was written to first places.
|
||||||
@ -316,7 +316,7 @@ struct NgramDistanceImpl
|
|||||||
calculateNeedleStats<true>,
|
calculateNeedleStats<true>,
|
||||||
needle,
|
needle,
|
||||||
needle_size,
|
needle_size,
|
||||||
common_stats,
|
common_stats.get(),
|
||||||
needle_ngram_storage.get());
|
needle_ngram_storage.get());
|
||||||
|
|
||||||
size_t distance = needle_stats_size;
|
size_t distance = needle_stats_size;
|
||||||
@ -326,7 +326,7 @@ struct NgramDistanceImpl
|
|||||||
calculateHaystackStatsAndMetric<true>,
|
calculateHaystackStatsAndMetric<true>,
|
||||||
haystack,
|
haystack,
|
||||||
haystack_size,
|
haystack_size,
|
||||||
common_stats,
|
common_stats.get(),
|
||||||
distance,
|
distance,
|
||||||
haystack_ngram_storage.get());
|
haystack_ngram_storage.get());
|
||||||
|
|
||||||
@ -378,7 +378,7 @@ struct NgramDistanceImpl
|
|||||||
const size_t needle_offsets_size = needle_offsets.size();
|
const size_t needle_offsets_size = needle_offsets.size();
|
||||||
size_t prev_offset = 0;
|
size_t prev_offset = 0;
|
||||||
|
|
||||||
NgramStats common_stats = {};
|
std::unique_ptr<NgramCount[]> common_stats{new NgramCount[map_size]{}};
|
||||||
|
|
||||||
std::unique_ptr<UInt16[]> needle_ngram_storage(new UInt16[max_string_size]);
|
std::unique_ptr<UInt16[]> needle_ngram_storage(new UInt16[max_string_size]);
|
||||||
std::unique_ptr<UInt16[]> haystack_ngram_storage(new UInt16[max_string_size]);
|
std::unique_ptr<UInt16[]> haystack_ngram_storage(new UInt16[max_string_size]);
|
||||||
@ -394,7 +394,7 @@ struct NgramDistanceImpl
|
|||||||
calculateNeedleStats<true>,
|
calculateNeedleStats<true>,
|
||||||
needle,
|
needle,
|
||||||
needle_size,
|
needle_size,
|
||||||
common_stats,
|
common_stats.get(),
|
||||||
needle_ngram_storage.get());
|
needle_ngram_storage.get());
|
||||||
|
|
||||||
size_t distance = needle_stats_size;
|
size_t distance = needle_stats_size;
|
||||||
@ -403,7 +403,7 @@ struct NgramDistanceImpl
|
|||||||
calculateHaystackStatsAndMetric<true>,
|
calculateHaystackStatsAndMetric<true>,
|
||||||
haystack.data(),
|
haystack.data(),
|
||||||
haystack_size,
|
haystack_size,
|
||||||
common_stats,
|
common_stats.get(),
|
||||||
distance,
|
distance,
|
||||||
haystack_ngram_storage.get());
|
haystack_ngram_storage.get());
|
||||||
|
|
||||||
@ -430,17 +430,16 @@ struct NgramDistanceImpl
|
|||||||
PaddedPODArray<Float32> & res)
|
PaddedPODArray<Float32> & res)
|
||||||
{
|
{
|
||||||
/// zeroing our map
|
/// zeroing our map
|
||||||
NgramStats common_stats = {};
|
std::unique_ptr<NgramCount[]> common_stats{new NgramCount[map_size]{}};
|
||||||
|
|
||||||
/// The main motivation is to not allocate more on stack because we have already allocated a lot (128Kb).
|
/// We can reuse these storages in one thread because we care only about what was written to first places.
|
||||||
/// And we can reuse these storages in one thread because we care only about what was written to first places.
|
std::unique_ptr<UInt16[]> ngram_storage(new NgramCount[max_string_size]);
|
||||||
std::unique_ptr<UInt16[]> ngram_storage(new UInt16[max_string_size]);
|
|
||||||
|
|
||||||
/// We use unsafe versions of getting ngrams, so I decided to use padded_data even in needle case.
|
/// We use unsafe versions of getting ngrams, so I decided to use padded_data even in needle case.
|
||||||
const size_t needle_size = needle.size();
|
const size_t needle_size = needle.size();
|
||||||
needle.resize(needle_size + default_padding);
|
needle.resize(needle_size + default_padding);
|
||||||
|
|
||||||
const size_t needle_stats_size = dispatchSearcher(calculateNeedleStats<false>, needle.data(), needle_size, common_stats, nullptr);
|
const size_t needle_stats_size = dispatchSearcher(calculateNeedleStats<false>, needle.data(), needle_size, common_stats.get(), nullptr);
|
||||||
|
|
||||||
size_t distance = needle_stats_size;
|
size_t distance = needle_stats_size;
|
||||||
size_t prev_offset = 0;
|
size_t prev_offset = 0;
|
||||||
@ -453,7 +452,7 @@ struct NgramDistanceImpl
|
|||||||
size_t haystack_stats_size = dispatchSearcher(
|
size_t haystack_stats_size = dispatchSearcher(
|
||||||
calculateHaystackStatsAndMetric<true>,
|
calculateHaystackStatsAndMetric<true>,
|
||||||
reinterpret_cast<const char *>(haystack),
|
reinterpret_cast<const char *>(haystack),
|
||||||
haystack_size, common_stats,
|
haystack_size, common_stats.get(),
|
||||||
distance,
|
distance,
|
||||||
ngram_storage.get());
|
ngram_storage.get());
|
||||||
/// For !symmetric version we should not use haystack_stats_size.
|
/// For !symmetric version we should not use haystack_stats_size.
|
||||||
|
@ -58,8 +58,8 @@ public:
|
|||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
|
/// Initially allocate a piece of memory for 64 elements. NOTE: This is just a guess.
|
||||||
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
|
static constexpr size_t INITIAL_SIZE_DEGREE = 6;
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
struct MethodOneNumber
|
struct MethodOneNumber
|
||||||
|
@ -118,8 +118,8 @@ public:
|
|||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const override;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
|
/// Initially allocate a piece of memory for 64 elements. NOTE: This is just a guess.
|
||||||
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
|
static constexpr size_t INITIAL_SIZE_DEGREE = 6;
|
||||||
|
|
||||||
void executeMethodImpl(
|
void executeMethodImpl(
|
||||||
const std::vector<const ColumnArray::Offsets *> & offsets_by_depth,
|
const std::vector<const ColumnArray::Offsets *> & offsets_by_depth,
|
||||||
|
@ -55,8 +55,8 @@ public:
|
|||||||
private:
|
private:
|
||||||
const Context & context;
|
const Context & context;
|
||||||
|
|
||||||
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
|
/// Initially allocate a piece of memory for 64 elements. NOTE: This is just a guess.
|
||||||
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
|
static constexpr size_t INITIAL_SIZE_DEGREE = 6;
|
||||||
|
|
||||||
struct UnpackedArrays
|
struct UnpackedArrays
|
||||||
{
|
{
|
||||||
|
@ -42,12 +42,12 @@ void AIOContextPool::doMonitor()
|
|||||||
void AIOContextPool::waitForCompletion()
|
void AIOContextPool::waitForCompletion()
|
||||||
{
|
{
|
||||||
/// array to hold completion events
|
/// array to hold completion events
|
||||||
io_event events[max_concurrent_events];
|
std::vector<io_event> events(max_concurrent_events);
|
||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
const auto num_events = getCompletionEvents(events, max_concurrent_events);
|
const auto num_events = getCompletionEvents(events.data(), max_concurrent_events);
|
||||||
fulfillPromises(events, num_events);
|
fulfillPromises(events.data(), num_events);
|
||||||
notifyProducers(num_events);
|
notifyProducers(num_events);
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
|
@ -1030,7 +1030,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
|||||||
commands_for_part.emplace_back(command);
|
commands_for_part.emplace_back(command);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (source_part->isStoredOnDisk() && !isStorageTouchedByMutations(storage_from_source_part, metadata_snapshot, commands_for_part, context_for_reading))
|
if (source_part->isStoredOnDisk() && !isStorageTouchedByMutations(
|
||||||
|
storage_from_source_part, metadata_snapshot, commands_for_part, context_for_reading))
|
||||||
{
|
{
|
||||||
LOG_TRACE(log, "Part {} doesn't change up to mutation version {}", source_part->name, future_part.part_info.mutation);
|
LOG_TRACE(log, "Part {} doesn't change up to mutation version {}", source_part->name, future_part.part_info.mutation);
|
||||||
return data.cloneAndLoadDataPartOnSameDisk(source_part, "tmp_clone_", future_part.part_info, metadata_snapshot);
|
return data.cloneAndLoadDataPartOnSameDisk(source_part, "tmp_clone_", future_part.part_info, metadata_snapshot);
|
||||||
@ -1042,7 +1043,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
|||||||
|
|
||||||
BlockInputStreamPtr in = nullptr;
|
BlockInputStreamPtr in = nullptr;
|
||||||
Block updated_header;
|
Block updated_header;
|
||||||
std::optional<MutationsInterpreter> interpreter;
|
std::unique_ptr<MutationsInterpreter> interpreter;
|
||||||
|
|
||||||
const auto data_settings = data.getSettings();
|
const auto data_settings = data.getSettings();
|
||||||
MutationCommands for_interpreter;
|
MutationCommands for_interpreter;
|
||||||
@ -1057,7 +1058,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
|||||||
|
|
||||||
if (!for_interpreter.empty())
|
if (!for_interpreter.empty())
|
||||||
{
|
{
|
||||||
interpreter.emplace(storage_from_source_part, metadata_snapshot, for_interpreter, context_for_reading, true);
|
interpreter = std::make_unique<MutationsInterpreter>(
|
||||||
|
storage_from_source_part, metadata_snapshot, for_interpreter, context_for_reading, true);
|
||||||
in = interpreter->execute();
|
in = interpreter->execute();
|
||||||
updated_header = interpreter->getUpdatedHeader();
|
updated_header = interpreter->getUpdatedHeader();
|
||||||
in->setProgressCallback(MergeProgressCallback(merge_entry, watch_prev_elapsed, stage_progress));
|
in->setProgressCallback(MergeProgressCallback(merge_entry, watch_prev_elapsed, stage_progress));
|
||||||
|
@ -7,24 +7,24 @@ insert into array_intersect values ('2019-01-01', [1,2]);
|
|||||||
insert into array_intersect values ('2019-01-01', [1]);
|
insert into array_intersect values ('2019-01-01', [1]);
|
||||||
insert into array_intersect values ('2019-01-01', []);
|
insert into array_intersect values ('2019-01-01', []);
|
||||||
|
|
||||||
select arrayIntersect(arr, [1,2]) from array_intersect order by arr;
|
select arraySort(arrayIntersect(arr, [1,2])) from array_intersect order by arr;
|
||||||
select arrayIntersect(arr, []) from array_intersect order by arr;
|
select arraySort(arrayIntersect(arr, [])) from array_intersect order by arr;
|
||||||
select arrayIntersect([], arr) from array_intersect order by arr;
|
select arraySort(arrayIntersect([], arr)) from array_intersect order by arr;
|
||||||
select arrayIntersect([1,2], arr) from array_intersect order by arr;
|
select arraySort(arrayIntersect([1,2], arr)) from array_intersect order by arr;
|
||||||
select arrayIntersect([1,2], [1,2,3,4]) from array_intersect order by arr;
|
select arraySort(arrayIntersect([1,2], [1,2,3,4])) from array_intersect order by arr;
|
||||||
select arrayIntersect([], []) from array_intersect order by arr;
|
select arraySort(arrayIntersect([], [])) from array_intersect order by arr;
|
||||||
|
|
||||||
optimize table array_intersect;
|
optimize table array_intersect;
|
||||||
|
|
||||||
select arrayIntersect(arr, [1,2]) from array_intersect order by arr;
|
select arraySort(arrayIntersect(arr, [1,2])) from array_intersect order by arr;
|
||||||
select arrayIntersect(arr, []) from array_intersect order by arr;
|
select arraySort(arrayIntersect(arr, [])) from array_intersect order by arr;
|
||||||
select arrayIntersect([], arr) from array_intersect order by arr;
|
select arraySort(arrayIntersect([], arr)) from array_intersect order by arr;
|
||||||
select arrayIntersect([1,2], arr) from array_intersect order by arr;
|
select arraySort(arrayIntersect([1,2], arr)) from array_intersect order by arr;
|
||||||
select arrayIntersect([1,2], [1,2,3,4]) from array_intersect order by arr;
|
select arraySort(arrayIntersect([1,2], [1,2,3,4])) from array_intersect order by arr;
|
||||||
select arrayIntersect([], []) from array_intersect order by arr;
|
select arraySort(arrayIntersect([], [])) from array_intersect order by arr;
|
||||||
|
|
||||||
drop table if exists array_intersect;
|
drop table if exists array_intersect;
|
||||||
|
|
||||||
select '-';
|
select '-';
|
||||||
select arrayIntersect([-100], [156]);
|
select arraySort(arrayIntersect([-100], [156]));
|
||||||
select arrayIntersect([1], [257]);
|
select arraySort(arrayIntersect([1], [257]));
|
||||||
|
@ -5,5 +5,5 @@
|
|||||||
[2]
|
[2]
|
||||||
[]
|
[]
|
||||||
[]
|
[]
|
||||||
[3,1,2]
|
[1,2,3]
|
||||||
[]
|
[]
|
||||||
|
@ -1,9 +1,9 @@
|
|||||||
SELECT arrayIntersect(['a', 'b', 'c'], ['a', 'a']);
|
SELECT arraySort(arrayIntersect(['a', 'b', 'c'], ['a', 'a']));
|
||||||
SELECT arrayIntersect([1, 1], [2, 2]);
|
SELECT arraySort(arrayIntersect([1, 1], [2, 2]));
|
||||||
SELECT arrayIntersect([1, 1], [1, 2]);
|
SELECT arraySort(arrayIntersect([1, 1], [1, 2]));
|
||||||
SELECT arrayIntersect([1, 1, 1], [3], [2, 2, 2]);
|
SELECT arraySort(arrayIntersect([1, 1, 1], [3], [2, 2, 2]));
|
||||||
SELECT arrayIntersect([1, 2], [1, 2], [2]);
|
SELECT arraySort(arrayIntersect([1, 2], [1, 2], [2]));
|
||||||
SELECT arrayIntersect([1, 1], [2, 1], [2, 2], [1]);
|
SELECT arraySort(arrayIntersect([1, 1], [2, 1], [2, 2], [1]));
|
||||||
SELECT arrayIntersect([]);
|
SELECT arraySort(arrayIntersect([]));
|
||||||
SELECT arrayIntersect([1, 2, 3]);
|
SELECT arraySort(arrayIntersect([1, 2, 3]));
|
||||||
SELECT arrayIntersect([1, 1], [2, 1], [2, 2], [2, 2, 2]);
|
SELECT arraySort(arrayIntersect([1, 1], [2, 1], [2, 2], [2, 2, 2]));
|
||||||
|
@ -1,2 +1,2 @@
|
|||||||
[0,3,2] id2
|
[0,2,3] id2
|
||||||
[3,1,2] id1
|
[1,2,3] id1
|
||||||
|
@ -11,7 +11,7 @@ INSERT INTO tags(id, seqs) VALUES ('id1', [1,2,3]), ('id2', [0,2,3]), ('id1', [1
|
|||||||
|
|
||||||
WITH
|
WITH
|
||||||
(SELECT [0, 1, 2, 3]) AS arr1
|
(SELECT [0, 1, 2, 3]) AS arr1
|
||||||
SELECT arrayIntersect(argMax(seqs, create_time), arr1) AS common, id
|
SELECT arraySort(arrayIntersect(argMax(seqs, create_time), arr1)) AS common, id
|
||||||
FROM tags
|
FROM tags
|
||||||
WHERE id LIKE 'id%'
|
WHERE id LIKE 'id%'
|
||||||
GROUP BY id;
|
GROUP BY id;
|
||||||
|
Loading…
Reference in New Issue
Block a user