Remove unused field from buffers

This commit is contained in:
alesapin 2022-04-08 15:06:43 +02:00
parent 90a863bbc2
commit 0db8808f2e
6 changed files with 10 additions and 18 deletions

View File

@ -71,7 +71,7 @@ std::unique_ptr<ReadBufferFromFileBase> DiskAzureBlobStorage::readFile(
LOG_TEST(log, "Read from file by path: {}", backQuote(metadata_disk->getPath() + path)); LOG_TEST(log, "Read from file by path: {}", backQuote(metadata_disk->getPath() + path));
auto reader_impl = std::make_unique<ReadBufferFromAzureBlobStorageGather>( auto reader_impl = std::make_unique<ReadBufferFromAzureBlobStorageGather>(
path, blob_container_client, metadata, settings->max_single_read_retries, blob_container_client, metadata, settings->max_single_read_retries,
settings->max_single_download_retries, read_settings); settings->max_single_download_retries, read_settings);
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)

View File

@ -168,7 +168,7 @@ std::unique_ptr<ReadBufferFromFileBase> DiskWebServer::readFile(const String & p
RemoteMetadata meta(path, remote_path); RemoteMetadata meta(path, remote_path);
meta.remote_fs_objects.emplace_back(std::make_pair(remote_path, iter->second.size)); meta.remote_fs_objects.emplace_back(std::make_pair(remote_path, iter->second.size));
auto web_impl = std::make_unique<ReadBufferFromWebServerGather>(path, url, meta, getContext(), read_settings); auto web_impl = std::make_unique<ReadBufferFromWebServerGather>(url, meta, getContext(), read_settings);
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
{ {

View File

@ -82,7 +82,7 @@ std::unique_ptr<ReadBufferFromFileBase> DiskHDFS::readFile(const String & path,
"Read from file by path: {}. Existing HDFS objects: {}", "Read from file by path: {}. Existing HDFS objects: {}",
backQuote(metadata_disk->getPath() + path), metadata.remote_fs_objects.size()); backQuote(metadata_disk->getPath() + path), metadata.remote_fs_objects.size());
auto hdfs_impl = std::make_unique<ReadBufferFromHDFSGather>(path, config, remote_fs_root_path, metadata, read_settings); auto hdfs_impl = std::make_unique<ReadBufferFromHDFSGather>(config, remote_fs_root_path, metadata, read_settings);
auto buf = std::make_unique<ReadIndirectBufferFromRemoteFS>(std::move(hdfs_impl)); auto buf = std::make_unique<ReadIndirectBufferFromRemoteFS>(std::move(hdfs_impl));
return std::make_unique<SeekAvoidingReadBuffer>(std::move(buf), settings->min_bytes_for_seek); return std::make_unique<SeekAvoidingReadBuffer>(std::move(buf), settings->min_bytes_for_seek);
} }

View File

