This commit is contained in:
feng lv 2021-09-30 16:02:17 +00:00
parent 1f90c8dc22
commit 1758ff2d32
7 changed files with 275 additions and 208 deletions

View File

@ -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) \

View File

@ -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());
}
}

View File

@ -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;

View File

@ -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;

View File

@ -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;
};
}

View File

@ -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();
}

View File

@ -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();
};