mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #59025 from ClickHouse/cpu-scheduler
Split resource scheduler off `IO/` into `Common/Scheduler/`
This commit is contained in:
commit
273a06856a
@ -76,8 +76,8 @@
|
||||
#include <Databases/registerDatabases.h>
|
||||
#include <Dictionaries/registerDictionaries.h>
|
||||
#include <Disks/registerDisks.h>
|
||||
#include <IO/Resource/registerSchedulerNodes.h>
|
||||
#include <IO/Resource/registerResourceManagers.h>
|
||||
#include <Common/Scheduler/Nodes/registerSchedulerNodes.h>
|
||||
#include <Common/Scheduler/Nodes/registerResourceManagers.h>
|
||||
#include <Common/Config/ConfigReloader.h>
|
||||
#include <Server/HTTPHandlerFactory.h>
|
||||
#include "MetricsTransmitter.h"
|
||||
|
@ -83,10 +83,11 @@ add_subdirectory (Formats)
|
||||
|
||||
add_headers_and_sources(clickhouse_common_io Common)
|
||||
add_headers_and_sources(clickhouse_common_io Common/HashTable)
|
||||
add_headers_and_sources(clickhouse_common_io Common/Scheduler)
|
||||
add_headers_and_sources(clickhouse_common_io Common/Scheduler/Nodes)
|
||||
add_headers_and_sources(clickhouse_common_io Common/SSH)
|
||||
add_headers_and_sources(clickhouse_common_io IO)
|
||||
add_headers_and_sources(clickhouse_common_io IO/Archives)
|
||||
add_headers_and_sources(clickhouse_common_io IO/Resource)
|
||||
add_headers_and_sources(clickhouse_common_io IO/S3)
|
||||
list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_delete.cpp)
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/ResourceLink.h>
|
||||
#include <Common/Scheduler/ResourceLink.h>
|
||||
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/ISchedulerNode.h>
|
||||
#include <Common/Scheduler/ISchedulerNode.h>
|
||||
|
||||
namespace DB
|
||||
{
|
@ -6,7 +6,7 @@
|
||||
#include <base/defines.h>
|
||||
#include <base/types.h>
|
||||
|
||||
#include <IO/ResourceRequest.h>
|
||||
#include <Common/Scheduler/ResourceRequest.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
|
@ -1,8 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/ISchedulerNode.h>
|
||||
#include <IO/ResourceBudget.h>
|
||||
#include <IO/ResourceRequest.h>
|
||||
#include <Common/Scheduler/ISchedulerNode.h>
|
||||
#include <Common/Scheduler/ResourceBudget.h>
|
||||
#include <Common/Scheduler/ResourceRequest.h>
|
||||
|
||||
#include <memory>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <IO/Resource/ClassifiersConfig.h>
|
||||
#include <Common/Scheduler/Nodes/ClassifiersConfig.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
|
@ -1,8 +1,8 @@
|
||||
#include <IO/Resource/DynamicResourceManager.h>
|
||||
#include <Common/Scheduler/Nodes/DynamicResourceManager.h>
|
||||
|
||||
#include <IO/SchedulerNodeFactory.h>
|
||||
#include <IO/ResourceManagerFactory.h>
|
||||
#include <IO/ISchedulerQueue.h>
|
||||
#include <Common/Scheduler/Nodes/SchedulerNodeFactory.h>
|
||||
#include <Common/Scheduler/ResourceManagerFactory.h>
|
||||
#include <Common/Scheduler/ISchedulerQueue.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
@ -1,8 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/IResourceManager.h>
|
||||
#include <IO/SchedulerRoot.h>
|
||||
#include <IO/Resource/ClassifiersConfig.h>
|
||||
#include <Common/Scheduler/IResourceManager.h>
|
||||
#include <Common/Scheduler/SchedulerRoot.h>
|
||||
#include <Common/Scheduler/Nodes/ClassifiersConfig.h>
|
||||
|
||||
#include <mutex>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <IO/Resource/FairPolicy.h>
|
||||
#include <Common/Scheduler/Nodes/FairPolicy.h>
|
||||
|
||||
#include <IO/SchedulerNodeFactory.h>
|
||||
#include <Common/Scheduler/Nodes/SchedulerNodeFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/ISchedulerNode.h>
|
||||
#include <Common/Scheduler/ISchedulerNode.h>
|
||||
|
||||
#include <Common/Stopwatch.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <IO/Resource/FifoQueue.h>
|
||||
#include <Common/Scheduler/Nodes/FifoQueue.h>
|
||||
|
||||
#include <IO/SchedulerNodeFactory.h>
|
||||
#include <Common/Scheduler/Nodes/SchedulerNodeFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <Common/Stopwatch.h>
|
||||
|
||||
#include <IO/ISchedulerQueue.h>
|
||||
#include <Common/Scheduler/ISchedulerQueue.h>
|
||||
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <IO/Resource/PriorityPolicy.h>
|
||||
#include <Common/Scheduler/Nodes/PriorityPolicy.h>
|
||||
|
||||
#include <IO/SchedulerNodeFactory.h>
|
||||
#include <Common/Scheduler/Nodes/SchedulerNodeFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/ISchedulerNode.h>
|
||||
#include <Common/Scheduler/ISchedulerNode.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <unordered_map>
|
@ -3,7 +3,7 @@
|
||||
#include <Common/ErrorCodes.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <IO/ISchedulerNode.h>
|
||||
#include <Common/Scheduler/ISchedulerNode.h>
|
||||
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <IO/Resource/SemaphoreConstraint.h>
|
||||
#include <Common/Scheduler/Nodes/SemaphoreConstraint.h>
|
||||
|
||||
#include <IO/SchedulerNodeFactory.h>
|
||||
#include <Common/Scheduler/Nodes/SchedulerNodeFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/ISchedulerConstraint.h>
|
||||
#include <Common/Scheduler/ISchedulerConstraint.h>
|
||||
|
||||
#include <mutex>
|
||||
#include <limits>
|
@ -1,8 +1,8 @@
|
||||
#include <IO/Resource/StaticResourceManager.h>
|
||||
#include <Common/Scheduler/Nodes/StaticResourceManager.h>
|
||||
|
||||
#include <IO/SchedulerNodeFactory.h>
|
||||
#include <IO/ResourceManagerFactory.h>
|
||||
#include <IO/ISchedulerQueue.h>
|
||||
#include <Common/Scheduler/Nodes/SchedulerNodeFactory.h>
|
||||
#include <Common/Scheduler/ResourceManagerFactory.h>
|
||||
#include <Common/Scheduler/ISchedulerQueue.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
@ -1,8 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/IResourceManager.h>
|
||||
#include <IO/SchedulerRoot.h>
|
||||
#include <IO/Resource/ClassifiersConfig.h>
|
||||
#include <Common/Scheduler/IResourceManager.h>
|
||||
#include <Common/Scheduler/SchedulerRoot.h>
|
||||
#include <Common/Scheduler/Nodes/ClassifiersConfig.h>
|
||||
|
||||
#include <mutex>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <IO/Resource/ThrottlerConstraint.h>
|
||||
#include <Common/Scheduler/Nodes/ThrottlerConstraint.h>
|
||||
|
||||
#include <IO/SchedulerNodeFactory.h>
|
||||
#include <Common/Scheduler/Nodes/SchedulerNodeFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/ISchedulerConstraint.h>
|
||||
#include <Common/Scheduler/ISchedulerConstraint.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <mutex>
|
@ -1,5 +1,5 @@
|
||||
#include <IO/Resource/registerResourceManagers.h>
|
||||
#include <IO/ResourceManagerFactory.h>
|
||||
#include <Common/Scheduler/Nodes/registerResourceManagers.h>
|
||||
#include <Common/Scheduler/ResourceManagerFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
@ -1,8 +1,8 @@
|
||||
#include <IO/Resource/registerSchedulerNodes.h>
|
||||
#include <Common/Scheduler/Nodes/registerSchedulerNodes.h>
|
||||
|
||||
#include <IO/ISchedulerNode.h>
|
||||
#include <IO/ISchedulerConstraint.h>
|
||||
#include <IO/SchedulerNodeFactory.h>
|
||||
#include <Common/Scheduler/ISchedulerNode.h>
|
||||
#include <Common/Scheduler/ISchedulerConstraint.h>
|
||||
#include <Common/Scheduler/Nodes/SchedulerNodeFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
@ -1,14 +1,14 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/IResourceManager.h>
|
||||
#include <IO/SchedulerRoot.h>
|
||||
#include <IO/ResourceGuard.h>
|
||||
#include <IO/SchedulerNodeFactory.h>
|
||||
#include <IO/Resource/PriorityPolicy.h>
|
||||
#include <IO/Resource/FifoQueue.h>
|
||||
#include <IO/Resource/SemaphoreConstraint.h>
|
||||
#include <IO/Resource/registerSchedulerNodes.h>
|
||||
#include <IO/Resource/registerResourceManagers.h>
|
||||
#include <Common/Scheduler/IResourceManager.h>
|
||||
#include <Common/Scheduler/SchedulerRoot.h>
|
||||
#include <Common/Scheduler/ResourceGuard.h>
|
||||
#include <Common/Scheduler/Nodes/SchedulerNodeFactory.h>
|
||||
#include <Common/Scheduler/Nodes/PriorityPolicy.h>
|
||||
#include <Common/Scheduler/Nodes/FifoQueue.h>
|
||||
#include <Common/Scheduler/Nodes/SemaphoreConstraint.h>
|
||||
#include <Common/Scheduler/Nodes/registerSchedulerNodes.h>
|
||||
#include <Common/Scheduler/Nodes/registerResourceManagers.h>
|
||||
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
|
@ -1,14 +1,14 @@
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <IO/Resource/tests/ResourceTest.h>
|
||||
#include <Common/Scheduler/Nodes/tests/ResourceTest.h>
|
||||
|
||||
#include <IO/Resource/FairPolicy.h>
|
||||
#include <Common/Scheduler/Nodes/FairPolicy.h>
|
||||
|
||||
using namespace DB;
|
||||
|
||||
using ResourceTest = ResourceTestClass;
|
||||
|
||||
TEST(IOResourceFairPolicy, Factory)
|
||||
TEST(SchedulerFairPolicy, Factory)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
@ -17,7 +17,7 @@ TEST(IOResourceFairPolicy, Factory)
|
||||
EXPECT_TRUE(dynamic_cast<FairPolicy *>(fair.get()) != nullptr);
|
||||
}
|
||||
|
||||
TEST(IOResourceFairPolicy, FairnessWeights)
|
||||
TEST(SchedulerFairPolicy, FairnessWeights)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
@ -41,7 +41,7 @@ TEST(IOResourceFairPolicy, FairnessWeights)
|
||||
t.consumed("B", 20);
|
||||
}
|
||||
|
||||
TEST(IOResourceFairPolicy, Activation)
|
||||
TEST(SchedulerFairPolicy, Activation)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
@ -77,7 +77,7 @@ TEST(IOResourceFairPolicy, Activation)
|
||||
t.consumed("B", 10);
|
||||
}
|
||||
|
||||
TEST(IOResourceFairPolicy, FairnessMaxMin)
|
||||
TEST(SchedulerFairPolicy, FairnessMaxMin)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
@ -101,7 +101,7 @@ TEST(IOResourceFairPolicy, FairnessMaxMin)
|
||||
t.consumed("A", 20);
|
||||
}
|
||||
|
||||
TEST(IOResourceFairPolicy, HierarchicalFairness)
|
||||
TEST(SchedulerFairPolicy, HierarchicalFairness)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
@ -1,14 +1,14 @@
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <IO/Resource/tests/ResourceTest.h>
|
||||
#include <Common/Scheduler/Nodes/tests/ResourceTest.h>
|
||||
|
||||
#include <IO/Resource/PriorityPolicy.h>
|
||||
#include <Common/Scheduler/Nodes/PriorityPolicy.h>
|
||||
|
||||
using namespace DB;
|
||||
|
||||
using ResourceTest = ResourceTestClass;
|
||||
|
||||
TEST(IOResourcePriorityPolicy, Factory)
|
||||
TEST(SchedulerPriorityPolicy, Factory)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
@ -17,7 +17,7 @@ TEST(IOResourcePriorityPolicy, Factory)
|
||||
EXPECT_TRUE(dynamic_cast<PriorityPolicy *>(prio.get()) != nullptr);
|
||||
}
|
||||
|
||||
TEST(IOResourcePriorityPolicy, Priorities)
|
||||
TEST(SchedulerPriorityPolicy, Priorities)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
@ -51,7 +51,7 @@ TEST(IOResourcePriorityPolicy, Priorities)
|
||||
t.consumed("C", 0);
|
||||
}
|
||||
|
||||
TEST(IOResourcePriorityPolicy, Activation)
|
||||
TEST(SchedulerPriorityPolicy, Activation)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
@ -92,7 +92,7 @@ TEST(IOResourcePriorityPolicy, Activation)
|
||||
t.consumed("C", 0);
|
||||
}
|
||||
|
||||
TEST(IOResourcePriorityPolicy, SinglePriority)
|
||||
TEST(SchedulerPriorityPolicy, SinglePriority)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
@ -1,8 +1,8 @@
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <IO/Resource/tests/ResourceTest.h>
|
||||
#include <Common/Scheduler/Nodes/tests/ResourceTest.h>
|
||||
|
||||
#include <IO/Resource/DynamicResourceManager.h>
|
||||
#include <Common/Scheduler/Nodes/DynamicResourceManager.h>
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
|
||||
using namespace DB;
|
||||
@ -10,7 +10,7 @@ using namespace DB;
|
||||
using ResourceTest = ResourceTestManager<DynamicResourceManager>;
|
||||
using TestGuard = ResourceTest::Guard;
|
||||
|
||||
TEST(IOResourceDynamicResourceManager, Smoke)
|
||||
TEST(SchedulerDynamicResourceManager, Smoke)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
@ -45,7 +45,7 @@ TEST(IOResourceDynamicResourceManager, Smoke)
|
||||
}
|
||||
}
|
||||
|
||||
TEST(IOResourceDynamicResourceManager, Fairness)
|
||||
TEST(SchedulerDynamicResourceManager, Fairness)
|
||||
{
|
||||
// Total cost for A and B cannot differ for more than 1 (every request has cost equal to 1).
|
||||
// Requests from A use `value = 1` and from B `value = -1` is used.
|
@ -1,8 +1,8 @@
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <IO/Resource/tests/ResourceTest.h>
|
||||
#include <Common/Scheduler/Nodes/tests/ResourceTest.h>
|
||||
|
||||
#include <IO/Resource/StaticResourceManager.h>
|
||||
#include <Common/Scheduler/Nodes/StaticResourceManager.h>
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
|
||||
using namespace DB;
|
||||
@ -10,7 +10,7 @@ using namespace DB;
|
||||
using ResourceTest = ResourceTestManager<StaticResourceManager>;
|
||||
using TestGuard = ResourceTest::Guard;
|
||||
|
||||
TEST(IOResourceStaticResourceManager, Smoke)
|
||||
TEST(SchedulerStaticResourceManager, Smoke)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
||||
@ -42,7 +42,7 @@ TEST(IOResourceStaticResourceManager, Smoke)
|
||||
}
|
||||
}
|
||||
|
||||
TEST(IOResourceStaticResourceManager, Prioritization)
|
||||
TEST(SchedulerStaticResourceManager, Prioritization)
|
||||
{
|
||||
std::optional<Priority> last_priority;
|
||||
auto check = [&] (Priority priority)
|
@ -1,8 +1,8 @@
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <IO/SchedulerRoot.h>
|
||||
#include <Common/Scheduler/SchedulerRoot.h>
|
||||
|
||||
#include <IO/Resource/tests/ResourceTest.h>
|
||||
#include <Common/Scheduler/Nodes/tests/ResourceTest.h>
|
||||
|
||||
#include <future>
|
||||
|
||||
@ -73,7 +73,7 @@ struct ResourceHolder
|
||||
}
|
||||
};
|
||||
|
||||
TEST(IOSchedulerRoot, Smoke)
|
||||
TEST(SchedulerRoot, Smoke)
|
||||
{
|
||||
ResourceTest t;
|
||||
|
@ -1,18 +1,18 @@
|
||||
#include <chrono>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <IO/Resource/tests/ResourceTest.h>
|
||||
#include <Common/Scheduler/Nodes/tests/ResourceTest.h>
|
||||
|
||||
#include <IO/Resource/FairPolicy.h>
|
||||
#include <IO/Resource/ThrottlerConstraint.h>
|
||||
#include "IO/ISchedulerNode.h"
|
||||
#include "IO/ResourceRequest.h"
|
||||
#include <Common/Scheduler/Nodes/FairPolicy.h>
|
||||
#include <Common/Scheduler/Nodes/ThrottlerConstraint.h>
|
||||
#include "Common/Scheduler/ISchedulerNode.h"
|
||||
#include "Common/Scheduler/ResourceRequest.h"
|
||||
|
||||
using namespace DB;
|
||||
|
||||
using ResourceTest = ResourceTestClass;
|
||||
|
||||
TEST(IOResourceThrottlerConstraint, LeakyBucketConstraint)
|
||||
TEST(SchedulerThrottlerConstraint, LeakyBucketConstraint)
|
||||
{
|
||||
ResourceTest t;
|
||||
EventQueue::TimePoint start = std::chrono::system_clock::now();
|
||||
@ -42,7 +42,7 @@ TEST(IOResourceThrottlerConstraint, LeakyBucketConstraint)
|
||||
t.consumed("A", 10);
|
||||
}
|
||||
|
||||
TEST(IOResourceThrottlerConstraint, Unlimited)
|
||||
TEST(SchedulerThrottlerConstraint, Unlimited)
|
||||
{
|
||||
ResourceTest t;
|
||||
EventQueue::TimePoint start = std::chrono::system_clock::now();
|
||||
@ -59,7 +59,7 @@ TEST(IOResourceThrottlerConstraint, Unlimited)
|
||||
}
|
||||
}
|
||||
|
||||
TEST(IOResourceThrottlerConstraint, Pacing)
|
||||
TEST(SchedulerThrottlerConstraint, Pacing)
|
||||
{
|
||||
ResourceTest t;
|
||||
EventQueue::TimePoint start = std::chrono::system_clock::now();
|
||||
@ -79,7 +79,7 @@ TEST(IOResourceThrottlerConstraint, Pacing)
|
||||
}
|
||||
}
|
||||
|
||||
TEST(IOResourceThrottlerConstraint, BucketFilling)
|
||||
TEST(SchedulerThrottlerConstraint, BucketFilling)
|
||||
{
|
||||
ResourceTest t;
|
||||
EventQueue::TimePoint start = std::chrono::system_clock::now();
|
||||
@ -113,7 +113,7 @@ TEST(IOResourceThrottlerConstraint, BucketFilling)
|
||||
t.consumed("A", 3);
|
||||
}
|
||||
|
||||
TEST(IOResourceThrottlerConstraint, PeekAndAvgLimits)
|
||||
TEST(SchedulerThrottlerConstraint, PeekAndAvgLimits)
|
||||
{
|
||||
ResourceTest t;
|
||||
EventQueue::TimePoint start = std::chrono::system_clock::now();
|
||||
@ -141,7 +141,7 @@ TEST(IOResourceThrottlerConstraint, PeekAndAvgLimits)
|
||||
}
|
||||
}
|
||||
|
||||
TEST(IOResourceThrottlerConstraint, ThrottlerAndFairness)
|
||||
TEST(SchedulerThrottlerConstraint, ThrottlerAndFairness)
|
||||
{
|
||||
ResourceTest t;
|
||||
EventQueue::TimePoint start = std::chrono::system_clock::now();
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/ResourceRequest.h>
|
||||
#include <Common/Scheduler/ResourceRequest.h>
|
||||
#include <atomic>
|
||||
|
||||
namespace DB
|
@ -2,9 +2,9 @@
|
||||
|
||||
#include <base/types.h>
|
||||
|
||||
#include <IO/ResourceRequest.h>
|
||||
#include <IO/ResourceLink.h>
|
||||
#include <IO/ISchedulerConstraint.h>
|
||||
#include <Common/Scheduler/ResourceRequest.h>
|
||||
#include <Common/Scheduler/ResourceLink.h>
|
||||
#include <Common/Scheduler/ISchedulerConstraint.h>
|
||||
|
||||
#include <condition_variable>
|
||||
#include <mutex>
|
@ -2,8 +2,8 @@
|
||||
|
||||
#include <base/types.h>
|
||||
|
||||
#include <IO/ResourceRequest.h>
|
||||
#include <IO/ISchedulerQueue.h>
|
||||
#include <Common/Scheduler/ResourceRequest.h>
|
||||
#include <Common/Scheduler/ISchedulerQueue.h>
|
||||
|
||||
|
||||
namespace DB
|
@ -3,7 +3,7 @@
|
||||
#include <Common/ErrorCodes.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <IO/IResourceManager.h>
|
||||
#include <Common/Scheduler/IResourceManager.h>
|
||||
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
@ -5,8 +5,8 @@
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
|
||||
#include <IO/ISchedulerNode.h>
|
||||
#include <IO/ISchedulerConstraint.h>
|
||||
#include <Common/Scheduler/ISchedulerNode.h>
|
||||
#include <Common/Scheduler/ISchedulerConstraint.h>
|
||||
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
|
@ -14,7 +14,7 @@
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
|
||||
#include <IO/AsyncReadCounters.h>
|
||||
#include <IO/IResourceManager.h>
|
||||
#include <Common/Scheduler/IResourceManager.h>
|
||||
|
||||
#include <Poco/Util/Application.h>
|
||||
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <Common/getRandomASCIIString.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/Throttler.h>
|
||||
#include <IO/ResourceGuard.h>
|
||||
#include <Common/Scheduler/ResourceGuard.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
|
@ -6,7 +6,7 @@
|
||||
|
||||
#include <IO/ReadBufferFromIStream.h>
|
||||
#include <IO/ReadBufferFromS3.h>
|
||||
#include <IO/ResourceGuard.h>
|
||||
#include <Common/Scheduler/ResourceGuard.h>
|
||||
#include <IO/S3/getObjectInfo.h>
|
||||
#include <IO/S3/Requests.h>
|
||||
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <Interpreters/Cache/FileCache_fwd.h>
|
||||
#include <Common/Throttler_fwd.h>
|
||||
#include <Common/Priority.h>
|
||||
#include <IO/ResourceLink.h>
|
||||
#include <Common/Scheduler/ResourceLink.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -11,7 +11,7 @@
|
||||
#include <Common/Throttler.h>
|
||||
#include <Interpreters/Cache/FileCache.h>
|
||||
|
||||
#include <IO/ResourceGuard.h>
|
||||
#include <Common/Scheduler/ResourceGuard.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/S3Common.h>
|
||||
#include <IO/S3/Requests.h>
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/Throttler_fwd.h>
|
||||
#include <IO/ResourceLink.h>
|
||||
#include <Common/Scheduler/ResourceLink.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -59,7 +59,7 @@
|
||||
#include <Access/SettingsConstraintsAndProfileIDs.h>
|
||||
#include <Access/ExternalAuthenticators.h>
|
||||
#include <Access/GSSAcceptor.h>
|
||||
#include <IO/ResourceManagerFactory.h>
|
||||
#include <Common/Scheduler/ResourceManagerFactory.h>
|
||||
#include <Backups/BackupsWorker.h>
|
||||
#include <Dictionaries/Embedded/GeoDictionariesLoader.h>
|
||||
#include <Interpreters/EmbeddedDictionaries.h>
|
||||
|
@ -21,7 +21,7 @@
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/MergeTreeTransactionHolder.h>
|
||||
#include <IO/IResourceManager.h>
|
||||
#include <Common/Scheduler/IResourceManager.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Server/HTTP/HTTPContext.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#if USE_HDFS
|
||||
#include <Storages/HDFS/HDFSCommon.h>
|
||||
#include <IO/ResourceGuard.h>
|
||||
#include <Common/Scheduler/ResourceGuard.h>
|
||||
#include <IO/Progress.h>
|
||||
#include <Common/Throttler.h>
|
||||
#include <Common/safe_cast.h>
|
||||
|
@ -4,7 +4,7 @@
|
||||
|
||||
#include <Storages/HDFS/WriteBufferFromHDFS.h>
|
||||
#include <Storages/HDFS/HDFSCommon.h>
|
||||
#include <IO/ResourceGuard.h>
|
||||
#include <Common/Scheduler/ResourceGuard.h>
|
||||
#include <Common/Throttler.h>
|
||||
#include <Common/safe_cast.h>
|
||||
#include <hdfs/hdfs.h>
|
||||
|
@ -4,15 +4,15 @@
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <IO/ISchedulerNode.h>
|
||||
#include <IO/IResourceManager.h>
|
||||
#include <IO/Resource/FairPolicy.h>
|
||||
#include <IO/Resource/PriorityPolicy.h>
|
||||
#include <IO/Resource/SemaphoreConstraint.h>
|
||||
#include <IO/Resource/ThrottlerConstraint.h>
|
||||
#include <IO/Resource/FifoQueue.h>
|
||||
#include <Common/Scheduler/ISchedulerNode.h>
|
||||
#include <Common/Scheduler/IResourceManager.h>
|
||||
#include <Common/Scheduler/Nodes/FairPolicy.h>
|
||||
#include <Common/Scheduler/Nodes/PriorityPolicy.h>
|
||||
#include <Common/Scheduler/Nodes/SemaphoreConstraint.h>
|
||||
#include <Common/Scheduler/Nodes/ThrottlerConstraint.h>
|
||||
#include <Common/Scheduler/Nodes/FifoQueue.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include "IO/ResourceRequest.h"
|
||||
#include "Common/Scheduler/ResourceRequest.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
|
Loading…
Reference in New Issue
Block a user