ClickHouse/src/Storages/StorageKeeperMap.h

137 lines
4.2 KiB
C++
Raw Normal View History

2022-07-26 09:08:55 +00:00
#pragma once
#include <Interpreters/Context.h>
#include <Interpreters/IKeyValueEntity.h>
2022-07-26 09:08:55 +00:00
#include <QueryPipeline/Pipe.h>
#include <Storages/IStorage.h>
2022-07-27 13:20:45 +00:00
#include <Storages/StorageInMemoryMetadata.h>
2022-08-08 14:09:37 +00:00
#include <Common/PODArray_fwd.h>
2022-07-27 13:20:45 +00:00
#include <Common/ZooKeeper/ZooKeeper.h>
#include <span>
2022-07-26 09:08:55 +00:00
namespace DB
{
2022-08-23 13:15:31 +00:00
namespace ErrorCodes
{
extern const int INVALID_STATE;
}
2022-07-27 13:20:45 +00:00
// KV store using (Zoo|CH)Keeper
2022-08-23 13:15:31 +00:00
class StorageKeeperMap final : public IStorage, public IKeyValueEntity, WithContext
2022-07-26 09:08:55 +00:00
{
public:
StorageKeeperMap(
2022-08-23 13:15:31 +00:00
ContextPtr context_,
2022-07-27 13:20:45 +00:00
const StorageID & table_id,
const StorageInMemoryMetadata & metadata,
bool attach,
2022-07-27 13:20:45 +00:00
std::string_view primary_key_,
2022-08-23 13:15:31 +00:00
const std::string & root_path_,
2022-08-31 08:14:28 +00:00
UInt64 keys_limit_);
2022-07-26 09:08:55 +00:00
Pipe read(
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
unsigned num_streams) override;
2022-07-27 13:20:45 +00:00
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override;
2022-07-26 09:08:55 +00:00
2022-08-23 13:15:31 +00:00
void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override;
void drop() override;
2022-07-27 13:20:45 +00:00
std::string getName() const override { return "KeeperMap"; }
Names getPrimaryKey() const override { return {primary_key}; }
2022-07-26 09:08:55 +00:00
Chunk getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray<UInt8> & null_map, const Names &) const override;
2022-07-27 13:20:45 +00:00
Chunk getBySerializedKeys(std::span<const std::string> keys, PaddedPODArray<UInt8> * null_map) const;
2022-07-26 09:08:55 +00:00
Block getSampleBlock(const Names &) const override;
2022-09-08 12:45:36 +00:00
void checkTableCanBeRenamed(const StorageID & new_name) const override;
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
2022-07-27 13:20:45 +00:00
bool supportsParallelInsert() const override { return true; }
bool supportsIndexForIn() const override { return true; }
bool mayBenefitFromIndexForIn(
const ASTPtr & node, ContextPtr /*query_context*/, const StorageMetadataPtr & /*metadata_snapshot*/) const override
{
return node->getColumnName() == primary_key;
}
2022-08-31 08:14:28 +00:00
zkutil::ZooKeeperPtr getClient() const;
2022-07-26 09:08:55 +00:00
const std::string & rootKeeperPath() const;
2022-07-27 13:20:45 +00:00
std::string fullPathForKey(std::string_view key) const;
2022-07-26 09:08:55 +00:00
2022-08-31 08:14:28 +00:00
UInt64 keysLimit() const;
2022-08-23 13:15:31 +00:00
template <bool throw_on_error>
void checkTable() const
{
auto is_table_valid = isTableValid();
if (!is_table_valid.has_value())
{
static constexpr std::string_view error_msg = "Failed to activate table because of connection issues. It will be activated "
"once a connection is established and metadata is verified";
if constexpr (throw_on_error)
throw Exception(ErrorCodes::INVALID_STATE, error_msg);
else
{
LOG_ERROR(log, fmt::runtime(error_msg));
return;
}
}
if (!*is_table_valid)
{
static constexpr std::string_view error_msg
= "Failed to activate table because of invalid metadata in ZooKeeper. Please DETACH table";
if constexpr (throw_on_error)
throw Exception(ErrorCodes::INVALID_STATE, error_msg);
else
{
LOG_ERROR(log, fmt::runtime(error_msg));
return;
}
}
}
2022-08-08 09:43:29 +00:00
2022-07-27 13:20:45 +00:00
private:
2022-08-31 08:23:44 +00:00
bool dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock);
2022-08-24 17:27:07 +00:00
2022-08-31 08:14:28 +00:00
std::optional<bool> isTableValid() const;
2022-09-01 14:39:57 +00:00
std::string root_path;
2022-07-27 13:20:45 +00:00
std::string primary_key;
2022-08-24 17:27:07 +00:00
2022-08-08 09:43:29 +00:00
std::string metadata_path;
2022-08-24 17:27:07 +00:00
2022-08-23 13:15:31 +00:00
std::string tables_path;
2022-08-24 17:27:07 +00:00
std::string table_path;
2022-08-23 13:15:31 +00:00
std::string dropped_path;
2022-08-24 17:27:07 +00:00
std::string dropped_lock_path;
2022-08-23 13:15:31 +00:00
std::string zookeeper_name;
2022-09-08 12:45:36 +00:00
std::string metadata_string;
2022-08-31 08:14:28 +00:00
uint64_t keys_limit{0};
2022-08-23 13:15:31 +00:00
mutable std::mutex zookeeper_mutex;
mutable zkutil::ZooKeeperPtr zookeeper_client{nullptr};
2022-07-27 13:20:45 +00:00
2022-08-23 13:15:31 +00:00
mutable std::mutex init_mutex;
mutable std::optional<bool> table_is_valid;
2022-08-10 08:52:36 +00:00
Poco::Logger * log;
2022-07-26 09:08:55 +00:00
};
}