This commit is contained in:
Antonio Andelic 2024-07-31 14:09:07 +02:00
parent e65cdcea1b
commit 6b7c5eb5da
3 changed files with 41 additions and 19 deletions

View File

@ -79,6 +79,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
extern const int LIMIT_EXCEEDED;
extern const int CANNOT_RESTORE_TABLE;
extern const int INVALID_STATE;
}
namespace
@ -497,7 +498,7 @@ StorageKeeperMap::StorageKeeperMap(
}
table_is_valid = true;
table_status = TableStatus::VALID;
/// we are the first table created for the specified Keeper path, i.e. we are the first replica
return;
}
@ -656,7 +657,18 @@ bool StorageKeeperMap::dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::E
void StorageKeeperMap::drop()
{
checkTable<true>();
auto current_table_status = getTableStatus();
if (current_table_status == TableStatus::UNKNOWN)
{
static constexpr auto error_msg = "Failed to activate table because of connection issues. It will be activated "
"once a connection is established and metadata is verified";
throw Exception(ErrorCodes::INVALID_STATE, error_msg);
}
/// if only column metadata is wrong we can still drop the table correctly
if (current_table_status == TableStatus::INVALID_KEEPER_STRUCTURE)
return;
auto client = getClient();
// we allow ZNONODE in case we got hardware error on previous drop
@ -1017,11 +1029,11 @@ UInt64 StorageKeeperMap::keysLimit() const
return keys_limit;
}
std::optional<bool> StorageKeeperMap::isTableValid() const
StorageKeeperMap::TableStatus StorageKeeperMap::getTableStatus() const
{
std::lock_guard lock{init_mutex};
if (table_is_valid.has_value())
return table_is_valid;
if (table_status != TableStatus::UNKNOWN)
return table_status;
[&]
{
@ -1034,7 +1046,7 @@ std::optional<bool> StorageKeeperMap::isTableValid() const
if (metadata_stat.numChildren == 0)
{
table_is_valid = false;
table_status = TableStatus::INVALID_KEEPER_STRUCTURE;
return;
}
@ -1045,7 +1057,7 @@ std::optional<bool> StorageKeeperMap::isTableValid() const
"Table definition does not match to the one stored in the path {}. Stored definition: {}",
zk_root_path,
stored_metadata_string);
table_is_valid = false;
table_status = TableStatus::INVALID_METADATA;
return;
}
@ -1058,7 +1070,7 @@ std::optional<bool> StorageKeeperMap::isTableValid() const
Coordination::Responses responses;
client->tryMulti(requests, responses);
table_is_valid = false;
table_status = TableStatus::INVALID_KEEPER_STRUCTURE;
if (responses[0]->error != Coordination::Error::ZOK)
{
LOG_ERROR(log, "Table node ({}) is missing", zk_table_path);
@ -1077,18 +1089,18 @@ std::optional<bool> StorageKeeperMap::isTableValid() const
return;
}
table_is_valid = true;
table_status = TableStatus::VALID;
}
catch (const Coordination::Exception & e)
{
tryLogCurrentException(log);
if (!Coordination::isHardwareError(e.code))
table_is_valid = false;
table_status = TableStatus::INVALID_KEEPER_STRUCTURE;
}
}();
return table_is_valid;
return table_status;
}
Chunk StorageKeeperMap::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPODArray<UInt8> & null_map, const Names &) const

View File

@ -80,8 +80,8 @@ public:
template <bool throw_on_error>
void checkTable() const
{
auto is_table_valid = isTableValid();
if (!is_table_valid.has_value())
auto current_table_status = getTableStatus();
if (table_status == TableStatus::UNKNOWN)
{
static constexpr auto error_msg = "Failed to activate table because of connection issues. It will be activated "
"once a connection is established and metadata is verified";
@ -94,10 +94,10 @@ public:
}
}
if (!*is_table_valid)
if (current_table_status != TableStatus::VALID)
{
static constexpr auto error_msg
= "Failed to activate table because of invalid metadata in ZooKeeper. Please DETACH table";
= "Failed to activate table because of invalid metadata in ZooKeeper. Please DROP/DETACH table";
if constexpr (throw_on_error)
throw Exception(ErrorCodes::INVALID_STATE, error_msg);
else
@ -111,7 +111,15 @@ public:
private:
bool dropTable(zkutil::ZooKeeperPtr zookeeper, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock);
std::optional<bool> isTableValid() const;
enum class TableStatus : uint8_t
{
UNKNOWN,
INVALID_METADATA,
INVALID_KEEPER_STRUCTURE,
VALID
};
TableStatus getTableStatus() const;
void restoreDataImpl(
const BackupPtr & backup,
@ -143,7 +151,8 @@ private:
mutable zkutil::ZooKeeperPtr zookeeper_client{nullptr};
mutable std::mutex init_mutex;
mutable std::optional<bool> table_is_valid;
mutable TableStatus table_status{TableStatus::UNKNOWN};
LoggerPtr log;
};

View File

@ -67,6 +67,7 @@ def run_query(query):
def test_keeper_map_without_zk(started_cluster):
run_query("DROP TABLE IF EXISTS test_keeper_map_without_zk SYNC")
assert_keeper_exception_after_partition(
"CREATE TABLE test_keeper_map_without_zk (key UInt64, value UInt64) ENGINE = KeeperMap('/test_keeper_map_without_zk') PRIMARY KEY(key);"
)
@ -107,12 +108,12 @@ def test_keeper_map_without_zk(started_cluster):
)
assert "Failed to activate table because of invalid metadata in ZooKeeper" in error
node.query("DETACH TABLE test_keeper_map_without_zk")
client.stop()
def test_keeper_map_with_failed_drop(started_cluster):
run_query("DROP TABLE IF EXISTS test_keeper_map_with_failed_drop SYNC")
run_query("DROP TABLE IF EXISTS test_keeper_map_with_failed_drop_another SYNC")
run_query(
"CREATE TABLE test_keeper_map_with_failed_drop (key UInt64, value UInt64) ENGINE = KeeperMap('/test_keeper_map_with_failed_drop') PRIMARY KEY(key);"
)