Fix oss-fuzz build errors (#49236)

This commit is contained in:
Nikita Taranov 2023-04-29 18:44:50 +02:00 committed by GitHub
parent 7d867d7632
commit f1ff2c0135
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 29 additions and 33 deletions

View File

@ -163,15 +163,11 @@ public:
if constexpr (std::is_same_v<Data, QuantileTiming<Value>>)
{
/// QuantileTiming only supports unsigned integers. Too large values are also meaningless.
#ifdef OS_DARWIN
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wimplicit-const-int-float-conversion"
#endif
if (isNaN(value) || value > std::numeric_limits<Int64>::max() || value < 0)
return;
#ifdef OS_DARWIN
# pragma clang diagnostic pop
#endif
}
if constexpr (has_second_arg)

View File

@ -13,6 +13,7 @@
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <base/scope_guard.h>
extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size)
try

View File

@ -32,13 +32,15 @@ MergeTreeBackgroundExecutor<Queue>::MergeTreeBackgroundExecutor(
size_t threads_count_,
size_t max_tasks_count_,
CurrentMetrics::Metric metric_,
CurrentMetrics::Metric max_tasks_metric_)
CurrentMetrics::Metric max_tasks_metric_,
std::string_view policy)
: name(name_)
, threads_count(threads_count_)
, max_tasks_count(max_tasks_count_)
, metric(metric_)
, max_tasks_metric(max_tasks_metric_, 2 * max_tasks_count) // active + pending
, pool(std::make_unique<ThreadPool>(CurrentMetrics::MergeTreeBackgroundExecutorThreads, CurrentMetrics::MergeTreeBackgroundExecutorThreadsActive))
, pool(std::make_unique<ThreadPool>(
CurrentMetrics::MergeTreeBackgroundExecutorThreads, CurrentMetrics::MergeTreeBackgroundExecutorThreadsActive))
{
if (max_tasks_count == 0)
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Task count for MergeTreeBackgroundExecutor must not be zero");
@ -52,20 +54,9 @@ MergeTreeBackgroundExecutor<Queue>::MergeTreeBackgroundExecutor(
for (size_t number = 0; number < threads_count; ++number)
pool->scheduleOrThrowOnError([this] { threadFunction(); });
}
template <class Queue>
MergeTreeBackgroundExecutor<Queue>::MergeTreeBackgroundExecutor(
String name_,
size_t threads_count_,
size_t max_tasks_count_,
CurrentMetrics::Metric metric_,
CurrentMetrics::Metric max_tasks_metric_,
std::string_view policy)
requires requires(Queue queue) { queue.updatePolicy(policy); } // Because we use explicit template instantiation
: MergeTreeBackgroundExecutor(name_, threads_count_, max_tasks_count_, metric_, max_tasks_metric_)
{
pending.updatePolicy(policy);
if (!policy.empty())
pending.updatePolicy(policy);
}
template <class Queue>
@ -326,5 +317,4 @@ void MergeTreeBackgroundExecutor<Queue>::threadFunction()
template class MergeTreeBackgroundExecutor<RoundRobinRuntimeQueue>;
template class MergeTreeBackgroundExecutor<PriorityRuntimeQueue>;
template class MergeTreeBackgroundExecutor<DynamicRuntimeQueue>;
}

View File

@ -13,16 +13,22 @@
#include <boost/noncopyable.hpp>
#include <Poco/Event.h>
#include <Common/CurrentMetrics.h>
#include <Common/ThreadPool_fwd.h>
#include <Common/Stopwatch.h>
#include <base/defines.h>
#include <Storages/MergeTree/IExecutableTask.h>
#include <base/defines.h>
#include <Common/CurrentMetrics.h>
#include <Common/Exception.h>
#include <Common/Stopwatch.h>
#include <Common/ThreadPool_fwd.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
struct TaskRuntimeData;
using TaskRuntimeDataPtr = std::shared_ptr<TaskRuntimeData>;
@ -92,6 +98,11 @@ public:
void setCapacity(size_t count) { queue.set_capacity(count); }
bool empty() { return queue.empty(); }
[[noreturn]] void updatePolicy(std::string_view)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method updatePolicy() is not implemented");
}
static constexpr std::string_view name = "round_robin";
private:
@ -126,6 +137,11 @@ public:
void setCapacity(size_t count) { buffer.reserve(count); }
bool empty() { return buffer.empty(); }
[[noreturn]] void updatePolicy(std::string_view)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method updatePolicy() is not implemented");
}
static constexpr std::string_view name = "shortest_task_first";
private:
@ -239,20 +255,14 @@ template <class Queue>
class MergeTreeBackgroundExecutor final : boost::noncopyable
{
public:
MergeTreeBackgroundExecutor(
String name_,
size_t threads_count_,
size_t max_tasks_count_,
CurrentMetrics::Metric metric_,
CurrentMetrics::Metric max_tasks_metric_);
MergeTreeBackgroundExecutor(
String name_,
size_t threads_count_,
size_t max_tasks_count_,
CurrentMetrics::Metric metric_,
CurrentMetrics::Metric max_tasks_metric_,
std::string_view policy)
requires requires(Queue queue) { queue.updatePolicy(policy); }; // Because we use explicit template instantiation
std::string_view policy = {});
~MergeTreeBackgroundExecutor();
/// Handler for hot-reloading
@ -271,7 +281,6 @@ public:
/// Update scheduling policy for pending tasks. It does nothing if `new_policy` is the same or unknown.
void updateSchedulingPolicy(std::string_view new_policy)
requires requires(Queue queue) { queue.updatePolicy(new_policy); } // Because we use explicit template instantiation
{
std::lock_guard lock(mutex);
pending.updatePolicy(new_policy);