ClickHouse/dbms/include/DB/Storages/MergeTree/MergeTreeData.h

540 lines
20 KiB
C++
Raw Normal View History

2014-03-09 17:36:01 +00:00
#pragma once
#include <DB/Core/SortDescription.h>
#include <DB/Interpreters/Context.h>
#include <DB/Interpreters/ExpressionActions.h>
#include <DB/Storages/IStorage.h>
#include <DB/Storages/MergeTree/ActiveDataPartSet.h>
2015-07-16 21:03:53 +00:00
#include <DB/Storages/MergeTree/MergeTreeSettings.h>
2014-04-02 07:59:43 +00:00
#include <DB/IO/ReadBufferFromString.h>
2014-07-16 09:32:15 +00:00
#include <DB/IO/WriteBufferFromFile.h>
2015-04-16 06:12:35 +00:00
#include <DB/IO/ReadBufferFromFile.h>
2014-07-28 10:36:11 +00:00
#include <DB/DataTypes/DataTypeString.h>
2014-07-30 12:10:34 +00:00
#include <DB/DataTypes/DataTypesNumberFixed.h>
2016-04-24 09:44:47 +00:00
#include <DB/DataStreams/GraphiteRollupSortedBlockInputStream.h>
2016-02-14 04:58:47 +00:00
#include <DB/Storages/MergeTree/MergeTreeDataPart.h>
2014-04-02 13:45:39 +00:00
2015-06-02 20:22:53 +00:00
struct SimpleIncrement;
2014-03-09 17:36:01 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int INVALID_PARTITION_NAME;
extern const int TOO_MUCH_PARTS;
extern const int NO_SUCH_DATA_PART;
extern const int DUPLICATE_DATA_PART;
extern const int DIRECTORY_ALREADY_EXISTS;
extern const int TOO_MANY_UNEXPECTED_DATA_PARTS;
extern const int NO_SUCH_COLUMN_IN_TABLE;
extern const int TABLE_DIFFERS_TOO_MUCH;
}
2017-02-07 17:52:41 +00:00
/// Data structure for *MergeTree engines.
/// Merge tree is used for incremental sorting of data.
/// The table consists of several sorted parts.
/// During insertion new data is sorted according to the primary key and is written to the new part.
/// Parts are merged in the background according to a heuristic algorithm.
/// For each part the index file is created containing primary key values for every n-th row.
/// This allows efficient selection by primary key range predicate.
///
/// Additionally:
///
/// The date column is specified. For each part min and max dates are remembered.
/// Essentially it is an index too.
///
/// Data is partitioned by month. Parts belonging to different months are not merged - for the ease of
/// administration (data sync and backup).
///
/// File structure:
/// Part directory - / min-date _ max-date _ min-id _ max-id _ level /
/// Inside the part directory:
/// checksums.txt - contains the list of all files along with their sizes and checksums.
/// columns.txt - contains the list of all columns and their types.
/// primary.idx - contains the primary index.
/// [Column].bin - contains compressed column data.
/// [Column].mrk - marks, pointing to seek positions allowing to skip n * k rows.
///
/// Several modes are implemented. Modes determine additional actions during merge:
/// - Ordinary - don't do anything special
/// - Collapsing - collapse pairs of rows with the opposite values of sign_columns for the same values
/// of primary key (cf. CollapsingSortedBlockInputStream.h)
/// - Replacing - for all rows with the same primary key keep only the latest one. Or, if the version
/// column is set, keep the latest row with the maximal version.
/// - Summing - sum all numeric columns not contained in the primary key for all rows with the same primary key.
/// - Aggregating - merge columns containing aggregate function states for all rows with the same primary key.
/// - Unsorted - during the merge the data is not sorted but merely concatenated; this allows reading the data
/// in the same batches as they were written.
/// - Graphite - performs coarsening of historical data for Graphite (a system for quantitative monitoring).
/// The MergeTreeData class contains a list of parts and the data structure parameters.
/// To read and modify the data use other classes:
/// - MergeTreeDataSelectExecutor
/// - MergeTreeDataWriter
/// - MergeTreeDataMerger
class MergeTreeData : public ITableDeclaration
2014-03-09 17:36:01 +00:00
{
2016-01-28 01:00:27 +00:00
friend class ReshardingWorker;
2014-03-09 17:36:01 +00:00
public:
2017-02-07 17:52:41 +00:00
/// Function to call if the part is suspected to contain corrupt data.
2016-02-14 04:58:47 +00:00
using BrokenPartCallback = std::function<void (const String &)>;
using DataPart = MergeTreeDataPart;
2016-02-14 04:58:47 +00:00
using MutableDataPartPtr = std::shared_ptr<DataPart>;
2017-02-07 17:52:41 +00:00
/// After the DataPart is added to the working set, it cannot be changed.
2016-02-14 04:58:47 +00:00
using DataPartPtr = std::shared_ptr<const DataPart>;
2014-03-09 17:36:01 +00:00
struct DataPartPtrLess { bool operator() (const DataPartPtr & lhs, const DataPartPtr & rhs) const { return *lhs < *rhs; } };
2016-02-14 04:58:47 +00:00
using DataParts = std::set<DataPartPtr, DataPartPtrLess>;
using DataPartsVector = std::vector<DataPartPtr>;
2014-03-13 12:48:07 +00:00
2017-02-07 17:52:41 +00:00
/// For resharding.
2016-01-28 01:00:27 +00:00
using MutableDataParts = std::set<MutableDataPartPtr, DataPartPtrLess>;
2016-01-28 16:06:57 +00:00
using PerShardDataParts = std::unordered_map<size_t, MutableDataPartPtr>;
2014-03-09 17:36:01 +00:00
2017-02-07 17:52:41 +00:00
/// Some operations on the set of parts return a Transaction object.
/// If neither commit() nor rollback() was called, the destructor rollbacks the operation.
2014-07-01 15:58:25 +00:00
class Transaction : private boost::noncopyable
{
public:
Transaction() {}
void commit()
{
clear();
}
void rollback()
{
if (data && (!parts_to_remove_on_rollback.empty() || !parts_to_add_on_rollback.empty()))
{
LOG_DEBUG(data->log, "Undoing transaction");
data->replaceParts(parts_to_remove_on_rollback, parts_to_add_on_rollback, true);
clear();
}
2014-07-01 15:58:25 +00:00
}
~Transaction()
{
try
{
rollback();
2014-07-01 15:58:25 +00:00
}
catch(...)
{
tryLogCurrentException("~MergeTreeData::Transaction");
}
}
private:
friend class MergeTreeData;
MergeTreeData * data = nullptr;
2017-02-07 17:52:41 +00:00
/// What to do on rollback.
DataPartsVector parts_to_remove_on_rollback;
DataPartsVector parts_to_add_on_rollback;
void clear()
{
data = nullptr;
parts_to_remove_on_rollback.clear();
parts_to_add_on_rollback.clear();
}
2014-07-01 15:58:25 +00:00
};
2017-02-07 17:52:41 +00:00
/// An object that stores the names of temporary files created in the part directory during ALTER of its
/// columns.
2014-07-11 12:47:45 +00:00
class AlterDataPartTransaction : private boost::noncopyable
{
public:
2017-02-07 17:52:41 +00:00
/// Renames temporary files, finishing the ALTER of the part.
2014-07-11 12:47:45 +00:00
void commit();
2017-02-07 17:52:41 +00:00
/// If commit() was not called, deletes temporary files, canceling the ALTER.
2014-07-11 12:47:45 +00:00
~AlterDataPartTransaction();
2017-02-07 17:52:41 +00:00
/// Review the changes before the commit.
const NamesAndTypesList & getNewColumns() const { return new_columns; }
const DataPart::Checksums & getNewChecksums() const { return new_checksums; }
2014-07-11 12:47:45 +00:00
private:
friend class MergeTreeData;
2014-07-14 14:07:47 +00:00
AlterDataPartTransaction(DataPartPtr data_part_) : data_part(data_part_), alter_lock(data_part->alter_mutex) {}
void clear()
{
alter_lock.unlock();
data_part = nullptr;
}
2014-07-11 12:47:45 +00:00
DataPartPtr data_part;
2016-01-29 02:22:43 +00:00
std::unique_lock<std::mutex> alter_lock;
2014-07-14 14:07:47 +00:00
DataPart::Checksums new_checksums;
NamesAndTypesList new_columns;
2017-02-07 17:52:41 +00:00
/// If the value is an empty string, the file is not temporary, and it must be deleted.
2014-07-11 12:47:45 +00:00
NameToNameMap rename_map;
};
2016-02-14 04:58:47 +00:00
using AlterDataPartTransactionPtr = std::unique_ptr<AlterDataPartTransaction>;
2014-07-11 12:47:45 +00:00
2016-04-15 17:13:51 +00:00
2017-02-07 17:52:41 +00:00
/// Parameters for various modes.
2016-04-15 17:13:51 +00:00
struct MergingParams
{
2017-02-07 17:52:41 +00:00
/// Merging mode. See above.
2016-04-15 17:42:51 +00:00
enum Mode
{
2017-02-07 17:52:41 +00:00
Ordinary = 0, /// Enum values are saved. Do not change them.
2016-04-15 17:42:51 +00:00
Collapsing = 1,
Summing = 2,
Aggregating = 3,
Unsorted = 4,
Replacing = 5,
2016-04-24 09:44:47 +00:00
Graphite = 6,
2016-04-15 17:42:51 +00:00
};
Mode mode;
2017-02-07 17:52:41 +00:00
/// For collapsing mode.
2016-04-15 17:13:51 +00:00
String sign_column;
2017-02-07 17:52:41 +00:00
/// For Summing mode. If empty - columns_to_sum is determined automatically.
2016-04-15 17:13:51 +00:00
Names columns_to_sum;
2017-02-07 17:52:41 +00:00
/// For Replacing mode. Can be empty.
2016-04-15 17:13:51 +00:00
String version_column;
2016-04-15 17:42:51 +00:00
2017-02-07 17:52:41 +00:00
/// For Graphite mode.
2016-04-24 09:44:47 +00:00
Graphite::Params graphite_params;
2017-02-07 17:52:41 +00:00
/// Check that needed columns are present and have correct types.
2016-04-15 17:42:51 +00:00
void check(const NamesAndTypesList & columns) const;
2016-04-24 09:44:47 +00:00
String getModeName() const;
2014-03-13 12:48:07 +00:00
};
2016-04-15 17:42:51 +00:00
2017-02-07 17:52:41 +00:00
/// Attach the table corresponding to the directory in full_path (must end with /), with the given columns.
/// Correctness of names and paths is not checked.
///
/// primary_expr_ast - expression used for sorting; empty for UnsortedMergeTree.
/// index_granularity - how many rows correspond to one primary key value.
/// require_part_metadata - should checksums.txt and columns.txt exist in the part directory.
/// attach - whether the existing table is attached or the new table is created.
2014-03-13 12:48:07 +00:00
MergeTreeData( const String & full_path_, NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
2016-01-28 01:00:27 +00:00
Context & context_,
2014-03-13 12:48:07 +00:00
ASTPtr & primary_expr_ast_,
const String & date_column_name_,
2017-02-07 17:52:41 +00:00
const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported.
2014-03-13 12:48:07 +00:00
size_t index_granularity_,
2016-04-15 17:13:51 +00:00
const MergingParams & merging_params_,
2014-05-08 07:12:01 +00:00
const MergeTreeSettings & settings_,
2014-07-09 13:39:19 +00:00
const String & log_name_,
2014-07-23 09:15:41 +00:00
bool require_part_metadata_,
bool attach,
BrokenPartCallback broken_part_callback_ = [](const String &){});
2014-03-09 17:36:01 +00:00
2017-02-07 17:52:41 +00:00
/// Load the set of data parts from disk. Call once - immediately after the object is created.
2014-08-13 08:07:52 +00:00
void loadDataParts(bool skip_sanity_checks);
2014-03-13 12:48:07 +00:00
bool supportsSampling() const { return !!sampling_expression; }
bool supportsPrewhere() const { return true; }
2015-07-08 04:38:46 +00:00
bool supportsFinal() const
{
2016-04-15 17:42:51 +00:00
return merging_params.mode == MergingParams::Collapsing
|| merging_params.mode == MergingParams::Summing
|| merging_params.mode == MergingParams::Aggregating
|| merging_params.mode == MergingParams::Replacing;
2015-07-08 04:38:46 +00:00
}
2015-08-17 21:09:36 +00:00
Int64 getMaxDataPartIndex();
2014-03-13 12:48:07 +00:00
std::string getTableName() const override
{
throw Exception("Logical error: calling method getTableName of not a table.", ErrorCodes::LOGICAL_ERROR);
2014-05-28 14:54:42 +00:00
}
2014-03-13 12:48:07 +00:00
const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
2014-03-13 12:48:07 +00:00
NameAndTypePair getColumn(const String & column_name) const override
2014-07-28 10:36:11 +00:00
{
if (column_name == "_part")
return NameAndTypePair("_part", std::make_shared<DataTypeString>());
if (column_name == "_part_index")
return NameAndTypePair("_part_index", std::make_shared<DataTypeUInt64>());
if (column_name == "_sample_factor")
return NameAndTypePair("_sample_factor", std::make_shared<DataTypeFloat64>());
return ITableDeclaration::getColumn(column_name);
2014-07-28 10:36:11 +00:00
}
bool hasColumn(const String & column_name) const override
2014-07-28 10:36:11 +00:00
{
return ITableDeclaration::hasColumn(column_name)
|| column_name == "_part"
|| column_name == "_part_index"
|| column_name == "_sample_factor";
2014-07-28 10:36:11 +00:00
}
String getFullPath() const { return full_path; }
2014-03-13 12:48:07 +00:00
2014-05-08 07:12:01 +00:00
String getLogName() const { return log_name; }
2017-02-07 17:52:41 +00:00
/// Returns a copy of the list so that the caller shouldn't worry about locks.
DataParts getDataParts() const;
DataPartsVector getDataPartsVector() const;
DataParts getAllDataParts() const;
2014-05-16 15:55:57 +00:00
2017-02-07 17:52:41 +00:00
/// Total size of active parts in bytes.
size_t getTotalActiveSizeInBytes() const;
size_t getMaxPartsCountForMonth() const;
2014-03-13 17:44:00 +00:00
2017-02-07 17:52:41 +00:00
/// Returns a pair of min block number in the given month and a bool denoting if there is at least one part.
std::pair<Int64, bool> getMinBlockNumberForMonth(DayNum_t month) const;
2017-02-07 17:52:41 +00:00
/// Returns true if block_number is contained in some part of the given month.
bool hasBlockNumberInMonth(Int64 block_number, DayNum_t month) const;
2017-02-07 17:52:41 +00:00
/// If the table contains too many active parts, sleep for a while to give them time to merge.
/// If until is non-null, wake up from the sleep earlier if the event happened.
void delayInsertIfNeeded(Poco::Event * until = nullptr);
2014-05-27 08:43:01 +00:00
2017-02-07 17:52:41 +00:00
/// Returns an active part with the given name or a part containing it. If there is no such part,
/// returns nullptr.
2014-07-25 11:38:46 +00:00
DataPartPtr getActiveContainingPart(const String & part_name);
2017-02-07 17:52:41 +00:00
/// Returns the part with the given name or nullptr if no such part.
2014-07-25 11:38:46 +00:00
DataPartPtr getPartIfExists(const String & part_name);
2016-01-28 01:00:27 +00:00
DataPartPtr getShardedPartIfExists(const String & part_name, size_t shard_no);
2014-03-13 17:44:00 +00:00
2017-02-07 17:52:41 +00:00
/// Renames temporary part to a permanent part and adds it to the working set.
/// If increment != nullptr, part index is determing using increment. Otherwise part index remains unchanged.
/// It is assumed that the part does not intersect with existing parts.
/// If out_transaction != nullptr, sets it to an object allowing to rollback part addition (but not the renaming).
2015-06-02 20:22:53 +00:00
void renameTempPartAndAdd(MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr);
2014-04-07 15:45:46 +00:00
2017-02-07 17:52:41 +00:00
/// The same as renameTempPartAndAdd but the part can intersect existing parts.
/// Deletes and returns all parts covered by the added part (in ascending order).
DataPartsVector renameTempPartAndReplace(
MutableDataPartPtr & part, SimpleIncrement * increment = nullptr, Transaction * out_transaction = nullptr);
2014-07-01 15:58:25 +00:00
2017-02-07 17:52:41 +00:00
/// Removes from the working set parts in remove and adds parts in add. Parts in add must already be in
/// all_data_parts.
/// If clear_without_timeout is true, the parts will be deleted at once, or during the next call to
/// clearOldParts (ignoring old_parts_lifetime).
2014-07-07 10:23:24 +00:00
void replaceParts(const DataPartsVector & remove, const DataPartsVector & add, bool clear_without_timeout);
2014-03-13 17:44:00 +00:00
2017-02-07 17:52:41 +00:00
/// Adds new part to the list of known parts and to the working set.
2014-09-29 20:26:46 +00:00
void attachPart(const DataPartPtr & part);
2014-08-08 08:28:13 +00:00
2017-02-07 17:52:41 +00:00
/// Renames the part to detached/<prefix>_<part> and forgets about it. The data won't be deleted in
/// clearOldParts.
/// If restore_covered is true, adds to the working set inactive parts, which were merged into the deleted part.
2014-09-29 20:26:46 +00:00
void renameAndDetachPart(const DataPartPtr & part, const String & prefix = "", bool restore_covered = false, bool move_to_detached = true);
2014-08-08 08:28:13 +00:00
2017-02-07 17:52:41 +00:00
/// Removes the part from the list of parts (including all_data_parts), but doesn't move the directory.
2014-09-29 20:26:46 +00:00
void detachPartInPlace(const DataPartPtr & part);
2014-04-02 10:10:37 +00:00
2017-02-07 17:52:41 +00:00
/// Returns old inactive parts that can be deleted. At the same time removes them from the list of parts
/// but not from the disk.
2014-07-25 11:15:11 +00:00
DataPartsVector grabOldParts();
2017-02-07 17:52:41 +00:00
/// Reverts the changes made by grabOldParts().
2014-07-25 11:15:11 +00:00
void addOldParts(const DataPartsVector & parts);
2017-02-07 17:52:41 +00:00
/// Delete irrelevant parts.
2014-07-25 11:15:11 +00:00
void clearOldParts();
2014-03-13 17:44:00 +00:00
void clearOldTemporaryDirectories();
2017-02-07 17:52:41 +00:00
/// After the call to dropAllData() no method can be called.
/// Deletes the data directory and flushes the uncompressed blocks cache and the marks cache.
2014-03-13 17:44:00 +00:00
void dropAllData();
2017-02-07 17:52:41 +00:00
/// Moves the entire data directory.
/// Flushes the uncompressed blocks cache and the marks cache.
/// Must be called with locked lockStructureForAlter().
2014-07-28 14:33:30 +00:00
void setPath(const String & full_path, bool move_data);
2014-03-13 17:44:00 +00:00
2017-02-07 17:52:41 +00:00
/// Check if the ALTER can be performed:
/// - all needed columns are present.
/// - all type conversions can be done.
/// - columns corresponding to primary key, sign, sampling expression and date are not affected.
/// If something is wrong, throws an exception.
void checkAlter(const AlterCommands & commands);
2014-07-11 12:47:45 +00:00
2017-02-07 17:52:41 +00:00
/// Performs ALTER of the data part, writes the result to temporary files.
/// Returns an object allowing to rename temporary files to permanent files.
/// If the number of affected columns is suspiciously high and skip_sanity_checks is false, throws an exception.
/// If no data transformations are necessary, returns nullptr.
2016-05-13 21:08:19 +00:00
AlterDataPartTransactionPtr alterDataPart(
const DataPartPtr & part,
const NamesAndTypesList & new_columns,
const ASTPtr & new_primary_key,
2016-05-13 21:08:19 +00:00
bool skip_sanity_checks);
2014-07-11 12:47:45 +00:00
2017-02-07 17:52:41 +00:00
/// Must be called with locked lockStructureForAlter().
void setColumnsList(const NamesAndTypesList & new_columns) { columns = std::make_shared<NamesAndTypesList>(new_columns); }
2014-03-14 17:03:52 +00:00
2017-02-07 17:52:41 +00:00
/// Should be called if part data is suspected to be corrupted.
2014-07-23 09:15:41 +00:00
void reportBrokenPart(const String & name)
{
broken_part_callback(name);
}
2014-03-14 17:03:52 +00:00
ExpressionActionsPtr getPrimaryExpression() const { return primary_expr; }
SortDescription getSortDescription() const { return sort_descr; }
2017-02-07 17:52:41 +00:00
/// Check that the part is not broken and calculate the checksums for it if they are not present.
2014-08-08 08:28:13 +00:00
/// Проверить, что кусок не сломан и посчитать для него чексуммы, если их нет.
MutableDataPartPtr loadPartAndFixMetadata(const String & relative_path);
/** Create local backup (snapshot) for parts with specified prefix.
* Backup is created in directory clickhouse_dir/shadow/i/, where i - incremental number,
* or if 'with_name' is specified - backup is created in directory with specified name.
*/
void freezePartition(const std::string & prefix, const String & with_name);
2017-02-07 17:52:41 +00:00
/// Returns the size of partition in bytes.
2016-01-28 01:00:27 +00:00
size_t getPartitionSize(const std::string & partition_name) const;
struct ColumnSize
{
size_t marks = 0;
size_t data_compressed = 0;
size_t data_uncompressed = 0;
size_t getTotalCompressedSize() const
{
return marks + data_compressed;
}
};
size_t getColumnCompressedSize(const std::string & name) const
2014-09-19 11:44:29 +00:00
{
2016-01-29 02:22:43 +00:00
std::lock_guard<std::mutex> lock{data_parts_mutex};
2014-09-23 11:35:27 +00:00
2014-09-19 11:44:29 +00:00
const auto it = column_sizes.find(name);
return it == std::end(column_sizes) ? 0 : it->second.data_compressed;
2014-09-19 11:44:29 +00:00
}
using ColumnSizes = std::unordered_map<std::string, ColumnSize>;
ColumnSizes getColumnSizes() const
{
2016-01-29 02:22:43 +00:00
std::lock_guard<std::mutex> lock{data_parts_mutex};
return column_sizes;
}
/// NOTE Could be off after DROPped and MODIFYed columns in ALTER. Doesn't include primary.idx.
size_t getTotalCompressedSize() const
{
std::lock_guard<std::mutex> lock{data_parts_mutex};
size_t total_size = 0;
for (const auto & col : column_sizes)
total_size += col.second.getTotalCompressedSize();
return total_size;
}
2017-02-07 17:52:41 +00:00
/// For ATTACH/DETACH/DROP/RESHARD PARTITION.
2014-10-03 17:57:01 +00:00
static String getMonthName(const Field & partition);
2016-01-28 01:00:27 +00:00
static String getMonthName(DayNum_t month);
2014-10-03 17:57:01 +00:00
static DayNum_t getMonthDayNum(const Field & partition);
2016-01-28 01:00:27 +00:00
static DayNum_t getMonthFromName(const String & month_name);
2017-02-07 17:52:41 +00:00
/// Get month from the part name or a sufficient prefix.
static DayNum_t getMonthFromPartPrefix(const String & part_prefix);
2014-10-03 17:57:01 +00:00
2016-01-28 01:00:27 +00:00
Context & context;
2014-03-14 17:03:52 +00:00
const String date_column_name;
const ASTPtr sampling_expression;
const size_t index_granularity;
2014-03-13 12:48:07 +00:00
2017-02-07 17:52:41 +00:00
/// Merging params - what additional actions to perform during merge.
2016-04-15 17:13:51 +00:00
const MergingParams merging_params;
2014-03-13 12:48:07 +00:00
2014-03-14 17:03:52 +00:00
const MergeTreeSettings settings;
2014-03-13 12:48:07 +00:00
ASTPtr primary_expr_ast;
Block primary_key_sample;
DataTypes primary_key_data_types;
2014-03-22 14:44:44 +00:00
2014-03-14 17:03:52 +00:00
private:
2016-03-07 04:31:10 +00:00
friend struct MergeTreeDataPart;
friend class StorageMergeTree;
friend class ReplicatedMergeTreeAlterThread;
friend class MergeTreeDataMerger;
2016-02-14 04:58:47 +00:00
2014-07-09 13:39:19 +00:00
bool require_part_metadata;
2014-03-13 12:48:07 +00:00
ExpressionActionsPtr primary_expr;
SortDescription sort_descr;
String full_path;
2014-03-09 17:36:01 +00:00
2014-03-13 12:48:07 +00:00
NamesAndTypesListPtr columns;
/// Current column sizes in compressed and uncompressed form.
ColumnSizes column_sizes;
2014-03-09 17:36:01 +00:00
2014-07-23 09:15:41 +00:00
BrokenPartCallback broken_part_callback;
2014-05-08 07:12:01 +00:00
String log_name;
2014-03-13 12:48:07 +00:00
Logger * log;
2014-03-09 17:36:01 +00:00
2017-02-07 17:52:41 +00:00
/// Current set of data parts.
2014-03-09 17:36:01 +00:00
DataParts data_parts;
2016-01-29 02:22:43 +00:00
mutable std::mutex data_parts_mutex;
2014-03-09 17:36:01 +00:00
2017-02-07 17:52:41 +00:00
/// The set of all data parts including already merged but not yet deleted. Usually it is small (tens of elements).
/// The part is referenced from here, from the list of current parts and from each thread reading from it.
/// This means that if reference count is 1 - the part is not used right now and can be deleted.
2014-03-09 17:36:01 +00:00
DataParts all_data_parts;
2016-01-29 02:22:43 +00:00
mutable std::mutex all_data_parts_mutex;
2014-03-09 17:36:01 +00:00
2017-02-07 17:52:41 +00:00
/// Used to serialize calls to grabOldParts.
std::mutex grab_old_parts_mutex;
2017-02-07 17:52:41 +00:00
/// The same for clearOldTemporaryDirectories.
2016-04-18 21:38:06 +00:00
std::mutex clear_old_temporary_directories_mutex;
2017-02-07 17:52:41 +00:00
/// For each shard of the set of sharded parts.
2016-01-28 01:00:27 +00:00
PerShardDataParts per_shard_data_parts;
2017-02-07 17:52:41 +00:00
/// Check that columns list doesn't contain multidimensional arrays.
/// If attach is true (attaching an existing table), writes an error message to log.
/// Otherwise (new table or alter) throws an exception.
void checkNoMultidimensionalArrays(const NamesAndTypesList & columns, bool attach) const;
void initPrimaryKey();
2017-02-07 17:52:41 +00:00
/// Expression for column type conversion.
/// If no conversions are needed, out_expression=nullptr.
/// out_rename_map maps column files for the out_expression onto new table files.
/// out_force_update_metadata denotes if metadata must be changed even if out_rename_map is empty (used
/// for transformation-free changing of Enum values list).
/// Files to be deleted are mapped to an empty string in out_rename_map.
/// If part == nullptr, just checks that all type conversions are possible.
2014-09-29 20:26:46 +00:00
void createConvertExpression(const DataPartPtr & part, const NamesAndTypesList & old_columns, const NamesAndTypesList & new_columns,
ExpressionActionsPtr & out_expression, NameToNameMap & out_rename_map, bool & out_force_update_metadata) const;
2014-09-19 11:44:29 +00:00
2017-02-07 17:52:41 +00:00
/// Calculates column sizes in compressed form for the current state of data_parts. Call with data_parts mutex locked.
2014-09-19 11:44:29 +00:00
void calculateColumnSizes();
2017-02-07 17:52:41 +00:00
/// Adds or subtracts the contribution of the part to compressed column sizes.
2014-09-19 11:44:29 +00:00
void addPartContributionToColumnSizes(const DataPartPtr & part);
void removePartContributionToColumnSizes(const DataPartPtr & part);
2014-03-09 17:36:01 +00:00
};
}