Fix split build

This commit is contained in:
kssenii 2021-10-19 20:41:52 +03:00
parent df5dc4e991
commit 143e912e4e
18 changed files with 47 additions and 45 deletions

View File

@ -77,6 +77,7 @@ add_headers_and_sources(clickhouse_common_io IO)
add_headers_and_sources(clickhouse_common_io IO/S3) add_headers_and_sources(clickhouse_common_io IO/S3)
list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_delete.cpp) list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_delete.cpp)
add_headers_and_sources(dbms Disks/IO)
if (USE_SQLITE) if (USE_SQLITE)
add_headers_and_sources(dbms Databases/SQLite) add_headers_and_sources(dbms Databases/SQLite)
endif() endif()

View File

@ -11,7 +11,6 @@
#include <IO/ReadBuffer.h> #include <IO/ReadBuffer.h>
#include <IO/BufferWithOwnMemory.h> #include <IO/BufferWithOwnMemory.h>
#include <Compression/CompressionInfo.h> #include <Compression/CompressionInfo.h>
#include <IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <IO/Operators.h> #include <IO/Operators.h>
#include <Disks/DiskRestartProxy.h> #include <Disks/DiskRestartProxy.h>

View File

@ -3,17 +3,17 @@
#include <base/logger_useful.h> #include <base/logger_useful.h>
#include <Common/escapeForFileName.h> #include <Common/escapeForFileName.h>
#include <Disks/IDiskRemote.h>
#include <IO/ReadBufferFromWebServer.h>
#include <IO/ReadWriteBufferFromHTTP.h> #include <IO/ReadWriteBufferFromHTTP.h>
#include <IO/ReadIndirectBufferFromRemoteFS.h>
#include <IO/SeekAvoidingReadBuffer.h> #include <IO/SeekAvoidingReadBuffer.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <IO/ThreadPoolRemoteFSReader.h>
#include <IO/AsynchronousReadIndirectBufferFromRemoteFS.h> #include <Disks/IDiskRemote.h>
#include <Disks/ReadBufferFromRemoteFSGather.h> #include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
#include <Storages/MergeTree/MergeTreeData.h> #include <Storages/MergeTree/MergeTreeData.h>

View File

@ -1,12 +1,14 @@
#include <Disks/HDFS/DiskHDFS.h> #include <Disks/HDFS/DiskHDFS.h>
#include <IO/SeekAvoidingReadBuffer.h> #include <IO/SeekAvoidingReadBuffer.h>
#include <IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
#include <IO/ReadIndirectBufferFromRemoteFS.h>
#include <IO/WriteIndirectBufferFromRemoteFS.h>
#include <Disks/ReadBufferFromRemoteFSGather.h>
#include <Storages/HDFS/WriteBufferFromHDFS.h> #include <Storages/HDFS/WriteBufferFromHDFS.h>
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
#include <base/logger_useful.h> #include <base/logger_useful.h>
#include <base/FnTraits.h> #include <base/FnTraits.h>

View File

@ -12,7 +12,7 @@
#include <Common/checkStackSize.h> #include <Common/checkStackSize.h>
#include <boost/algorithm/string.hpp> #include <boost/algorithm/string.hpp>
#include <Common/filesystemHelpers.h> #include <Common/filesystemHelpers.h>
#include <IO/ThreadPoolRemoteFSReader.h> #include <Disks/IO/ThreadPoolRemoteFSReader.h>
namespace DB namespace DB

View File

