mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-14 10:22:10 +00:00
fix
This commit is contained in:
parent
1f90c8dc22
commit
1758ff2d32
@ -11,10 +11,10 @@ class ASTStorage;
|
||||
|
||||
#define FILELOG_RELATED_SETTINGS(M) \
|
||||
/* default is stream_poll_timeout_ms */ \
|
||||
M(Milliseconds, filelog_poll_timeout_ms, 0, "Timeout for single poll from FileLog.", 0) \
|
||||
M(UInt64, filelog_poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single filelog poll.", 0) \
|
||||
M(UInt64, filelog_max_block_size, 0, "Number of row collected by poll(s) for flushing data from filelog.", 0) \
|
||||
M(UInt64, filelog_max_threads, 8, "Number of max threads to parse files, default is 8", 0)
|
||||
M(Milliseconds, poll_timeout_ms, 0, "Timeout for single poll from StorageFileLog.", 0) \
|
||||
M(UInt64, poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single StorageFileLog poll.", 0) \
|
||||
M(UInt64, max_block_size, 0, "Number of row collected by poll(s) for flushing data from StorageFileLog.", 0) \
|
||||
M(UInt64, max_threads, 8, "Number of max threads to parse files, default is 8", 0)
|
||||
|
||||
#define LIST_OF_FILELOG_SETTINGS(M) \
|
||||
FILELOG_RELATED_SETTINGS(M) \
|
||||
|
@ -17,14 +17,16 @@ FileLogSource::FileLogSource(
|
||||
StorageFileLog & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const ContextPtr & context_,
|
||||
const Names & columns,
|
||||
size_t max_block_size_,
|
||||
size_t poll_time_out_,
|
||||
size_t stream_number_,
|
||||
size_t max_streams_number_)
|
||||
: SourceWithProgress(metadata_snapshot_->getSampleBlockWithVirtuals(storage_.getVirtuals()))
|
||||
: SourceWithProgress(metadata_snapshot_->getSampleBlockForColumns(columns, storage_.getVirtuals(), storage_.getStorageID()))
|
||||
, storage(storage_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, context(context_)
|
||||
, column_names(columns)
|
||||
, max_block_size(max_block_size_)
|
||||
, poll_time_out(poll_time_out_)
|
||||
, stream_number(stream_number_)
|
||||
@ -34,19 +36,20 @@ FileLogSource::FileLogSource(
|
||||
metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID()))
|
||||
{
|
||||
buffer = std::make_unique<ReadBufferFromFileLog>(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_);
|
||||
/// The last FileLogSource responsible for open files
|
||||
if (stream_number == max_streams_number - 1)
|
||||
{
|
||||
storage.openFilesAndSetPos();
|
||||
}
|
||||
}
|
||||
|
||||
FileLogSource::~FileLogSource()
|
||||
{
|
||||
/// The last FileLogSource responsible for close files
|
||||
if (stream_number == max_streams_number - 1)
|
||||
auto & file_infos = storage.getFileInfos();
|
||||
|
||||
size_t files_per_stream = file_infos.file_names.size() / max_streams_number;
|
||||
size_t start = stream_number * files_per_stream;
|
||||
size_t end = stream_number == max_streams_number - 1 ? file_infos.file_names.size() : (stream_number + 1) * files_per_stream;
|
||||
|
||||
/// Each stream responsible for close it's files and store meta
|
||||
for (size_t i = start; i < end; ++i)
|
||||
{
|
||||
storage.closeFilesAndStoreMeta();
|
||||
storage.closeFileAndStoreMeta(file_infos.file_names[i]);
|
||||
}
|
||||
}
|
||||
|
||||
@ -99,14 +102,21 @@ Chunk FileLogSource::generate()
|
||||
if (total_rows == 0)
|
||||
return {};
|
||||
|
||||
auto result_columns = executor.getResultColumns();
|
||||
auto result_block = non_virtual_header.cloneWithColumns(executor.getResultColumns());
|
||||
auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns));
|
||||
|
||||
for (auto & column : virtual_columns)
|
||||
{
|
||||
result_columns.emplace_back(std::move(column));
|
||||
}
|
||||
for (const auto & column : virtual_block.getColumnsWithTypeAndName())
|
||||
result_block.insert(column);
|
||||
|
||||
return Chunk(std::move(result_columns), total_rows);
|
||||
auto converting_dag = ActionsDAG::makeConvertingActions(
|
||||
result_block.cloneEmpty().getColumnsWithTypeAndName(),
|
||||
getPort().getHeader().getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
|
||||
auto converting_actions = std::make_shared<ExpressionActions>(std::move(converting_dag));
|
||||
converting_actions->execute(result_block);
|
||||
|
||||
return Chunk(result_block.getColumns(), result_block.rows());
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <Storages/FileLog/ReadBufferFromFileLog.h>
|
||||
#include <Storages/FileLog/StorageFileLog.h>
|
||||
|
||||
|
||||
namespace Poco
|
||||
{
|
||||
class Logger;
|
||||
@ -19,6 +18,7 @@ public:
|
||||
StorageFileLog & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const ContextPtr & context_,
|
||||
const Names & columns,
|
||||
size_t max_block_size_,
|
||||
size_t poll_time_out_,
|
||||
size_t stream_number_,
|
||||
@ -37,6 +37,7 @@ private:
|
||||
StorageFileLog & storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
ContextPtr context;
|
||||
Names column_names;
|
||||
UInt64 max_block_size;
|
||||
|
||||
size_t poll_time_out;
|
||||
|
@ -31,15 +31,8 @@ ReadBufferFromFileLog::ReadBufferFromFileLog(
|
||||
, stream_number(stream_number_)
|
||||
, max_streams_number(max_streams_number_)
|
||||
{
|
||||
cleanUnprocessed();
|
||||
allowed = false;
|
||||
}
|
||||
|
||||
void ReadBufferFromFileLog::cleanUnprocessed()
|
||||
{
|
||||
records.clear();
|
||||
current = records.begin();
|
||||
BufferBase::set(nullptr, 0, 0);
|
||||
allowed = false;
|
||||
}
|
||||
|
||||
bool ReadBufferFromFileLog::poll()
|
||||
@ -50,11 +43,10 @@ bool ReadBufferFromFileLog::poll()
|
||||
return true;
|
||||
}
|
||||
|
||||
buffer_status = BufferStatus::NO_RECORD_RETURNED;
|
||||
|
||||
auto new_records = pollBatch(batch_size);
|
||||
if (new_records.empty())
|
||||
{
|
||||
buffer_status = BufferStatus::NO_RECORD_RETURNED;
|
||||
LOG_TRACE(log, "No records returned");
|
||||
return false;
|
||||
}
|
||||
@ -106,33 +98,28 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_
|
||||
|
||||
for (size_t i = start; i < end; ++i)
|
||||
{
|
||||
auto file_name = file_infos.file_names[i];
|
||||
auto & file_ctx = file_infos.context_by_name.at(file_name);
|
||||
const auto & file_name = file_infos.file_names[i];
|
||||
|
||||
auto & file_ctx = StorageFileLog::findInMap(file_infos.context_by_name, file_name);
|
||||
if (file_ctx.status == StorageFileLog::FileStatus::NO_CHANGE)
|
||||
continue;
|
||||
|
||||
auto & file_meta = file_infos.meta_by_inode.at(file_infos.inode_by_name.at(file_name));
|
||||
auto & file_meta = StorageFileLog::findInMap(file_infos.meta_by_inode, file_ctx.inode);
|
||||
|
||||
Record record;
|
||||
while (read_records_size < need_records_size && static_cast<UInt64>(file_ctx.reader.tellg()) < file_meta.last_open_end)
|
||||
{
|
||||
if (!file_ctx.reader.good())
|
||||
{
|
||||
throw Exception("Can not read from file " + file_name + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM);
|
||||
}
|
||||
UInt64 start_offset = file_ctx.reader.tellg();
|
||||
StorageFileLog::assertStreamGood(file_ctx.reader);
|
||||
|
||||
std::getline(file_ctx.reader, record.data);
|
||||
record.file_name = file_name;
|
||||
record.offset = start_offset;
|
||||
record.offset = file_ctx.reader.tellg();
|
||||
new_records.emplace_back(record);
|
||||
++read_records_size;
|
||||
}
|
||||
|
||||
UInt64 current_position = file_ctx.reader.tellg();
|
||||
if (!file_ctx.reader.good())
|
||||
{
|
||||
throw Exception("Can not read from file " + file_name + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM);
|
||||
}
|
||||
StorageFileLog::assertStreamGood(file_ctx.reader);
|
||||
|
||||
file_meta.last_writen_position = current_position;
|
||||
|
||||
|
@ -83,14 +83,10 @@ private:
|
||||
|
||||
using TaskThread = BackgroundSchedulePool::TaskHolder;
|
||||
|
||||
TaskThread wait_task;
|
||||
|
||||
Records pollBatch(size_t batch_size_);
|
||||
|
||||
void readNewRecords(Records & new_records, size_t batch_size_);
|
||||
|
||||
void cleanUnprocessed();
|
||||
|
||||
bool nextImpl() override;
|
||||
};
|
||||
}
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Disks/StoragePolicy.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
@ -19,9 +20,9 @@
|
||||
#include <Storages/FileLog/StorageFileLog.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/StorageMaterializedView.h>
|
||||
#include <Poco/File.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/Macros.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
@ -41,6 +42,7 @@ namespace ErrorCodes
|
||||
extern const int READ_META_FILE_FAILED;
|
||||
extern const int FILE_STREAM_ERROR;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int TABLE_METADATA_ALREADY_EXISTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -54,20 +56,24 @@ StorageFileLog::StorageFileLog(
|
||||
const StorageID & table_id_,
|
||||
ContextPtr context_,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & relative_path_,
|
||||
const String & path_,
|
||||
const String & relative_data_path_,
|
||||
const String & format_name_,
|
||||
std::unique_ptr<FileLogSettings> settings,
|
||||
const String & comment,
|
||||
bool attach)
|
||||
: IStorage(table_id_)
|
||||
, WithContext(context_->getGlobalContext())
|
||||
, filelog_settings(std::move(settings))
|
||||
, path(getContext()->getUserFilesPath() + "/" + relative_path_)
|
||||
, path(path_)
|
||||
, relative_data_path(relative_data_path_)
|
||||
, format_name(format_name_)
|
||||
, log(&Poco::Logger::get("StorageFileLog (" + table_id_.table_name + ")"))
|
||||
, milliseconds_to_wait(RESCHEDULE_MS)
|
||||
{
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(columns_);
|
||||
storage_metadata.setComment(comment);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
|
||||
try
|
||||
@ -75,9 +81,11 @@ StorageFileLog::StorageFileLog(
|
||||
loadMetaFiles(attach);
|
||||
loadFiles();
|
||||
|
||||
#ifndef NDEBUG
|
||||
assert(file_infos.file_names.size() == file_infos.meta_by_inode.size());
|
||||
assert(file_infos.file_names.size() == file_infos.inode_by_name.size());
|
||||
assert(file_infos.file_names.size() == file_infos.context_by_name.size());
|
||||
#endif
|
||||
|
||||
if (path_is_directory)
|
||||
directory_watch = std::make_unique<FileLogDirectoryWatcher>(path);
|
||||
@ -93,17 +101,20 @@ StorageFileLog::StorageFileLog(
|
||||
|
||||
void StorageFileLog::loadMetaFiles(bool attach)
|
||||
{
|
||||
const auto database = DatabaseCatalog::instance().getDatabase(getStorageID().getDatabaseName());
|
||||
const auto table_name = getStorageID().getTableName();
|
||||
|
||||
root_meta_path = database->getMetadataPath() + "/." + table_name;
|
||||
/// We just use default storage policy
|
||||
auto storage_policy = getContext()->getStoragePolicy("default");
|
||||
auto data_volume = storage_policy->getVolume(0);
|
||||
root_meta_path = std::filesystem::path(data_volume->getDisk()->getPath()) / getStorageID().getTableName();
|
||||
|
||||
/// Create table, just create meta data directory
|
||||
if (!attach)
|
||||
{
|
||||
if (std::filesystem::exists(root_meta_path))
|
||||
{
|
||||
std::filesystem::remove_all(root_meta_path);
|
||||
throw Exception(
|
||||
ErrorCodes::TABLE_METADATA_ALREADY_EXISTS,
|
||||
"Metadata files already exist by path: {}, remove them manually if it is intended",
|
||||
root_meta_path);
|
||||
}
|
||||
std::filesystem::create_directories(root_meta_path);
|
||||
}
|
||||
@ -113,7 +124,7 @@ void StorageFileLog::loadMetaFiles(bool attach)
|
||||
/// Meta file may lost, log and create directory
|
||||
if (!std::filesystem::exists(root_meta_path))
|
||||
{
|
||||
LOG_INFO(log, "Meta files of table {} may have lost.", getStorageID().getTableName());
|
||||
LOG_ERROR(log, "Metadata files of table {} are lost.", getStorageID().getTableName());
|
||||
std::filesystem::create_directories(root_meta_path);
|
||||
}
|
||||
/// Load all meta info to file_infos;
|
||||
@ -123,19 +134,27 @@ void StorageFileLog::loadMetaFiles(bool attach)
|
||||
|
||||
void StorageFileLog::loadFiles()
|
||||
{
|
||||
if (!symlinkStartsWith(path, getContext()->getUserFilesPath()))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS, "The absolute data path should start with user_files_path {}", getContext()->getUserFilesPath());
|
||||
}
|
||||
|
||||
if (std::filesystem::is_regular_file(path))
|
||||
auto absolute_path = std::filesystem::absolute(path);
|
||||
absolute_path = absolute_path.lexically_normal(); /// Normalize path.
|
||||
|
||||
if (std::filesystem::is_regular_file(absolute_path))
|
||||
{
|
||||
path_is_directory = false;
|
||||
root_data_path = getContext()->getUserFilesPath();
|
||||
root_data_path = absolute_path.parent_path();
|
||||
|
||||
file_infos.file_names.push_back(std::filesystem::path(path).filename());
|
||||
file_infos.file_names.push_back(absolute_path.filename());
|
||||
}
|
||||
else if (std::filesystem::is_directory(path))
|
||||
else if (std::filesystem::is_directory(absolute_path))
|
||||
{
|
||||
root_data_path = path;
|
||||
root_data_path = absolute_path;
|
||||
/// Just consider file with depth 1
|
||||
for (const auto & dir_entry : std::filesystem::directory_iterator{path})
|
||||
for (const auto & dir_entry : std::filesystem::directory_iterator{absolute_path})
|
||||
{
|
||||
if (dir_entry.is_regular_file())
|
||||
{
|
||||
@ -145,76 +164,86 @@ void StorageFileLog::loadFiles()
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The path {} neither a regular file, nor a directory", path);
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The path {} neither a regular file, nor a directory", absolute_path.c_str());
|
||||
}
|
||||
|
||||
/// Get files inode
|
||||
for (const auto & file : file_infos.file_names)
|
||||
{
|
||||
auto inode = getInode(getFullDataPath(file));
|
||||
file_infos.inode_by_name.emplace(file, inode);
|
||||
file_infos.context_by_name.emplace(file, FileContext{});
|
||||
file_infos.context_by_name.emplace(file, FileContext{.inode = inode});
|
||||
}
|
||||
|
||||
/// Update file meta or create file meta
|
||||
for (const auto & file_inode : file_infos.inode_by_name)
|
||||
for (const auto & [file, ctx] : file_infos.context_by_name)
|
||||
{
|
||||
if (auto it = file_infos.meta_by_inode.find(file_inode.second); it != file_infos.meta_by_inode.end())
|
||||
if (auto it = file_infos.meta_by_inode.find(ctx.inode); it != file_infos.meta_by_inode.end())
|
||||
{
|
||||
/// data file have been renamed, need update meta file's name
|
||||
if (it->second.file_name != file_inode.first)
|
||||
if (it->second.file_name != file)
|
||||
{
|
||||
it->second.file_name = file_inode.first;
|
||||
it->second.file_name = file;
|
||||
if (std::filesystem::exists(getFullMetaPath(it->second.file_name)))
|
||||
{
|
||||
std::filesystem::rename(getFullMetaPath(it->second.file_name), getFullMetaPath(file_inode.first));
|
||||
std::filesystem::rename(getFullMetaPath(it->second.file_name), getFullMetaPath(file));
|
||||
}
|
||||
}
|
||||
}
|
||||
/// New file
|
||||
else
|
||||
{
|
||||
FileMeta meta{file_inode.first, 0, 0};
|
||||
file_infos.meta_by_inode.emplace(file_inode.second, meta);
|
||||
FileMeta meta{file, 0, 0};
|
||||
file_infos.meta_by_inode.emplace(ctx.inode, meta);
|
||||
}
|
||||
}
|
||||
|
||||
/// Clear unneeded meta file, because data files may be deleted
|
||||
if (file_infos.meta_by_inode.size() > file_infos.inode_by_name.size())
|
||||
if (file_infos.meta_by_inode.size() > file_infos.context_by_name.size())
|
||||
{
|
||||
InodeToFileMeta valid_metas;
|
||||
valid_metas.reserve(file_infos.inode_by_name.size());
|
||||
valid_metas.reserve(file_infos.context_by_name.size());
|
||||
for (const auto & it : file_infos.meta_by_inode)
|
||||
{
|
||||
if (file_infos.inode_by_name.contains(it.second.file_name))
|
||||
auto file_name = it.second.file_name;
|
||||
if (file_infos.context_by_name.contains(file_name))
|
||||
valid_metas.emplace(it);
|
||||
/// Delete meta file from filesystem
|
||||
else
|
||||
std::filesystem::remove(getFullMetaPath(file_name));
|
||||
}
|
||||
file_infos.meta_by_inode.swap(valid_metas);
|
||||
}
|
||||
}
|
||||
|
||||
void StorageFileLog::serialize(bool with_end_pos) const
|
||||
void StorageFileLog::serialize() const
|
||||
{
|
||||
for (const auto & it : file_infos.meta_by_inode)
|
||||
{
|
||||
auto full_name = getFullMetaPath(it.second.file_name);
|
||||
if (!std::filesystem::exists(full_name))
|
||||
{
|
||||
Poco::File{full_name}.createFile();
|
||||
}
|
||||
WriteBufferFromFile buf(full_name);
|
||||
writeIntText(it.first, buf);
|
||||
writeChar('\n', buf);
|
||||
writeIntText(it.second.last_writen_position, buf);
|
||||
|
||||
if (with_end_pos)
|
||||
{
|
||||
writeChar('\n', buf);
|
||||
writeIntText(it.second.last_open_end, buf);
|
||||
FS::createFile(full_name);
|
||||
}
|
||||
WriteBufferFromFile out(full_name);
|
||||
writeIntText(it.first, out);
|
||||
writeChar('\n', out);
|
||||
writeIntText(it.second.last_writen_position, out);
|
||||
}
|
||||
}
|
||||
|
||||
void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const
|
||||
{
|
||||
auto full_name = getFullMetaPath(file_meta.file_name);
|
||||
if (!std::filesystem::exists(full_name))
|
||||
{
|
||||
FS::createFile(full_name);
|
||||
}
|
||||
WriteBufferFromFile out(full_name);
|
||||
writeIntText(inode, out);
|
||||
writeChar('\n', out);
|
||||
writeIntText(file_meta.last_writen_position, out);
|
||||
}
|
||||
|
||||
void StorageFileLog::deserialize()
|
||||
{
|
||||
for (const auto & dir_entry : std::filesystem::directory_iterator{root_meta_path})
|
||||
@ -228,19 +257,16 @@ void StorageFileLog::deserialize()
|
||||
root_meta_path);
|
||||
}
|
||||
|
||||
ReadBufferFromFile buf(dir_entry.path().c_str());
|
||||
ReadBufferFromFile in(dir_entry.path().c_str());
|
||||
FileMeta meta;
|
||||
UInt64 inode, last_written_pos;
|
||||
|
||||
if (!tryReadIntText(inode, buf))
|
||||
if (!tryReadIntText(inode, in))
|
||||
{
|
||||
throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str());
|
||||
}
|
||||
if (!checkChar('\n', buf))
|
||||
{
|
||||
throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str());
|
||||
}
|
||||
if (!tryReadIntText(last_written_pos, buf))
|
||||
assertChar('\n', in);
|
||||
if (!tryReadIntText(last_written_pos, in))
|
||||
{
|
||||
throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str());
|
||||
}
|
||||
@ -248,15 +274,6 @@ void StorageFileLog::deserialize()
|
||||
meta.file_name = dir_entry.path().filename();
|
||||
meta.last_writen_position = last_written_pos;
|
||||
|
||||
/// May have last open end in meta file
|
||||
if (checkChar('\n', buf))
|
||||
{
|
||||
if (!tryReadIntText(meta.last_open_end, buf))
|
||||
{
|
||||
throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str());
|
||||
}
|
||||
}
|
||||
|
||||
file_infos.meta_by_inode.emplace(inode, meta);
|
||||
}
|
||||
}
|
||||
@ -280,6 +297,7 @@ Pipe StorageFileLog::read(
|
||||
size_t /* max_block_size */,
|
||||
unsigned /* num_streams */)
|
||||
{
|
||||
/// We need this lock, in case read and streamToViews execute at the same time
|
||||
std::lock_guard<std::mutex> lock(status_mutex);
|
||||
|
||||
updateFileInfos();
|
||||
@ -287,30 +305,30 @@ Pipe StorageFileLog::read(
|
||||
/// No files to parse
|
||||
if (file_infos.file_names.empty())
|
||||
{
|
||||
LOG_INFO(log, "There is a idle table named {}, no files need to parse.", getName());
|
||||
return Pipe{};
|
||||
}
|
||||
|
||||
auto modified_context = Context::createCopy(local_context);
|
||||
|
||||
auto max_streams_number = std::min<UInt64>(filelog_settings->filelog_max_threads, file_infos.file_names.size());
|
||||
auto max_streams_number = std::min<UInt64>(filelog_settings->max_threads, file_infos.file_names.size());
|
||||
|
||||
/// Each stream responsible for closing it's files and store meta
|
||||
openFilesAndSetPos();
|
||||
|
||||
Pipes pipes;
|
||||
pipes.reserve(max_streams_number);
|
||||
for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number)
|
||||
{
|
||||
Pipe pipe(std::make_shared<FileLogSource>(
|
||||
*this, metadata_snapshot, modified_context, getMaxBlockSize(), getPollTimeoutMillisecond(), stream_number, max_streams_number));
|
||||
|
||||
auto convert_actions_dag = ActionsDAG::makeConvertingActions(
|
||||
pipe.getHeader().getColumnsWithTypeAndName(),
|
||||
metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()).getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
|
||||
auto actions = std::make_shared<ExpressionActions>(
|
||||
convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes));
|
||||
|
||||
pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared<ExpressionTransform>(stream_header, actions); });
|
||||
pipes.emplace_back(std::move(pipe));
|
||||
pipes.emplace_back(std::make_shared<FileLogSource>(
|
||||
*this,
|
||||
metadata_snapshot,
|
||||
modified_context,
|
||||
column_names,
|
||||
getMaxBlockSize(),
|
||||
getPollTimeoutMillisecond(),
|
||||
stream_number,
|
||||
max_streams_number));
|
||||
}
|
||||
|
||||
return Pipe::unitePipes(std::move(pipes));
|
||||
@ -318,15 +336,29 @@ Pipe StorageFileLog::read(
|
||||
|
||||
void StorageFileLog::drop()
|
||||
{
|
||||
if (std::filesystem::exists(root_meta_path))
|
||||
std::filesystem::remove_all(root_meta_path);
|
||||
try
|
||||
{
|
||||
if (std::filesystem::exists(root_meta_path))
|
||||
std::filesystem::remove_all(root_meta_path);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
void StorageFileLog::startup()
|
||||
{
|
||||
if (task)
|
||||
try
|
||||
{
|
||||
task->holder->activateAndSchedule();
|
||||
if (task)
|
||||
{
|
||||
task->holder->activateAndSchedule();
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
@ -342,32 +374,31 @@ void StorageFileLog::shutdown()
|
||||
closeFilesAndStoreMeta();
|
||||
}
|
||||
|
||||
void StorageFileLog::assertStreamGood(const std::ifstream & reader)
|
||||
{
|
||||
if (!reader.good())
|
||||
{
|
||||
throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Stream is in bad state.");
|
||||
}
|
||||
}
|
||||
|
||||
void StorageFileLog::openFilesAndSetPos()
|
||||
{
|
||||
for (const auto & file : file_infos.file_names)
|
||||
{
|
||||
auto & file_ctx = file_infos.context_by_name.at(file);
|
||||
auto & file_ctx = findInMap(file_infos.context_by_name, file);
|
||||
if (file_ctx.status != FileStatus::NO_CHANGE)
|
||||
{
|
||||
file_ctx.reader.open(getFullDataPath(file));
|
||||
if (!file_ctx.reader.good())
|
||||
{
|
||||
throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Open file {} failed.", file);
|
||||
}
|
||||
assertStreamGood(file_ctx.reader);
|
||||
|
||||
file_ctx.reader.seekg(0, file_ctx.reader.end);
|
||||
if (!file_ctx.reader.good())
|
||||
{
|
||||
throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Seekg file {} failed.", file);
|
||||
}
|
||||
assertStreamGood(file_ctx.reader);
|
||||
|
||||
auto file_end = file_ctx.reader.tellg();
|
||||
if (!file_ctx.reader.good())
|
||||
{
|
||||
throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Tellg file {} failed.", file);
|
||||
}
|
||||
assertStreamGood(file_ctx.reader);
|
||||
|
||||
auto & meta = file_infos.meta_by_inode.at(file_infos.inode_by_name.at(file));
|
||||
auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode);
|
||||
if (meta.last_writen_position > static_cast<UInt64>(file_end))
|
||||
{
|
||||
throw Exception(ErrorCodes::FILE_STREAM_ERROR, "File {} has been broken.", file);
|
||||
@ -376,13 +407,10 @@ void StorageFileLog::openFilesAndSetPos()
|
||||
meta.last_open_end = file_end;
|
||||
|
||||
file_ctx.reader.seekg(meta.last_writen_position);
|
||||
if (!file_ctx.reader.good())
|
||||
{
|
||||
throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Seekg file {} failed.", file);
|
||||
}
|
||||
assertStreamGood(file_ctx.reader);
|
||||
}
|
||||
}
|
||||
serialize(true);
|
||||
serialize();
|
||||
}
|
||||
|
||||
void StorageFileLog::closeFilesAndStoreMeta()
|
||||
@ -395,23 +423,33 @@ void StorageFileLog::closeFilesAndStoreMeta()
|
||||
serialize();
|
||||
}
|
||||
|
||||
void StorageFileLog::closeFileAndStoreMeta(const String & file_name)
|
||||
{
|
||||
auto & file_ctx = findInMap(file_infos.context_by_name, file_name);
|
||||
if (file_ctx.reader.is_open())
|
||||
file_ctx.reader.close();
|
||||
|
||||
auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode);
|
||||
serialize(file_ctx.inode, meta);
|
||||
}
|
||||
|
||||
size_t StorageFileLog::getMaxBlockSize() const
|
||||
{
|
||||
return filelog_settings->filelog_max_block_size.changed ? filelog_settings->filelog_max_block_size.value
|
||||
: getContext()->getSettingsRef().max_insert_block_size.value;
|
||||
return filelog_settings->max_block_size.changed ? filelog_settings->max_block_size.value
|
||||
: getContext()->getSettingsRef().max_insert_block_size.value;
|
||||
}
|
||||
|
||||
size_t StorageFileLog::getPollMaxBatchSize() const
|
||||
{
|
||||
size_t batch_size = filelog_settings->filelog_poll_max_batch_size.changed ? filelog_settings->filelog_poll_max_batch_size.value
|
||||
: getContext()->getSettingsRef().max_block_size.value;
|
||||
size_t batch_size = filelog_settings->poll_max_batch_size.changed ? filelog_settings->poll_max_batch_size.value
|
||||
: getContext()->getSettingsRef().max_block_size.value;
|
||||
return std::min(batch_size, getMaxBlockSize());
|
||||
}
|
||||
|
||||
size_t StorageFileLog::getPollTimeoutMillisecond() const
|
||||
{
|
||||
return filelog_settings->filelog_poll_timeout_ms.changed ? filelog_settings->filelog_poll_timeout_ms.totalMilliseconds()
|
||||
: getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds();
|
||||
return filelog_settings->poll_timeout_ms.changed ? filelog_settings->poll_timeout_ms.totalMilliseconds()
|
||||
: getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds();
|
||||
}
|
||||
|
||||
bool StorageFileLog::checkDependencies(const StorageID & table_id)
|
||||
@ -503,10 +541,11 @@ bool StorageFileLog::streamToViews()
|
||||
throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR);
|
||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||
|
||||
auto max_streams_number = std::min<UInt64>(filelog_settings->filelog_max_threads.value, file_infos.file_names.size());
|
||||
auto max_streams_number = std::min<UInt64>(filelog_settings->max_threads.value, file_infos.file_names.size());
|
||||
/// No files to parse
|
||||
if (max_streams_number == 0)
|
||||
{
|
||||
LOG_INFO(log, "There is a idle table named {}, no files need to parse.", getName());
|
||||
return false;
|
||||
}
|
||||
|
||||
@ -519,36 +558,38 @@ bool StorageFileLog::streamToViews()
|
||||
InterpreterInsertQuery interpreter(insert, new_context, false, true, true);
|
||||
auto block_io = interpreter.execute();
|
||||
|
||||
/// Each stream responsible for closing it's files and store meta
|
||||
openFilesAndSetPos();
|
||||
|
||||
Pipes pipes;
|
||||
pipes.reserve(max_streams_number);
|
||||
for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number)
|
||||
{
|
||||
Pipe pipe(std::make_shared<FileLogSource>(
|
||||
*this, metadata_snapshot, new_context, getPollMaxBatchSize(), getPollTimeoutMillisecond(), stream_number, max_streams_number));
|
||||
|
||||
auto convert_actions_dag = ActionsDAG::makeConvertingActions(
|
||||
pipe.getHeader().getColumnsWithTypeAndName(),
|
||||
block_io.pipeline.getHeader().getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
|
||||
auto actions = std::make_shared<ExpressionActions>(
|
||||
convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes));
|
||||
|
||||
pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared<ExpressionTransform>(stream_header, actions); });
|
||||
pipes.emplace_back(std::move(pipe));
|
||||
pipes.emplace_back(std::make_shared<FileLogSource>(
|
||||
*this,
|
||||
metadata_snapshot,
|
||||
new_context,
|
||||
block_io.pipeline.getHeader().getNames(),
|
||||
getPollMaxBatchSize(),
|
||||
getPollTimeoutMillisecond(),
|
||||
stream_number,
|
||||
max_streams_number));
|
||||
}
|
||||
|
||||
auto input= Pipe::unitePipes(std::move(pipes));
|
||||
|
||||
assertBlocksHaveEqualStructure(input.getHeader(), block_io.pipeline.getHeader(), "StorageFileLog streamToViews");
|
||||
|
||||
block_io.pipeline.complete(std::move(input));
|
||||
|
||||
CompletedPipelineExecutor executor(block_io.pipeline);
|
||||
executor.execute();
|
||||
size_t rows = 0;
|
||||
{
|
||||
block_io.pipeline.complete(std::move(input));
|
||||
block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); });
|
||||
CompletedPipelineExecutor executor(block_io.pipeline);
|
||||
executor.execute();
|
||||
}
|
||||
|
||||
UInt64 milliseconds = watch.elapsedMilliseconds();
|
||||
LOG_DEBUG(log, "Pushing data to {} took {} ms.", table_id.getNameForLogs(), milliseconds);
|
||||
LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", rows, table_id.getNameForLogs(), milliseconds);
|
||||
|
||||
return updateFileInfos();
|
||||
}
|
||||
@ -569,7 +610,7 @@ void registerStorageFileLog(StorageFactory & factory)
|
||||
}
|
||||
|
||||
auto physical_cpu_cores = getNumberOfPhysicalCPUCores();
|
||||
auto num_threads = filelog_settings->filelog_max_threads.value;
|
||||
auto num_threads = filelog_settings->max_threads.value;
|
||||
|
||||
if (num_threads > physical_cpu_cores)
|
||||
{
|
||||
@ -580,12 +621,12 @@ void registerStorageFileLog(StorageFactory & factory)
|
||||
throw Exception("Number of threads to parse files can not be lower than 1", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
if (filelog_settings->filelog_max_block_size.changed && filelog_settings->filelog_max_block_size.value < 1)
|
||||
if (filelog_settings->max_block_size.changed && filelog_settings->max_block_size.value < 1)
|
||||
{
|
||||
throw Exception("filelog_max_block_size can not be lower than 1", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
if (filelog_settings->filelog_poll_max_batch_size.changed && filelog_settings->filelog_poll_max_batch_size.value < 1)
|
||||
if (filelog_settings->poll_max_batch_size.changed && filelog_settings->poll_max_batch_size.value < 1)
|
||||
{
|
||||
throw Exception("filelog_poll_max_batch_size can not be lower than 1", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
@ -601,7 +642,15 @@ void registerStorageFileLog(StorageFactory & factory)
|
||||
auto format = format_ast->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
return StorageFileLog::create(
|
||||
args.table_id, args.getContext(), args.columns, path, format, std::move(filelog_settings), args.attach);
|
||||
args.table_id,
|
||||
args.getContext(),
|
||||
args.columns,
|
||||
path,
|
||||
args.relative_data_path,
|
||||
format,
|
||||
std::move(filelog_settings),
|
||||
args.comment,
|
||||
args.attach);
|
||||
};
|
||||
|
||||
factory.registerStorage(
|
||||
@ -619,9 +668,13 @@ bool StorageFileLog::updateFileInfos()
|
||||
/// For table just watch one file, we can not use directory monitor to watch it
|
||||
if (!path_is_directory)
|
||||
{
|
||||
assert(
|
||||
file_infos.file_names.size() == file_infos.meta_by_inode.size() == file_infos.inode_by_name.size()
|
||||
== file_infos.context_by_name.size() == 1);
|
||||
#ifndef NDEBUG
|
||||
assert(file_infos.file_names.size() == file_infos.meta_by_inode.size());
|
||||
assert(file_infos.file_names.size() == file_infos.inode_by_name.size());
|
||||
assert(file_infos.file_names.size() == file_infos.context_by_name.size());
|
||||
assert(file_infos.file_names.size() == 1);
|
||||
#endif
|
||||
|
||||
if (auto it = file_infos.context_by_name.find(file_infos.file_names[0]); it != file_infos.context_by_name.end())
|
||||
{
|
||||
it->second.status = FileStatus::UPDATED;
|
||||
@ -642,7 +695,8 @@ bool StorageFileLog::updateFileInfos()
|
||||
{
|
||||
switch (event.type)
|
||||
{
|
||||
case Poco::DirectoryWatcher::DW_ITEM_ADDED: {
|
||||
case Poco::DirectoryWatcher::DW_ITEM_ADDED:
|
||||
{
|
||||
LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path);
|
||||
if (std::filesystem::is_regular_file(event.path))
|
||||
{
|
||||
@ -650,45 +704,40 @@ bool StorageFileLog::updateFileInfos()
|
||||
auto inode = getInode(event.path);
|
||||
|
||||
file_infos.file_names.push_back(file_name);
|
||||
file_infos.inode_by_name.emplace(file_name, inode);
|
||||
|
||||
FileMeta meta{file_name, 0, 0};
|
||||
file_infos.meta_by_inode.emplace(inode, meta);
|
||||
file_infos.context_by_name.emplace(file_name, FileContext{});
|
||||
file_infos.meta_by_inode.emplace(inode, FileMeta{.file_name = file_name});
|
||||
file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode});
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
||||
case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: {
|
||||
auto file_name = std::filesystem::path(event.path).filename();
|
||||
case Poco::DirectoryWatcher::DW_ITEM_MODIFIED:
|
||||
{
|
||||
String file_name = std::filesystem::path(event.path).filename();
|
||||
LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path);
|
||||
if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end())
|
||||
{
|
||||
it->second.status = FileStatus::UPDATED;
|
||||
}
|
||||
auto & file_ctx = findInMap(file_infos.context_by_name, file_name);
|
||||
file_ctx.status = FileStatus::UPDATED;
|
||||
break;
|
||||
}
|
||||
|
||||
case Poco::DirectoryWatcher::DW_ITEM_REMOVED:
|
||||
case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: {
|
||||
auto file_name = std::filesystem::path(event.path).filename();
|
||||
case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM:
|
||||
{
|
||||
String file_name = std::filesystem::path(event.path).filename();
|
||||
LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path);
|
||||
if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end())
|
||||
{
|
||||
it->second.status = FileStatus::REMOVED;
|
||||
}
|
||||
auto & file_ctx = findInMap(file_infos.context_by_name, file_name);
|
||||
file_ctx.status = FileStatus::REMOVED;
|
||||
break;
|
||||
}
|
||||
/// file rename
|
||||
case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: {
|
||||
case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO:
|
||||
{
|
||||
auto file_name = std::filesystem::path(event.path).filename();
|
||||
LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path);
|
||||
|
||||
file_infos.file_names.push_back(file_name);
|
||||
file_infos.context_by_name.emplace(file_name, FileContext{});
|
||||
|
||||
auto inode = getInode(event.path);
|
||||
file_infos.inode_by_name.emplace(file_name, inode);
|
||||
file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode});
|
||||
|
||||
if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end())
|
||||
{
|
||||
@ -705,19 +754,18 @@ bool StorageFileLog::updateFileInfos()
|
||||
}
|
||||
std::vector<String> valid_files;
|
||||
|
||||
/// Remove file infos with REMOVE status
|
||||
for (const auto & file_name : file_infos.file_names)
|
||||
{
|
||||
if (auto it = file_infos.context_by_name.find(file_name);
|
||||
it != file_infos.context_by_name.end() && it->second.status == FileStatus::REMOVED)
|
||||
{
|
||||
/// Erase meta_by_inode first, otherwise it become invalid
|
||||
file_infos.meta_by_inode.erase(it->second.inode);
|
||||
file_infos.context_by_name.erase(it);
|
||||
if (auto inode = file_infos.inode_by_name.find(file_name); inode != file_infos.inode_by_name.end())
|
||||
{
|
||||
file_infos.inode_by_name.erase(inode);
|
||||
file_infos.meta_by_inode.erase(inode->second);
|
||||
if (std::filesystem::exists(getFullMetaPath(file_name)))
|
||||
std::filesystem::remove(getFullMetaPath(file_name));
|
||||
}
|
||||
|
||||
if (std::filesystem::exists(getFullMetaPath(file_name)))
|
||||
std::filesystem::remove(getFullMetaPath(file_name));
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -727,9 +775,11 @@ bool StorageFileLog::updateFileInfos()
|
||||
file_infos.file_names.swap(valid_files);
|
||||
|
||||
/// These file infos should always have same size(one for one)
|
||||
#ifndef NDEBUG
|
||||
assert(file_infos.file_names.size() == file_infos.meta_by_inode.size());
|
||||
assert(file_infos.file_names.size() == file_infos.inode_by_name.size());
|
||||
assert(file_infos.file_names.size() == file_infos.context_by_name.size());
|
||||
#endif
|
||||
|
||||
return events.empty() || file_infos.file_names.empty();
|
||||
}
|
||||
|
@ -18,6 +18,11 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
class StorageFileLog final : public shared_ptr_helper<StorageFileLog>, public IStorage, WithContext
|
||||
{
|
||||
friend struct shared_ptr_helper<StorageFileLog>;
|
||||
@ -59,6 +64,7 @@ public:
|
||||
struct FileContext
|
||||
{
|
||||
FileStatus status = FileStatus::OPEN;
|
||||
UInt64 inode;
|
||||
std::ifstream reader;
|
||||
};
|
||||
|
||||
@ -69,13 +75,11 @@ public:
|
||||
UInt64 last_open_end = 0;
|
||||
};
|
||||
|
||||
using FileNameToInode = std::unordered_map<String, UInt64>;
|
||||
using InodeToFileMeta = std::unordered_map<UInt64, FileMeta>;
|
||||
using FileNameToContext = std::unordered_map<String, FileContext>;
|
||||
|
||||
struct FileInfos
|
||||
{
|
||||
FileNameToInode inode_by_name;
|
||||
InodeToFileMeta meta_by_inode;
|
||||
FileNameToContext context_by_name;
|
||||
/// file names without path
|
||||
@ -84,8 +88,8 @@ public:
|
||||
|
||||
auto & getFileInfos() { return file_infos; }
|
||||
|
||||
auto getFullMetaPath(const String & file_name) const { return root_meta_path + "/" + file_name; }
|
||||
auto getFullDataPath(const String & file_name) const { return root_data_path + "/" + file_name; }
|
||||
String getFullMetaPath(const String & file_name) const { return std::filesystem::path(root_meta_path) / file_name; }
|
||||
String getFullDataPath(const String & file_name) const { return std::filesystem::path(root_data_path) / file_name; }
|
||||
|
||||
NamesAndTypesList getVirtuals() const override;
|
||||
|
||||
@ -94,23 +98,40 @@ public:
|
||||
static UInt64 getInode(const String & file_name);
|
||||
|
||||
void openFilesAndSetPos();
|
||||
/// Used in shutdown()
|
||||
void closeFilesAndStoreMeta();
|
||||
/// Used in FileSource
|
||||
void closeFileAndStoreMeta(const String & file_name);
|
||||
|
||||
static void assertStreamGood(const std::ifstream & reader);
|
||||
|
||||
template <typename K, typename V>
|
||||
static V & findInMap(std::unordered_map<K, V> & map, const K & key)
|
||||
{
|
||||
if (auto it = map.find(key); it != map.end())
|
||||
return it->second;
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "The key {} doesn't exist.", key);
|
||||
}
|
||||
|
||||
protected:
|
||||
StorageFileLog(
|
||||
const StorageID & table_id_,
|
||||
ContextPtr context_,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & relative_path_,
|
||||
const String & path_,
|
||||
const String & relative_data_path_,
|
||||
const String & format_name_,
|
||||
std::unique_ptr<FileLogSettings> settings,
|
||||
const String & comment,
|
||||
bool attach);
|
||||
|
||||
private:
|
||||
std::unique_ptr<FileLogSettings> filelog_settings;
|
||||
|
||||
/// user_files_path/ + path_argument/
|
||||
const String path;
|
||||
/// For meta file
|
||||
const String relative_data_path;
|
||||
bool path_is_directory = true;
|
||||
|
||||
/// If path argument of the table is a regular file, it equals to user_files_path
|
||||
@ -157,8 +178,10 @@ private:
|
||||
|
||||
bool updateFileInfos();
|
||||
|
||||
/// Serialize all file meta
|
||||
void serialize(bool with_end_pos = false) const;
|
||||
/// Used in shutdown()
|
||||
void serialize() const;
|
||||
/// Used in FileSource closeFileAndStoreMeta(file_name);
|
||||
void serialize(UInt64 inode, const FileMeta & file_meta) const;
|
||||
|
||||
void deserialize();
|
||||
};
|
||||
|
Loading…
Reference in New Issue
Block a user