Corrected naming based on review feedback

This commit is contained in:
Alexander Burmak 2019-12-25 11:24:13 +03:00
parent 97f03ab064
commit 544bfd105b
6 changed files with 39 additions and 39 deletions

View File

@ -113,7 +113,7 @@ public:
explicit LogBlockOutputStream(StorageLog & storage_) explicit LogBlockOutputStream(StorageLog & storage_)
: storage(storage_), : storage(storage_),
lock(storage.rwlock), lock(storage.rwlock),
marks_stream(fullPath(storage.disk, storage.marks_file), 4096, O_APPEND | O_CREAT | O_WRONLY) marks_stream(fullPath(storage.disk, storage.marks_file_path), 4096, O_APPEND | O_CREAT | O_WRONLY)
{ {
} }
@ -248,7 +248,7 @@ void LogBlockInputStream::readData(const String & name, const IDataType & type,
if (!stream_for_prefix && mark_number) if (!stream_for_prefix && mark_number)
offset = file_it->second.marks[mark_number].offset; offset = file_it->second.marks[mark_number].offset;
auto & data_file_path = file_it->second.data_file; auto & data_file_path = file_it->second.data_file_path;
auto it = streams.try_emplace(stream_name, storage.disk, data_file_path, offset, max_read_buffer_size).first; auto it = streams.try_emplace(stream_name, storage.disk, data_file_path, offset, max_read_buffer_size).first;
return &it->second.compressed; return &it->second.compressed;
}; };
@ -311,8 +311,8 @@ void LogBlockOutputStream::writeSuffix()
Strings column_files; Strings column_files;
for (const auto & name_stream : streams) for (const auto & name_stream : streams)
column_files.push_back(storage.files[name_stream.first].data_file); column_files.push_back(storage.files[name_stream.first].data_file_path);
column_files.push_back(storage.marks_file); column_files.push_back(storage.marks_file_path);
storage.file_checker.update(column_files.begin(), column_files.end()); storage.file_checker.update(column_files.begin(), column_files.end());
@ -353,7 +353,7 @@ void LogBlockOutputStream::writeData(const String & name, const IDataType & type
streams.try_emplace( streams.try_emplace(
stream_name, stream_name,
storage.disk, storage.disk,
storage.files[stream_name].data_file, storage.files[stream_name].data_file_path,
columns.getCodecOrDefault(name), columns.getCodecOrDefault(name),
storage.max_compress_block_size); storage.max_compress_block_size);
}, settings.path); }, settings.path);
@ -433,7 +433,7 @@ StorageLog::StorageLog(
for (const auto & column : getColumns().getAllPhysical()) for (const auto & column : getColumns().getAllPhysical())
addFiles(column.name, *column.type); addFiles(column.name, *column.type);
marks_file = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME; marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME;
} }
@ -451,7 +451,7 @@ void StorageLog::addFiles(const String & column_name, const IDataType & type)
{ {
ColumnData & column_data = files[stream_name]; ColumnData & column_data = files[stream_name];
column_data.column_index = file_count; column_data.column_index = file_count;
column_data.data_file = table_path + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION; column_data.data_file_path = table_path + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION;
column_names_by_idx.push_back(stream_name); column_names_by_idx.push_back(stream_name);
++file_count; ++file_count;
@ -476,9 +476,9 @@ void StorageLog::loadMarks()
for (Files::iterator it = files.begin(); it != files.end(); ++it) for (Files::iterator it = files.begin(); it != files.end(); ++it)
files_by_index[it->second.column_index] = it; files_by_index[it->second.column_index] = it;
if (disk->exists(marks_file)) if (disk->exists(marks_file_path))
{ {
size_t file_size = disk->getFileSize(marks_file); size_t file_size = disk->getFileSize(marks_file_path);
if (file_size % (file_count * sizeof(Mark)) != 0) if (file_size % (file_count * sizeof(Mark)) != 0)
throw Exception("Size of marks file is inconsistent", ErrorCodes::SIZES_OF_MARKS_FILES_ARE_INCONSISTENT); throw Exception("Size of marks file is inconsistent", ErrorCodes::SIZES_OF_MARKS_FILES_ARE_INCONSISTENT);
@ -487,7 +487,7 @@ void StorageLog::loadMarks()
for (auto & file : files_by_index) for (auto & file : files_by_index)
file->second.marks.reserve(marks_count); file->second.marks.reserve(marks_count);
std::unique_ptr<ReadBuffer> marks_rb = disk->read(marks_file, 32768); std::unique_ptr<ReadBuffer> marks_rb = disk->read(marks_file_path, 32768);
while (!marks_rb->eof()) while (!marks_rb->eof())
{ {
for (size_t i = 0; i < files_by_index.size(); ++i) for (size_t i = 0; i < files_by_index.size(); ++i)
@ -518,9 +518,9 @@ void StorageLog::rename(const String & /*new_path_to_db*/, const String & new_da
file_checker.setPath(table_path + "sizes.json"); file_checker.setPath(table_path + "sizes.json");
for (auto & file : files) for (auto & file : files)
file.second.data_file = table_path + Poco::Path(file.second.data_file).getFileName(); file.second.data_file_path = table_path + Poco::Path(file.second.data_file_path).getFileName();
marks_file = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME; marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME;
} }
void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &)
@ -537,7 +537,7 @@ void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLo
addFiles(column.name, *column.type); addFiles(column.name, *column.type);
file_checker = FileChecker{disk, table_path + "sizes.json"}; file_checker = FileChecker{disk, table_path + "sizes.json"};
marks_file = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME; marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME;
} }

View File

@ -62,13 +62,6 @@ protected:
size_t max_compress_block_size_); size_t max_compress_block_size_);
private: private:
DiskPtr disk;
String database_name;
String table_name;
String table_path;
mutable std::shared_mutex rwlock;
/** Offsets to some row number in a file for column in table. /** Offsets to some row number in a file for column in table.
* They are needed so that you can read the data in several threads. * They are needed so that you can read the data in several threads.
*/ */
@ -86,16 +79,23 @@ private:
/// Does not necessarily match the column number among the columns of the table: columns with lengths of arrays are also numbered here. /// Does not necessarily match the column number among the columns of the table: columns with lengths of arrays are also numbered here.
size_t column_index; size_t column_index;
String data_file; String data_file_path;
Marks marks; Marks marks;
}; };
using Files = std::map<String, ColumnData>; using Files = std::map<String, ColumnData>; /// file name -> column data
Files files; /// name -> data DiskPtr disk;
String database_name;
String table_name;
String table_path;
mutable std::shared_mutex rwlock;
Files files;
Names column_names_by_idx; /// column_index -> name Names column_names_by_idx; /// column_index -> name
String marks_file; String marks_file_path;
/// The order of adding files should not change: it corresponds to the order of the columns in the marks file. /// The order of adding files should not change: it corresponds to the order of the columns in the marks file.
void addFiles(const String & column_name, const IDataType & type); void addFiles(const String & column_name, const IDataType & type);

