ClickHouse/dbms/src/Storages/StorageTinyLog.cpp

408 lines
12 KiB
C++
Raw Normal View History

2016-10-25 06:49:24 +00:00
#include <sys/stat.h>
#include <sys/types.h>
2012-06-25 00:17:19 +00:00
#include <map>
2015-01-25 05:07:51 +00:00
#include <Poco/Path.h>
2015-01-25 05:07:51 +00:00
#include <Poco/Util/XMLConfiguration.h>
2012-06-25 00:17:19 +00:00
#include <Common/escapeForFileName.h>
2012-06-25 00:17:19 +00:00
#include <Common/Exception.h>
2012-06-25 00:17:19 +00:00
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/CompressedReadBuffer.h>
#include <IO/CompressedWriteBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
2012-06-25 00:17:19 +00:00
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/DataTypesNumber.h>
2012-08-29 18:49:54 +00:00
#include <DataStreams/IProfilingBlockInputStream.h>
#include <DataStreams/IBlockOutputStream.h>
2015-01-18 08:25:56 +00:00
#include <Columns/ColumnArray.h>
2012-08-29 18:49:54 +00:00
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
#include <Interpreters/Context.h>
#include <Storages/StorageTinyLog.h>
#include <Poco/DirectoryIterator.h>
2012-06-25 00:17:19 +00:00
2017-07-27 23:23:13 +00:00
#define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin"
2012-06-25 00:17:19 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
extern const int CANNOT_CREATE_DIRECTORY;
extern const int CANNOT_READ_ALL_DATA;
extern const int DUPLICATE_COLUMN;
2017-07-27 23:23:13 +00:00
extern const int LOGICAL_ERROR;
}
2012-06-25 00:17:19 +00:00
class TinyLogBlockInputStream final : public IProfilingBlockInputStream
2012-06-25 00:17:19 +00:00
{
2015-01-18 08:25:56 +00:00
public:
TinyLogBlockInputStream(size_t block_size_, const Names & column_names_, StorageTinyLog & storage_, size_t max_read_buffer_size_)
: block_size(block_size_), column_names(column_names_), column_types(column_names.size()),
storage(storage_), max_read_buffer_size(max_read_buffer_size_) {}
2015-01-18 08:25:56 +00:00
String getName() const override { return "TinyLog"; }
2015-01-18 08:25:56 +00:00
String getID() const override;
2015-01-18 08:25:56 +00:00
protected:
Block readImpl() override;
2015-01-18 08:25:56 +00:00
private:
size_t block_size;
Names column_names;
DataTypes column_types;
StorageTinyLog & storage;
bool finished = false;
size_t max_read_buffer_size;
struct Stream
{
Stream(const std::string & data_path, size_t max_read_buffer_size)
: plain(data_path, std::min(static_cast<Poco::File::FileSize>(max_read_buffer_size), Poco::File(data_path).getSize())),
compressed(plain)
{
}
ReadBufferFromFile plain;
CompressedReadBuffer compressed;
};
using FileStreams = std::map<std::string, std::unique_ptr<Stream>>;
FileStreams streams;
void readData(const String & name, const IDataType & type, IColumn & column, size_t limit, bool read_offsets = true);
2015-01-18 08:25:56 +00:00
};
class TinyLogBlockOutputStream final : public IBlockOutputStream
2015-01-18 08:25:56 +00:00
{
public:
explicit TinyLogBlockOutputStream(StorageTinyLog & storage_)
: storage(storage_)
{
}
~TinyLogBlockOutputStream() override
{
try
{
writeSuffix();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void write(const Block & block) override;
void writeSuffix() override;
2015-04-02 23:58:26 +00:00
2015-01-18 08:25:56 +00:00
private:
StorageTinyLog & storage;
bool done = false;
struct Stream
{
Stream(const std::string & data_path, size_t max_compress_block_size) :
plain(data_path, max_compress_block_size, O_APPEND | O_CREAT | O_WRONLY),
compressed(plain, CompressionSettings(CompressionMethod::LZ4), max_compress_block_size)
{
}
WriteBufferFromFile plain;
CompressedWriteBuffer compressed;
void finalize()
{
compressed.next();
plain.next();
}
};
using FileStreams = std::map<std::string, std::unique_ptr<Stream>>;
FileStreams streams;
using WrittenStreams = std::set<std::string>;
void writeData(const String & name, const IDataType & type, const IColumn & column, WrittenStreams & written_streams);
2015-01-18 08:25:56 +00:00
};
2012-06-25 00:17:19 +00:00
String TinyLogBlockInputStream::getID() const
{
std::stringstream res;
res << "TinyLog(" << storage.getTableName() << ", " << &storage;
for (const auto & name : column_names)
res << ", " << name;
res << ")";
return res.str();
}
2012-06-25 00:17:19 +00:00
Block TinyLogBlockInputStream::readImpl()
{
Block res;
if (finished || (!streams.empty() && streams.begin()->second->compressed.eof()))
{
/** Close the files (before destroying the object).
* When many sources are created, but simultaneously reading only a few of them,
* buffers don't waste memory.
*/
finished = true;
streams.clear();
return res;
}
{
/// if there are no files in the folder, it means that the table is empty
if (Poco::DirectoryIterator(storage.full_path()) == Poco::DirectoryIterator())
return res;
}
/// If the files are not open, then open them.
if (streams.empty())
{
for (size_t i = 0, size = column_names.size(); i < size; ++i)
{
const auto & name = column_names[i];
column_types[i] = storage.getDataTypeByName(name);
}
}
/// Pointers to offset columns, shared for columns from nested data structures
using OffsetColumns = std::map<std::string, ColumnPtr>;
OffsetColumns offset_columns;
for (size_t i = 0, size = column_names.size(); i < size; ++i)
{
const auto & name = column_names[i];
MutableColumnPtr column;
bool read_offsets = true;
/// For nested structures, remember pointers to columns with offsets
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(column_types[i].get()))
{
String nested_name = DataTypeNested::extractNestedTableName(name);
if (offset_columns.count(nested_name) == 0)
offset_columns[nested_name] = ColumnArray::ColumnOffsets::create();
else
read_offsets = false; /// on previous iterations, the offsets were already calculated by `readData`
column = ColumnArray::create(type_arr->getNestedType()->createColumn(), offset_columns[nested_name]);
}
else
column = column_types[i]->createColumn();
try
{
readData(name, *column_types[i], *column, block_size, read_offsets);
}
catch (Exception & e)
{
e.addMessage("while reading column " + name + " at " + storage.full_path());
throw;
}
if (column->size())
res.insert(ColumnWithTypeAndName(std::move(column), column_types[i], name));
}
if (!res || streams.begin()->second->compressed.eof())
{
finished = true;
streams.clear();
}
return res;
2012-06-25 00:17:19 +00:00
}
void TinyLogBlockInputStream::readData(const String & name, const IDataType & type, IColumn & column, size_t limit, bool with_offsets)
2012-08-29 18:49:54 +00:00
{
IDataType::InputStreamGetter stream_getter = [&] (const IDataType::SubstreamPath & path) -> ReadBuffer *
{
if (!with_offsets && !path.empty() && path.back().type == IDataType::Substream::ArraySizes)
return nullptr;
2012-08-29 18:49:54 +00:00
String stream_name = IDataType::getFileNameForStream(name, path);
if (!streams.count(stream_name))
streams[stream_name] = std::make_unique<Stream>(storage.files[stream_name].data_file.path(), max_read_buffer_size);
return &streams[stream_name]->compressed;
};
type.deserializeBinaryBulkWithMultipleStreams(column, stream_getter, limit, 0, true, {}); /// TODO Use avg_value_size_hint.
2012-08-29 18:49:54 +00:00
}
void TinyLogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column, WrittenStreams & written_streams)
2012-08-29 18:49:54 +00:00
{
IDataType::OutputStreamGetter stream_getter = [&] (const IDataType::SubstreamPath & path) -> WriteBuffer *
{
String stream_name = IDataType::getFileNameForStream(name, path);
if (!streams.count(stream_name))
streams[stream_name] = std::make_unique<Stream>(storage.files[stream_name].data_file.path(), storage.max_compress_block_size);
else if (!written_streams.insert(stream_name).second)
return nullptr;
return &streams[stream_name]->compressed;
};
type.serializeBinaryBulkWithMultipleStreams(column, stream_getter, 0, 0, true, {});
2012-06-25 00:17:19 +00:00
}
2013-09-15 01:40:29 +00:00
void TinyLogBlockOutputStream::writeSuffix()
{
if (done)
return;
done = true;
2015-04-02 23:58:26 +00:00
/// Finish write.
2017-12-11 18:19:35 +00:00
for (auto & stream : streams)
stream.second->finalize();
std::vector<Poco::File> column_files;
for (auto & pair : streams)
column_files.push_back(storage.files[pair.first].data_file);
storage.file_checker.update(column_files.begin(), column_files.end());
2013-09-15 01:40:29 +00:00
streams.clear();
2013-09-15 01:40:29 +00:00
}
2012-06-25 00:17:19 +00:00
void TinyLogBlockOutputStream::write(const Block & block)
{
storage.check(block, true);
/// The set of written offset columns so that you do not write shared columns for nested structures multiple times
WrittenStreams written_streams;
2012-06-25 00:17:19 +00:00
for (size_t i = 0; i < block.columns(); ++i)
{
const ColumnWithTypeAndName & column = block.safeGetByPosition(i);
writeData(column.name, *column.type, *column.column, written_streams);
}
2012-06-25 00:17:19 +00:00
}
StorageTinyLog::StorageTinyLog(
const std::string & path_,
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
bool attach,
size_t max_compress_block_size_)
: IStorage{materialized_columns_, alias_columns_, column_defaults_},
path(path_), name(name_), columns(columns_),
max_compress_block_size(max_compress_block_size_),
file_checker(path + escapeForFileName(name) + '/' + "sizes.json"),
log(&Logger::get("StorageTinyLog"))
2012-06-25 00:17:19 +00:00
{
if (columns->empty())
throw Exception("Empty list of columns passed to StorageTinyLog constructor", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
String full_path = path + escapeForFileName(name) + '/';
if (!attach)
{
/// create files if they do not exist
if (0 != mkdir(full_path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST)
throwFromErrno("Cannot create directory " + full_path, ErrorCodes::CANNOT_CREATE_DIRECTORY);
}
for (const auto & col : getColumnsList())
addFiles(col.name, *col.type);
2012-08-29 18:49:54 +00:00
}
void StorageTinyLog::addFiles(const String & column_name, const IDataType & type)
2012-08-29 18:49:54 +00:00
{
if (files.end() != files.find(column_name))
throw Exception("Duplicate column with name " + column_name + " in constructor of StorageTinyLog.",
ErrorCodes::DUPLICATE_COLUMN);
IDataType::StreamCallback stream_callback = [&] (const IDataType::SubstreamPath & substream_path)
{
String stream_name = IDataType::getFileNameForStream(column_name, substream_path);
if (!files.count(stream_name))
{
ColumnData column_data;
files.insert(std::make_pair(stream_name, column_data));
files[stream_name].data_file = Poco::File(
path + escapeForFileName(name) + '/' + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION);
}
};
type.enumerateStreams(stream_callback, {});
2012-06-25 00:17:19 +00:00
}
2017-12-01 21:13:25 +00:00
void StorageTinyLog::rename(const String & new_path_to_db, const String & /*new_database_name*/, const String & new_table_name)
2012-06-25 00:17:19 +00:00
{
/// Rename directory with data.
Poco::File(path + escapeForFileName(name)).renameTo(new_path_to_db + escapeForFileName(new_table_name));
path = new_path_to_db;
name = new_table_name;
file_checker.setPath(path + escapeForFileName(name) + "/" + "sizes.json");
2012-06-25 00:17:19 +00:00
for (Files_t::iterator it = files.begin(); it != files.end(); ++it)
it->second.data_file = Poco::File(path + escapeForFileName(name) + '/' + Poco::Path(it->second.data_file.path()).getFileName());
2012-06-25 00:17:19 +00:00
}
BlockInputStreams StorageTinyLog::read(
const Names & column_names,
2017-12-01 21:13:25 +00:00
const SelectQueryInfo & /*query_info*/,
const Context & context,
QueryProcessingStage::Enum & processed_stage,
const size_t max_block_size,
2017-12-01 21:13:25 +00:00
const unsigned /*num_streams*/)
2012-06-25 00:17:19 +00:00
{
check(column_names);
processed_stage = QueryProcessingStage::FetchColumns;
return BlockInputStreams(1, std::make_shared<TinyLogBlockInputStream>(
max_block_size, column_names, *this, context.getSettingsRef().max_read_buffer_size));
2012-06-25 00:17:19 +00:00
}
2012-06-25 00:17:19 +00:00
BlockOutputStreamPtr StorageTinyLog::write(
2017-12-01 21:13:25 +00:00
const ASTPtr & /*query*/, const Settings & /*settings*/)
2012-06-25 00:17:19 +00:00
{
return std::make_shared<TinyLogBlockOutputStream>(*this);
2012-06-25 00:17:19 +00:00
}
bool StorageTinyLog::checkData() const
{
return file_checker.check();
}
2012-06-25 00:17:19 +00:00
}