mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
dbms.StorageLog: added support for checkData query [#METR-11709]
This commit is contained in:
parent
cf1afb1bd1
commit
b34c8e47ab
@ -1,9 +1,15 @@
|
||||
#pragma once
|
||||
|
||||
#include <Yandex/logger_useful.h>
|
||||
#include <DB/Columns/IColumn.h>
|
||||
#include <Poco/AutoPtr.h>
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
#include <string>
|
||||
#include <Poco/File.h>
|
||||
#include <DB/Common/escapeForFileName.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// хранит размеры всех столбцов, и может проверять не побились ли столбцы
|
||||
template <class Storage>
|
||||
@ -28,27 +34,26 @@ public:
|
||||
files_info_path = file_info_path_;
|
||||
}
|
||||
|
||||
template <class Iterator>
|
||||
void update(const Iterator & begin, const Iterator & end)
|
||||
using Files = std::vector<Poco::File>;
|
||||
|
||||
void update(const Files::iterator & begin, const Files::iterator & end)
|
||||
{
|
||||
for (auto it = begin; it != end; ++it)
|
||||
{
|
||||
auto & column_name = *it;
|
||||
auto & file = storage.getFiles()[column_name].data_file;
|
||||
files_info->setString(column_name + ".size", std::to_string(file.getSize()));
|
||||
}
|
||||
files_info->setString(escapeForFileName(Poco::Path(it->path()).getFileName()) + ".size", std::to_string(it->getSize()));
|
||||
|
||||
files_info->save(files_info_path);
|
||||
}
|
||||
|
||||
bool check() const
|
||||
bool check(const Files::iterator & begin, const Files::iterator & end) const
|
||||
{
|
||||
bool sizes_are_correct = true;
|
||||
for (auto & pair : storage.getFiles())
|
||||
for (auto it = begin; it != end; ++it)
|
||||
{
|
||||
if (files_info->has(pair.first))
|
||||
auto & file = *it;
|
||||
std::string filename = escapeForFileName(Poco::Path(it->path()).getFileName());
|
||||
if (files_info->has(filename))
|
||||
{
|
||||
auto & file = pair.second.data_file;
|
||||
size_t expected_size = std::stoull(files_info->getString(pair.first + ".size"));
|
||||
size_t expected_size = std::stoull(files_info->getString(filename + ".size"));
|
||||
size_t real_size = file.getSize();
|
||||
if (real_size != expected_size)
|
||||
{
|
||||
@ -69,3 +74,5 @@ private:
|
||||
Storage & storage;
|
||||
Logger * log;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||
#include <DB/Common/FileChecker.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -80,6 +81,8 @@ class LogBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
LogBlockOutputStream(StorageLog & storage_);
|
||||
~LogBlockOutputStream() { writeSuffix(); }
|
||||
|
||||
void write(const Block & block);
|
||||
void writeSuffix();
|
||||
private:
|
||||
@ -156,6 +159,22 @@ public:
|
||||
|
||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name);
|
||||
|
||||
/// Данные столбца
|
||||
struct ColumnData
|
||||
{
|
||||
/// Задает номер столбца в файле с засечками.
|
||||
/// Не обязательно совпадает с номером столбца среди столбцов таблицы: здесь нумеруются также столбцы с длинами массивов.
|
||||
size_t column_index;
|
||||
|
||||
Poco::File data_file;
|
||||
Marks marks;
|
||||
};
|
||||
typedef std::map<String, ColumnData> Files_t;
|
||||
|
||||
Files_t & getFiles() { return files; }
|
||||
|
||||
bool checkData() const override;
|
||||
|
||||
protected:
|
||||
String path;
|
||||
String name;
|
||||
@ -195,18 +214,8 @@ protected:
|
||||
unsigned threads = 1);
|
||||
|
||||
private:
|
||||
/// Данные столбца
|
||||
struct ColumnData
|
||||
{
|
||||
/// Задает номер столбца в файле с засечками.
|
||||
/// Не обязательно совпадает с номером столбца среди столбцов таблицы: здесь нумеруются также столбцы с длинами массивов.
|
||||
size_t column_index;
|
||||
|
||||
Poco::File data_file;
|
||||
Marks marks;
|
||||
};
|
||||
typedef std::map<String, ColumnData> Files_t;
|
||||
Files_t files; /// name -> data
|
||||
|
||||
Names column_names; /// column_index -> name
|
||||
|
||||
Poco::File marks_file;
|
||||
@ -218,6 +227,8 @@ private:
|
||||
|
||||
size_t max_compress_block_size;
|
||||
|
||||
FileChecker<StorageLog> file_checker;
|
||||
|
||||
/** Для обычных столбцов, в засечках указано количество строчек в блоке.
|
||||
* Для столбцов-массивов и вложенных структур, есть более одной группы засечек, соответствующих разным файлам:
|
||||
* - для внутренностей (файла name.bin) - указано суммарное количество элементов массивов в блоке,
|
||||
|
@ -281,6 +281,13 @@ void LogBlockOutputStream::writeSuffix()
|
||||
for (FileStreams::iterator it = streams.begin(); it != streams.end(); ++it)
|
||||
it->second->finalize();
|
||||
|
||||
std::vector<Poco::File> column_files;
|
||||
for (auto & pair : streams)
|
||||
column_files.push_back(storage.files[pair.first].data_file);
|
||||
column_files.push_back(storage.marks_file);
|
||||
|
||||
storage.file_checker.update(column_files.begin(), column_files.end());
|
||||
|
||||
streams.clear();
|
||||
}
|
||||
|
||||
@ -403,7 +410,8 @@ void LogBlockOutputStream::writeMarks(MarksForColumns marks)
|
||||
|
||||
|
||||
StorageLog::StorageLog(const std::string & path_, const std::string & name_, NamesAndTypesListPtr columns_, size_t max_compress_block_size_)
|
||||
: path(path_), name(name_), columns(columns_), loaded_marks(false), max_compress_block_size(max_compress_block_size_)
|
||||
: path(path_), name(name_), columns(columns_), loaded_marks(false), max_compress_block_size(max_compress_block_size_),
|
||||
file_checker(path + escapeForFileName(name) + '/' + "sizes.txt", *this)
|
||||
{
|
||||
if (columns->empty())
|
||||
throw Exception("Empty list of columns passed to StorageLog constructor", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
|
||||
@ -532,6 +540,7 @@ void StorageLog::rename(const String & new_path_to_db, const String & new_databa
|
||||
|
||||
path = new_path_to_db;
|
||||
name = new_table_name;
|
||||
file_checker.setPath(path + escapeForFileName(name) + '/' + "sizes.txt");
|
||||
|
||||
for (Files_t::iterator it = files.begin(); it != files.end(); ++it)
|
||||
{
|
||||
@ -667,5 +676,13 @@ BlockOutputStreamPtr StorageLog::write(
|
||||
return new LogBlockOutputStream(*this);
|
||||
}
|
||||
|
||||
bool StorageLog::checkData() const
|
||||
{
|
||||
std::vector<Poco::File> column_files;
|
||||
for (auto & pair : files)
|
||||
column_files.push_back(pair.second.data_file);
|
||||
column_files.push_back(marks_file);
|
||||
return file_checker.check(column_files.begin(), column_files.end());
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -266,12 +266,11 @@ void TinyLogBlockOutputStream::writeSuffix()
|
||||
for (FileStreams::iterator it = streams.begin(); it != streams.end(); ++it)
|
||||
it->second->finalize();
|
||||
|
||||
/// @TODO лишнее копирование. Можно б было использовать boost::transform_iterator, если б он работал с C++11 lambda
|
||||
std::vector<std::string> column_names;
|
||||
std::vector<Poco::File> column_files;
|
||||
for (auto & pair : streams)
|
||||
column_names.push_back(pair.first);
|
||||
column_files.push_back(storage.files[pair.first].data_file);
|
||||
|
||||
storage.file_checker.update(column_names.begin(), column_names.end());
|
||||
storage.file_checker.update(column_files.begin(), column_files.end());
|
||||
|
||||
streams.clear();
|
||||
}
|
||||
@ -407,7 +406,11 @@ void StorageTinyLog::drop()
|
||||
|
||||
bool StorageTinyLog::checkData() const
|
||||
{
|
||||
return file_checker.check();
|
||||
std::vector<Poco::File> column_files;
|
||||
for (auto & pair : files)
|
||||
column_files.push_back(pair.second.data_file);
|
||||
|
||||
return file_checker.check(column_files.begin(), column_files.end());
|
||||
}
|
||||
|
||||
StorageTinyLog::Files_t & StorageTinyLog::getFiles()
|
||||
|
@ -3,3 +3,9 @@ CREATE TABLE check_query_tiny_log (UInt32 N, String S) Engine = TinyLog;
|
||||
INSERT INTO check_query_tiny_log VALUES (1, "A"), (2, "B"), (3, "C")
|
||||
|
||||
DROP TABLE check_query_tiny_log;
|
||||
|
||||
CREATE TABLE check_query_log (UInt32 N, String S) Engine = Log;
|
||||
|
||||
INSERT INTO check_query_log VALUES (1, "A"), (2, "B"), (3, "C")
|
||||
|
||||
DROP TABLE check_query_log;
|
||||
|
Loading…
Reference in New Issue
Block a user