@ -1,7 +1,8 @@
#include "AsynchronousReadIndirectBufferFromRemoteFS.h" #include "AsynchronousReadIndirectBufferFromRemoteFS.h"
#include <Common/Stopwatch.h> #include <Common/Stopwatch.h>
#include <IO/ThreadPoolRemoteFSReader.h> #include <Disks/IO/ThreadPoolRemoteFSReader.h>
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
#include <base/logger_useful.h> #include <base/logger_useful.h>
@ -46,7 +47,13 @@ AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRe
, min_bytes_for_seek(min_bytes_for_seek_) , min_bytes_for_seek(min_bytes_for_seek_)
{ {
ProfileEvents::increment(ProfileEvents::RemoteFSAsyncBuffers); ProfileEvents::increment(ProfileEvents::RemoteFSAsyncBuffers);
buffer_events += impl->getFileName() + " : "; buffer_events += "Events for buffer: " + impl->getFileName() + " : ";
}
String AsynchronousReadIndirectBufferFromRemoteFS::getFileName() const
{
return impl->getFileName();
} }
@ -92,9 +99,9 @@ void AsynchronousReadIndirectBufferFromRemoteFS::prefetch()
} }
void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilPosition(size_t offset) void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilPosition(size_t position)
{ {
buffer_events += "-- Set last offset " + toString(offset) + "--"; buffer_events += "-- Set last offset " + toString(position) + "--";
if (prefetch_future.valid()) if (prefetch_future.valid())
{ {
LOG_DEBUG(&Poco::Logger::get("kssenii"), buffer_events); LOG_DEBUG(&Poco::Logger::get("kssenii"), buffer_events);
@ -108,13 +115,14 @@ void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilPosition(size_t off
// prefetch_future = {}; // prefetch_future = {};
} }
last_offset = offset; last_offset = position;
impl->setReadUntilPosition(offset); impl->setReadUntilPosition(position);
} }
bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl()
{ {
LOG_DEBUG(&Poco::Logger::get("kssenii"), buffer_events);
/// Everything is already read. /// Everything is already read.
if (absolute_position == last_offset) if (absolute_position == last_offset)
return false; return false;
@ -163,12 +171,7 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl()
prefetch_future = {}; prefetch_future = {};
/// TODO: it does not really seem to improve anything to call prefetch() here, LOG_DEBUG(&Poco::Logger::get("kssenii"), buffer_events);
/// but it does not make any worse at the same time.
/// Need to test, it might be useful because in fact sometimes (minority of cases though)
/// we can read without prefetching several times in a row.
prefetch();
return size; return size;
} }

View File

@ -6,14 +6,14 @@
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <IO/AsynchronousReader.h> #include <IO/AsynchronousReader.h>
#include <Disks/ReadBufferFromRemoteFSGather.h>
#include <Disks/IDiskRemote.h>
#include <utility> #include <utility>
namespace DB namespace DB
{ {
class ReadBufferFromRemoteFSGather;
/** /**
* Reads data from S3/HDFS/Web using stored paths in metadata. * Reads data from S3/HDFS/Web using stored paths in metadata.
* This class is an asynchronous version of ReadIndirectBufferFromRemoteFS. * This class is an asynchronous version of ReadIndirectBufferFromRemoteFS.
@ -43,7 +43,7 @@ public:
off_t getPosition() override { return absolute_position - available(); } off_t getPosition() override { return absolute_position - available(); }
String getFileName() const override { return impl->getFileName(); } String getFileName() const override;
void prefetch() override; void prefetch() override;

View File

@ -2,7 +2,7 @@
#include <Disks/IDiskRemote.h> #include <Disks/IDiskRemote.h>
#include <IO/SeekableReadBuffer.h> #include <IO/SeekableReadBuffer.h>
#include <IO/ReadBufferFromWebServer.h> #include <Disks/IO/ReadBufferFromWebServer.h>
#if USE_AWS_S3 #if USE_AWS_S3
#include <IO/ReadBufferFromS3.h> #include <IO/ReadBufferFromS3.h>
@ -21,12 +21,6 @@ namespace fs = std::filesystem;
namespace DB namespace DB
{ {
namespace ErrorCodes
{
extern const int CANNOT_SEEK_THROUGH_FILE;
}
#if USE_AWS_S3 #if USE_AWS_S3
SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const String & path, size_t offset) const SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const String & path, size_t offset) const
{ {

View File

@ -17,6 +17,7 @@ namespace ErrorCodes
{ {
extern const int CANNOT_SEEK_THROUGH_FILE; extern const int CANNOT_SEEK_THROUGH_FILE;
extern const int SEEK_POSITION_OUT_OF_BOUND; extern const int SEEK_POSITION_OUT_OF_BOUND;
extern const int LOGICAL_ERROR;
} }
@ -49,7 +50,7 @@ std::unique_ptr<ReadBuffer> ReadBufferFromWebServer::initialize()
if (last_offset) if (last_offset)
{ {
if (last_offset < offset) if (last_offset < offset)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyound right offset ({} > {})", offset, last_offset - 1); throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, last_offset - 1);
headers.emplace_back(std::make_pair("Range", fmt::format("bytes={}-{}", offset, last_offset - 1))); headers.emplace_back(std::make_pair("Range", fmt::format("bytes={}-{}", offset, last_offset - 1)));
LOG_DEBUG(log, "Reading with range: {}-{}", offset, last_offset); LOG_DEBUG(log, "Reading with range: {}-{}", offset, last_offset);
@ -133,7 +134,7 @@ bool ReadBufferFromWebServer::nextImpl()
return false; return false;
if (last_offset < offset) if (last_offset < offset)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyound right offset ({} > {})", offset, last_offset - 1); throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, last_offset - 1);
} }
if (impl) if (impl)
@ -155,7 +156,7 @@ bool ReadBufferFromWebServer::nextImpl()
{ {
/** /**
* impl was initialized before, pass position() to it to make * impl was initialized before, pass position() to it to make
* sure there is no pending data which was not read, becuase * sure there is no pending data which was not read, because
* this branch means we read sequentially. * this branch means we read sequentially.
*/ */
impl->position() = position(); impl->position() = position();

View File

@ -1,6 +1,6 @@
#include "ReadIndirectBufferFromRemoteFS.h" #include "ReadIndirectBufferFromRemoteFS.h"
#include <Disks/ReadBufferFromRemoteFSGather.h> #include <Disks/IO/ReadBufferFromRemoteFSGather.h>
namespace ProfileEvents namespace ProfileEvents

View File

@ -1,4 +1,4 @@
#include <IO/ThreadPoolRemoteFSReader.h> #include "ThreadPoolRemoteFSReader.h"
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/ProfileEvents.h> #include <Common/ProfileEvents.h>
@ -8,7 +8,7 @@
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <IO/SeekableReadBuffer.h> #include <IO/SeekableReadBuffer.h>
#include <Disks/ReadBufferFromRemoteFSGather.h> #include <Disks/IO/ReadBufferFromRemoteFSGather.h>
#include <future> #include <future>
#include <iostream> #include <iostream>

View File

@ -17,17 +17,19 @@
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <Common/thread_local_rng.h> #include <Common/thread_local_rng.h>
#include <Disks/ReadBufferFromRemoteFSGather.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <IO/ReadBufferFromS3.h> #include <IO/ReadBufferFromS3.h>
#include <IO/WriteIndirectBufferFromRemoteFS.h>
#include <IO/ReadBufferFromString.h> #include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/SeekAvoidingReadBuffer.h> #include <IO/SeekAvoidingReadBuffer.h>
#include <IO/WriteBufferFromS3.h> #include <IO/WriteBufferFromS3.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <IO/ReadIndirectBufferFromRemoteFS.h>
#include <IO/AsynchronousReadIndirectBufferFromRemoteFS.h> #include <Disks/IO/ReadBufferFromRemoteFSGather.h>
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
#include <aws/s3/model/CopyObjectRequest.h> // Y_IGNORE #include <aws/s3/model/CopyObjectRequest.h> // Y_IGNORE
#include <aws/s3/model/DeleteObjectsRequest.h> // Y_IGNORE #include <aws/s3/model/DeleteObjectsRequest.h> // Y_IGNORE