View File

@ -117,10 +117,10 @@ private:
{ {
started = true; started = true;
String data_file = storage.table_path + "data.bin"; String data_file_path = storage.table_path + "data.bin";
size_t buffer_size = std::min(max_read_buffer_size, storage.disk->getFileSize(data_file)); size_t buffer_size = std::min(max_read_buffer_size, storage.disk->getFileSize(data_file_path));
data_in.emplace(fullPath(storage.disk, data_file), 0, 0, buffer_size); data_in.emplace(fullPath(storage.disk, data_file_path), 0, 0, buffer_size);
block_in.emplace(*data_in, 0, index_begin, index_end); block_in.emplace(*data_in, 0, index_begin, index_end);
} }
} }
@ -202,8 +202,8 @@ StorageStripeLog::StorageStripeLog(
const ConstraintsDescription & constraints_, const ConstraintsDescription & constraints_,
bool attach, bool attach,
size_t max_compress_block_size_) size_t max_compress_block_size_)
: disk(disk_), database_name(database_name_), table_name(table_name_), : disk(std::move(disk_)), database_name(database_name_), table_name(table_name_),
table_path("data/" + escapeForFileName(database_name_) + '/' + escapeForFileName(table_name_) + '/'), table_path("data/" + escapeForFileName(database_name_) + '/' + escapeForFileName(table_name_) + '/'),
max_compress_block_size(max_compress_block_size_), max_compress_block_size(max_compress_block_size_),
file_checker(disk, table_path + "sizes.json"), file_checker(disk, table_path + "sizes.json"),
log(&Logger::get("StorageStripeLog")) log(&Logger::get("StorageStripeLog"))

