2022-07-26 09:08:55 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <Interpreters/Context.h>
|
2022-08-21 17:09:36 +00:00
|
|
|
#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,
|
2022-08-10 07:24:56 +00:00
|
|
|
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;
|
2022-08-10 07:24:56 +00:00
|
|
|
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
|
|
|
|
2022-08-21 17:09:36 +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
|
|
|
|
2022-08-21 17:09:36 +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
|
|
|
};
|
|
|
|
|
|
|
|
}
|