ClickHouse/dbms/src/Storages/StorageTinyLog.h

80 lines
2.0 KiB
C++
Raw Normal View History

2012-06-25 00:17:19 +00:00
#pragma once
2015-01-25 05:07:51 +00:00
#include <map>
2017-06-06 17:18:32 +00:00
#include <ext/shared_ptr_helper.h>
2012-06-25 00:17:19 +00:00
#include <Poco/File.h>
#include <Storages/IStorage.h>
#include <Common/FileChecker.h>
#include <Common/escapeForFileName.h>
2017-07-24 12:58:01 +00:00
#include <Core/Defines.h>
2012-06-25 00:17:19 +00:00
namespace DB
{
/** Implements a table engine that is suitable for small chunks of the log.
2017-04-16 15:00:33 +00:00
* It differs from StorageLog in the absence of mark files.
2012-06-25 00:17:19 +00:00
*/
class StorageTinyLog : public ext::shared_ptr_helper<StorageTinyLog>, public IStorage
2012-06-25 00:17:19 +00:00
{
friend class TinyLogBlockInputStream;
friend class TinyLogBlockOutputStream;
public:
std::string getName() const override { return "TinyLog"; }
std::string getTableName() const override { return name; }
BlockInputStreams read(
const Names & column_names,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum & processed_stage,
2017-06-02 15:54:39 +00:00
size_t max_block_size,
unsigned num_streams) override;
2017-05-21 22:25:25 +00:00
BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override;
bool checkData() const override;
2017-04-16 15:00:33 +00:00
/// Column data
struct ColumnData
{
Poco::File data_file;
};
using Files_t = std::map<String, ColumnData>;
std::string full_path() const { return path + escapeForFileName(name) + '/';}
String getDataPath() const override { return full_path(); }
2012-06-25 00:17:19 +00:00
private:
String path;
String name;
2012-06-25 00:17:19 +00:00
size_t max_compress_block_size;
2014-03-28 14:36:24 +00:00
Files_t files;
2012-08-29 18:49:54 +00:00
FileChecker file_checker;
Logger * log;
void addFile(const String & column_name, const IDataType & type, size_t level = 0);
void addFiles(const String & column_name, const IDataType & type);
protected:
StorageTinyLog(
const std::string & path_,
const std::string & name_,
const ColumnsDescription & columns_,
bool attach,
size_t max_compress_block_size_);
2012-06-25 00:17:19 +00:00
};
}