View File

@ -62,9 +62,9 @@ protected:
private: private:
struct ColumnData struct ColumnData
{ {
String data_file; String data_file_path;
}; };
using Files = std::map<String, ColumnData>; using Files = std::map<String, ColumnData>; /// file name -> column data
DiskPtr disk; DiskPtr disk;
String database_name; String database_name;

View File

@ -217,7 +217,7 @@ void TinyLogBlockInputStream::readData(const String & name, const IDataType & ty
String stream_name = IDataType::getFileNameForStream(name, path); String stream_name = IDataType::getFileNameForStream(name, path);
if (!streams.count(stream_name)) if (!streams.count(stream_name))
streams[stream_name] = std::make_unique<Stream>(storage.disk, storage.files[stream_name].data_file, max_read_buffer_size); streams[stream_name] = std::make_unique<Stream>(storage.disk, storage.files[stream_name].data_file_path, max_read_buffer_size);
return &streams[stream_name]->compressed; return &streams[stream_name]->compressed;
}; };
@ -242,7 +242,7 @@ IDataType::OutputStreamGetter TinyLogBlockOutputStream::createStreamGetter(const
const auto & columns = storage.getColumns(); const auto & columns = storage.getColumns();
if (!streams.count(stream_name)) if (!streams.count(stream_name))
streams[stream_name] = std::make_unique<Stream>(storage.disk, streams[stream_name] = std::make_unique<Stream>(storage.disk,
storage.files[stream_name].data_file, storage.files[stream_name].data_file_path,
columns.getCodecOrDefault(name), columns.getCodecOrDefault(name),
storage.max_compress_block_size); storage.max_compress_block_size);
@ -291,7 +291,7 @@ void TinyLogBlockOutputStream::writeSuffix()
Strings column_files; Strings column_files;
for (auto & pair : streams) for (auto & pair : streams)
column_files.push_back(storage.files[pair.first].data_file); column_files.push_back(storage.files[pair.first].data_file_path);
storage.file_checker.update(column_files.begin(), column_files.end()); storage.file_checker.update(column_files.begin(), column_files.end());
@ -322,7 +322,7 @@ StorageTinyLog::StorageTinyLog(
const ConstraintsDescription & constraints_, const ConstraintsDescription & constraints_,
bool attach, bool attach,
size_t max_compress_block_size_) size_t max_compress_block_size_)
: disk(disk_), database_name(database_name_), table_name(table_name_), : disk(std::move(disk_)), database_name(database_name_), table_name(table_name_),
table_path("data/" + escapeForFileName(database_name_) + '/' + escapeForFileName(table_name_) + '/'), table_path("data/" + escapeForFileName(database_name_) + '/' + escapeForFileName(table_name_) + '/'),
max_compress_block_size(max_compress_block_size_), max_compress_block_size(max_compress_block_size_),
file_checker(disk, table_path + "sizes.json"), file_checker(disk, table_path + "sizes.json"),
@ -355,7 +355,7 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type
{ {
ColumnData column_data; ColumnData column_data;
files.insert(std::make_pair(stream_name, column_data)); files.insert(std::make_pair(stream_name, column_data));
files[stream_name].data_file = table_path + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION; files[stream_name].data_file_path = table_path + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION;
} }
}; };
@ -378,7 +378,7 @@ void StorageTinyLog::rename(const String & /*new_path_to_db*/, const String & ne
file_checker.setPath(table_path + "sizes.json"); file_checker.setPath(table_path + "sizes.json");
for (auto & file : files) for (auto & file : files)
file.second.data_file = table_path + Poco::Path(file.second.data_file).getFileName(); file.second.data_file_path = table_path + Poco::Path(file.second.data_file_path).getFileName();
} }

View File

@ -61,9 +61,9 @@ protected:
private: private:
struct ColumnData struct ColumnData
{ {
String data_file; String data_file_path;
}; };
using Files = std::map<String, ColumnData>; using Files = std::map<String, ColumnData>; /// file name -> column data
DiskPtr disk; DiskPtr disk;
String database_name; String database_name;