mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Move initialization out of constructor
This commit is contained in:
parent
f081edce74
commit
b2cab429a7
@ -68,13 +68,17 @@ bool IFileCache::shouldBypassCache()
|
|||||||
LRUFileCache::LRUFileCache(const String & cache_base_path_, size_t max_size_, size_t max_element_size_, size_t max_file_segment_size_)
|
LRUFileCache::LRUFileCache(const String & cache_base_path_, size_t max_size_, size_t max_element_size_, size_t max_file_segment_size_)
|
||||||
: IFileCache(cache_base_path_, max_size_, max_element_size_, max_file_segment_size_)
|
: IFileCache(cache_base_path_, max_size_, max_element_size_, max_file_segment_size_)
|
||||||
, log(&Poco::Logger::get("LRUFileCache"))
|
, log(&Poco::Logger::get("LRUFileCache"))
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
void LRUFileCache::initialize()
|
||||||
{
|
{
|
||||||
if (fs::exists(cache_base_path))
|
if (fs::exists(cache_base_path))
|
||||||
loadCacheInfoIntoMemory();
|
loadCacheInfoIntoMemory();
|
||||||
else
|
else
|
||||||
fs::create_directories(cache_base_path);
|
fs::create_directories(cache_base_path);
|
||||||
|
|
||||||
startup_restore_finished = true;
|
is_initialized = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
void LRUFileCache::useCell(
|
void LRUFileCache::useCell(
|
||||||
@ -448,7 +452,7 @@ void LRUFileCache::remove(
|
|||||||
{
|
{
|
||||||
fs::remove(cache_file_path);
|
fs::remove(cache_file_path);
|
||||||
|
|
||||||
if (startup_restore_finished && offsets.empty())
|
if (is_initialized && offsets.empty())
|
||||||
{
|
{
|
||||||
auto key_path = getPathInLocalCache(key);
|
auto key_path = getPathInLocalCache(key);
|
||||||
|
|
||||||
|
@ -38,6 +38,9 @@ public:
|
|||||||
|
|
||||||
virtual ~IFileCache() = default;
|
virtual ~IFileCache() = default;
|
||||||
|
|
||||||
|
/// Restore cache from local filesystem.
|
||||||
|
virtual void initialize() = 0;
|
||||||
|
|
||||||
static bool shouldBypassCache();
|
static bool shouldBypassCache();
|
||||||
|
|
||||||
/// Cache capacity in bytes.
|
/// Cache capacity in bytes.
|
||||||
@ -71,6 +74,8 @@ protected:
|
|||||||
size_t max_element_size;
|
size_t max_element_size;
|
||||||
size_t max_file_segment_size;
|
size_t max_file_segment_size;
|
||||||
|
|
||||||
|
bool is_initialized = false;
|
||||||
|
|
||||||
mutable std::mutex mutex;
|
mutable std::mutex mutex;
|
||||||
|
|
||||||
virtual bool tryReserve(
|
virtual bool tryReserve(
|
||||||
@ -108,6 +113,8 @@ public:
|
|||||||
|
|
||||||
FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size) override;
|
FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size) override;
|
||||||
|
|
||||||
|
void initialize() override;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
using FileKeyAndOffset = std::pair<Key, size_t>;
|
using FileKeyAndOffset = std::pair<Key, size_t>;
|
||||||
using LRUQueue = std::list<FileKeyAndOffset>;
|
using LRUQueue = std::list<FileKeyAndOffset>;
|
||||||
@ -144,9 +151,6 @@ private:
|
|||||||
size_t current_size = 0;
|
size_t current_size = 0;
|
||||||
Poco::Logger * log;
|
Poco::Logger * log;
|
||||||
|
|
||||||
/// Needed in loadCacheInfoIntoMemory() method.
|
|
||||||
bool startup_restore_finished = false;
|
|
||||||
|
|
||||||
FileSegments getImpl(
|
FileSegments getImpl(
|
||||||
const Key & key, const FileSegment::Range & range,
|
const Key & key, const FileSegment::Range & range,
|
||||||
std::lock_guard<std::mutex> & cache_lock);
|
std::lock_guard<std::mutex> & cache_lock);
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
#include <Disks/RemoteDisksCommon.h>
|
#include <Disks/RemoteDisksCommon.h>
|
||||||
#include <Common/getRandomASCIIString.h>
|
#include <Common/getRandomASCIIString.h>
|
||||||
#include <Common/FileCache_fwd.h>
|
|
||||||
#include <Common/FileCacheFactory.h>
|
#include <Common/FileCacheFactory.h>
|
||||||
|
#include <Common/FileCache.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -74,6 +74,9 @@ FileCachePtr getCachePtrForDisk(
|
|||||||
size_t max_cache_elements = config.getUInt64(config_prefix + ".data_cache_max_elements", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS);
|
size_t max_cache_elements = config.getUInt64(config_prefix + ".data_cache_max_elements", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_ELEMENTS);
|
||||||
size_t max_file_segment_size = config.getUInt64(config_prefix + ".max_file_segment_size", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE);
|
size_t max_file_segment_size = config.getUInt64(config_prefix + ".max_file_segment_size", REMOTE_FS_OBJECTS_CACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE);
|
||||||
|
|
||||||
return FileCacheFactory::instance().getOrCreate(cache_base_path, max_cache_size, max_cache_elements, max_file_segment_size);
|
auto cache = FileCacheFactory::instance().getOrCreate(cache_base_path, max_cache_size, max_cache_elements, max_file_segment_size);
|
||||||
|
cache->initialize();
|
||||||
|
return cache;
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -140,6 +140,7 @@ void ParquetBlockInputFormat::prepareReader()
|
|||||||
nested_table_names = Nested::getAllTableNames(getPort().getHeader());
|
nested_table_names = Nested::getAllTableNames(getPort().getHeader());
|
||||||
|
|
||||||
int index = 0;
|
int index = 0;
|
||||||
|
std::cerr << "\n\nnum fields: " << schema->num_fields() << "\n\n";
|
||||||
for (int i = 0; i < schema->num_fields(); ++i)
|
for (int i = 0; i < schema->num_fields(); ++i)
|
||||||
{
|
{
|
||||||
/// STRUCT type require the number of indexes equal to the number of
|
/// STRUCT type require the number of indexes equal to the number of
|
||||||
@ -147,6 +148,7 @@ void ParquetBlockInputFormat::prepareReader()
|
|||||||
/// count the number of indices we need for this type.
|
/// count the number of indices we need for this type.
|
||||||
int indexes_count = countIndicesForType(schema->field(i)->type());
|
int indexes_count = countIndicesForType(schema->field(i)->type());
|
||||||
const auto & name = schema->field(i)->name();
|
const auto & name = schema->field(i)->name();
|
||||||
|
std::cerr << "name: " << name << "\n";
|
||||||
if (getPort().getHeader().has(name) || nested_table_names.contains(name))
|
if (getPort().getHeader().has(name) || nested_table_names.contains(name))
|
||||||
{
|
{
|
||||||
for (int j = 0; j != indexes_count; ++j)
|
for (int j = 0; j != indexes_count; ++j)
|
||||||
@ -154,6 +156,7 @@ void ParquetBlockInputFormat::prepareReader()
|
|||||||
}
|
}
|
||||||
index += indexes_count;
|
index += indexes_count;
|
||||||
}
|
}
|
||||||
|
std::cerr << "indexes count: " << index << "\n\n";
|
||||||
}
|
}
|
||||||
|
|
||||||
ParquetSchemaReader::ParquetSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) : ISchemaReader(in_), format_settings(format_settings_)
|
ParquetSchemaReader::ParquetSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) : ISchemaReader(in_), format_settings(format_settings_)
|
||||||
|
Loading…
Reference in New Issue
Block a user