@ -83,11 +83,10 @@ SeekableReadBufferPtr ReadBufferFromHDFSGather::createImplementationBuffer(const
#endif #endif
ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(const RemoteMetadata & metadata_, const ReadSettings & settings_, const String & path_) ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(const RemoteMetadata & metadata_, const ReadSettings & settings_)
: ReadBuffer(nullptr, 0) : ReadBuffer(nullptr, 0)
, metadata(metadata_) , metadata(metadata_)
, settings(settings_) , settings(settings_)
, canonical_path(path_)
, log(&Poco::Logger::get("ReadBufferFromRemoteFSGather")) , log(&Poco::Logger::get("ReadBufferFromRemoteFSGather"))
{ {
} }

View File

@ -27,8 +27,7 @@ friend class ReadIndirectBufferFromRemoteFS;
public: public:
ReadBufferFromRemoteFSGather( ReadBufferFromRemoteFSGather(
const RemoteMetadata & metadata_, const RemoteMetadata & metadata_,
const ReadSettings & settings_, const ReadSettings & settings_);
const String & path_);
String getFileName() const; String getFileName() const;
@ -89,8 +88,6 @@ private:
*/ */
size_t bytes_to_ignore = 0; size_t bytes_to_ignore = 0;
String canonical_path;
Poco::Logger * log; Poco::Logger * log;
}; };
@ -101,13 +98,12 @@ class ReadBufferFromS3Gather final : public ReadBufferFromRemoteFSGather
{ {
public: public:
ReadBufferFromS3Gather( ReadBufferFromS3Gather(
const String & path_,
std::shared_ptr<Aws::S3::S3Client> client_ptr_, std::shared_ptr<Aws::S3::S3Client> client_ptr_,
const String & bucket_, const String & bucket_,
IDiskRemote::Metadata metadata_, IDiskRemote::Metadata metadata_,
size_t max_single_read_retries_, size_t max_single_read_retries_,
const ReadSettings & settings_) const ReadSettings & settings_)
: ReadBufferFromRemoteFSGather(metadata_, settings_, path_) : ReadBufferFromRemoteFSGather(metadata_, settings_)
, client_ptr(std::move(client_ptr_)) , client_ptr(std::move(client_ptr_))
, bucket(bucket_) , bucket(bucket_)
, max_single_read_retries(max_single_read_retries_) , max_single_read_retries(max_single_read_retries_)
@ -130,13 +126,12 @@ class ReadBufferFromAzureBlobStorageGather final : public ReadBufferFromRemoteFS
{ {
public: public:
ReadBufferFromAzureBlobStorageGather( ReadBufferFromAzureBlobStorageGather(
const String & path_,
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_, std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
IDiskRemote::Metadata metadata_, IDiskRemote::Metadata metadata_,
size_t max_single_read_retries_, size_t max_single_read_retries_,
size_t max_single_download_retries_, size_t max_single_download_retries_,
const ReadSettings & settings_) const ReadSettings & settings_)
: ReadBufferFromRemoteFSGather(metadata_, settings_, path_) : ReadBufferFromRemoteFSGather(metadata_, settings_)
, blob_container_client(blob_container_client_) , blob_container_client(blob_container_client_)
, max_single_read_retries(max_single_read_retries_) , max_single_read_retries(max_single_read_retries_)
, max_single_download_retries(max_single_download_retries_) , max_single_download_retries(max_single_download_retries_)
@ -157,12 +152,11 @@ class ReadBufferFromWebServerGather final : public ReadBufferFromRemoteFSGather
{ {
public: public:
ReadBufferFromWebServerGather( ReadBufferFromWebServerGather(
const String & path_,
const String & uri_, const String & uri_,
RemoteMetadata metadata_, RemoteMetadata metadata_,
ContextPtr context_, ContextPtr context_,
const ReadSettings & settings_) const ReadSettings & settings_)
: ReadBufferFromRemoteFSGather(metadata_, settings_, path_) : ReadBufferFromRemoteFSGather(metadata_, settings_)
, uri(uri_) , uri(uri_)
, context(context_) , context(context_)
{ {
@ -182,12 +176,11 @@ class ReadBufferFromHDFSGather final : public ReadBufferFromRemoteFSGather
{ {
public: public:
ReadBufferFromHDFSGather( ReadBufferFromHDFSGather(
const String & path_,
const Poco::Util::AbstractConfiguration & config_, const Poco::Util::AbstractConfiguration & config_,
const String & hdfs_uri_, const String & hdfs_uri_,
IDiskRemote::Metadata metadata_, IDiskRemote::Metadata metadata_,
const ReadSettings & settings_) const ReadSettings & settings_)
: ReadBufferFromRemoteFSGather(metadata_, settings_, path_) : ReadBufferFromRemoteFSGather(metadata_, settings_)
, config(config_) , config(config_)
{ {
const size_t begin_of_path = hdfs_uri_.find('/', hdfs_uri_.find("//") + 2); const size_t begin_of_path = hdfs_uri_.find('/', hdfs_uri_.find("//") + 2);

View File

@ -237,7 +237,7 @@ std::unique_ptr<ReadBufferFromFileBase> DiskS3::readFile(const String & path, co
} }
auto s3_impl = std::make_unique<ReadBufferFromS3Gather>( auto s3_impl = std::make_unique<ReadBufferFromS3Gather>(
path, settings->client, bucket, metadata, settings->client, bucket, metadata,
settings->s3_max_single_read_retries, disk_read_settings); settings->s3_max_single_read_retries, disk_read_settings);
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)