Skip to content

Commit

Permalink
[BACKPORT 2.14][yugabyte#17713] docdb: Add master side migration for …
Browse files Browse the repository at this point in the history
…TableInfo namespace_name

Summary:
Original commit: 8afc03a / D27040
The namespace_name field was introduced in 2.3 but we didn't do any migration for it. This has led to issues on newer versions where newer code expects that this field is present, causing issues with ycql system.partitions, snapshots, metrics.

Using the master migration framework to migrate this field. As part of this, I have moved the namespace loader to be before the table loader (since that way we have the namespace info when loading tables).

Note: We still need to migrate the TableInfo stored in Raft Superblocks as well, but that will be done in a later diff, after we finalize the design there.
Jira: DB-6815

Test Plan:
```
ybd --cxx-test sys_catalog-test --gtest_filter SysCatalogTest.TestNamespaceNameMigration
ybd --cxx-test yb-admin-test_ent --gtest_filter AdminCliTest.TestListSnapshotWithNamespaceNameMigration
```

Reviewers: hsunder, zdrudi, asrivastava, skedia

Reviewed By: zdrudi

Subscribers: yql, yguan, ybase, bogdan

Tags: #jenkins-ready

Differential Revision: https://phorge.dev.yugabyte.com/D29560
  • Loading branch information
hulien22 committed Oct 24, 2023
1 parent 7f4b39d commit 531310d
Show file tree
Hide file tree
Showing 5 changed files with 157 additions and 4 deletions.
59 changes: 59 additions & 0 deletions ent/src/yb/tools/yb-admin-test_ent.cc
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
#include "yb/util/env_util.h"
#include "yb/util/monotime.h"
#include "yb/util/path_util.h"
#include "yb/util/pb_util.h"
#include "yb/util/subprocess.h"
#include "yb/util/test_util.h"
#include "yb/util/tsan_util.h"
Expand All @@ -49,6 +50,7 @@ DECLARE_uint64(TEST_yb_inbound_big_calls_parse_delay_ms);
DECLARE_int64(rpc_throttle_threshold_bytes);
DECLARE_bool(parallelize_bootstrap_producer);
DECLARE_bool(check_bootstrap_required);
DECLARE_bool(TEST_create_table_with_empty_namespace_name);

namespace yb {
namespace tools {
Expand Down Expand Up @@ -1503,5 +1505,62 @@ TEST_F(XClusterAdminCliTest_Large, TestBootstrapProducerPerformance) {
"Waiting for bootstrap_cdc_producer to complete"));
}

TEST_F(AdminCliTest, TestListSnapshotWithNamespaceNameMigration) {
// Start with a table missing its namespace_name (as if created before 2.3).
FLAGS_TEST_create_table_with_empty_namespace_name = true;
CreateTable(Transactional::kFalse);
FLAGS_TEST_create_table_with_empty_namespace_name = false;
const string& table_name = table_.name().table_name();
const string& keyspace = table_.name().namespace_name();

// Create snapshot of default table that gets created.
LOG(INFO) << ASSERT_RESULT(RunAdminToolCommand("create_snapshot", keyspace, table_name));

ListSnapshotsRequestPB req;
ListSnapshotsResponsePB resp;
RpcController rpc;
ASSERT_OK(ASSERT_RESULT(BackupServiceProxy())->ListSnapshots(req, &resp, &rpc));
ASSERT_EQ(resp.snapshots_size(), 1);
auto snapshot_id = resp.snapshots(0).id();
LOG(INFO) << "Created snapshot " << snapshot_id;
auto get_table_entry = [&]() -> Result<master::SysTablesEntryPB> {
for (auto& entry : resp.snapshots(0).entry().entries()) {
if (entry.type() == master::SysRowEntryType::TABLE) {
return pb_util::ParseFromSlice<master::SysTablesEntryPB>(entry.data());
}
}
return STATUS(NotFound, "Could not find TABLE entry");
};

// Old behaviour, snapshot doesn't have namespace_name.
master::SysTablesEntryPB table_meta = ASSERT_RESULT(get_table_entry());
ASSERT_FALSE(table_meta.has_namespace_name());

// Delete snapshot.
LOG(INFO) << ASSERT_RESULT(RunAdminToolCommand("delete_snapshot", snapshot_id));
ASSERT_OK(WaitFor([this]() -> Result<bool> {
ListSnapshotsRequestPB req;
ListSnapshotsResponsePB resp;
RpcController rpc;
RETURN_NOT_OK(VERIFY_RESULT(BackupServiceProxy())->ListSnapshots(req, &resp, &rpc));
return resp.snapshots_size() == 0;
}, 30s, "Complete delete snapshot"));

// Restart cluster, run namespace_name migration to populate the namespace_name field.
ASSERT_OK(cluster_->RestartSync());

// Create a new snapshot.
LOG(INFO) << ASSERT_RESULT(RunAdminToolCommand("create_snapshot", keyspace, table_name));

rpc.Reset();
ASSERT_OK(ASSERT_RESULT(BackupServiceProxy())->ListSnapshots(req, &resp, &rpc));
ASSERT_EQ(resp.snapshots_size(), 1);

// Ensure that the namespace_name field is now populated.
table_meta = ASSERT_RESULT(get_table_entry());
ASSERT_TRUE(table_meta.has_namespace_name());
ASSERT_EQ(table_meta.namespace_name(), keyspace);
}

} // namespace tools
} // namespace yb
22 changes: 22 additions & 0 deletions src/yb/master/catalog_loaders.cc
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,7 @@ Status TableLoader::Visit(const TableId& table_id, const SysTablesEntryPB& metad
CHECK(!ContainsKey(*catalog_manager_->table_ids_map_, table_id))
<< "Table already exists: " << table_id;

bool needs_async_write_to_sys_catalog = false;
// Setup the table info.
scoped_refptr<TableInfo> table = catalog_manager_->NewTableInfo(table_id);
auto l = table->LockForWrite();
Expand Down Expand Up @@ -106,6 +107,20 @@ Status TableLoader::Visit(const TableId& table_id, const SysTablesEntryPB& metad
}
}

// Backfill the SysTablesEntryPB namespace_name field.
if (pb.namespace_name().empty()) {
auto namespace_name = catalog_manager_->GetNamespaceNameUnlocked(pb.namespace_id());
if (!namespace_name.empty()) {
pb.set_namespace_name(namespace_name);
needs_async_write_to_sys_catalog = true;
LOG(INFO) << "Backfilling namespace_name " << namespace_name << " for table " << table_id;
} else {
LOG(WARNING) << Format(
"Could not find namespace name for table $0 with namespace id $1",
table_id, pb.namespace_id());
}
}

l.Commit();
catalog_manager_->HandleNewTableId(table->id());

Expand All @@ -123,6 +138,13 @@ Status TableLoader::Visit(const TableId& table_id, const SysTablesEntryPB& metad
"VerifyTransaction");
}

if (needs_async_write_to_sys_catalog) {
// Update the sys catalog asynchronously, so as to not block leader start up.
state_->AddPostLoadTask(
std::bind(&CatalogManager::WriteTableToSysCatalog, catalog_manager_, table_id),
"WriteTableToSysCatalog");
}

LOG(INFO) << "Loaded metadata for table " << table->ToString() << ", state: "
<< SysTablesEntryPB::State_Name(metadata.state());
VLOG(1) << "Metadata for table " << table->ToString() << ": " << metadata.ShortDebugString();
Expand Down
29 changes: 25 additions & 4 deletions src/yb/master/catalog_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -543,6 +543,9 @@ TAG_FLAG(enable_tablet_split_of_xcluster_bootstrapping_tables, runtime);
DEFINE_test_flag(bool, create_table_with_empty_pgschema_name, false,
"Create YSQL tables with an empty pgschema_name field in their schema.");

DEFINE_test_flag(bool, create_table_with_empty_namespace_name, false,
"Create YSQL tables with an empty namespace_name field in their schema.");

DEFINE_test_flag(int32, delay_split_registration_secs, 0,
"Delay creating child tablets and upserting them to sys catalog");

Expand Down Expand Up @@ -1283,9 +1286,9 @@ Status CatalogManager::RunLoaders(int64_t term, SysCatalogLoadingState* state) {
// Clear the hidden tablets vector.
hidden_tablets_.clear();

RETURN_NOT_OK(Load<NamespaceLoader>("namespaces", state, term));
RETURN_NOT_OK(Load<TableLoader>("tables", state, term));
RETURN_NOT_OK(Load<TabletLoader>("tablets", state, term));
RETURN_NOT_OK(Load<NamespaceLoader>("namespaces", state, term));
RETURN_NOT_OK(Load<UDTypeLoader>("user-defined types", state, term));
RETURN_NOT_OK(Load<ClusterConfigLoader>("cluster configuration", state, term));
RETURN_NOT_OK(Load<RedisConfigLoader>("Redis config", state, term));
Expand Down Expand Up @@ -1566,6 +1569,7 @@ Status CatalogManager::PrepareSysCatalogTable(int64_t term) {
SysTablesEntryPB& metadata = table->mutable_metadata()->mutable_dirty()->pb;
metadata.set_state(SysTablesEntryPB::RUNNING);
metadata.set_namespace_id(kSystemSchemaNamespaceId);
metadata.set_namespace_name(kSystemSchemaNamespaceName);
metadata.set_name(kSysCatalogTableName);
metadata.set_table_type(TableType::YQL_TABLE_TYPE);
SchemaToPB(sys_catalog_->schema(), metadata.mutable_schema());
Expand Down Expand Up @@ -4769,7 +4773,9 @@ scoped_refptr<TableInfo> CatalogManager::CreateTableInfo(const CreateTableReques
metadata->set_name(req.name());
metadata->set_table_type(req.table_type());
metadata->set_namespace_id(namespace_id);
metadata->set_namespace_name(namespace_name);
if (!FLAGS_TEST_create_table_with_empty_namespace_name) {
metadata->set_namespace_name(namespace_name);
}
metadata->set_version(0);
metadata->set_next_column_id(ColumnId(schema.max_col_id() + 1));
if (req.has_replication_info()) {
Expand Down Expand Up @@ -11825,14 +11831,29 @@ void CatalogManager::StartPostLoadTasks(const SysCatalogLoadingState& state) {
}
}

void CatalogManager::WriteTableToSysCatalog(const TableId& table_id) {
auto table_ptr = GetTableInfo(table_id);
if (!table_ptr) {
LOG_WITH_FUNC(WARNING) << Format("Could not find table $0 in table map.", table_id);
return;
}

LOG_WITH_FUNC(INFO) << Format(
"Writing table $0 to sys catalog as part of a migration.", table_id);
auto l = table_ptr->LockForWrite();
WARN_NOT_OK(sys_catalog_->ForceUpsert(leader_ready_term(), table_ptr),
"Failed to upsert migrated table into sys catalog.");
}

void CatalogManager::WriteTabletToSysCatalog(const TabletId& tablet_id) {
auto tablet_res = GetTabletInfo(tablet_id);
if (!tablet_res.ok()) {
LOG(WARNING) << Format("$0 could not find tablet $1 in tablet map.", __func__, tablet_id);
LOG_WITH_FUNC(WARNING) << Format("Could not find tablet $1 in tablet map.", tablet_id);
return;
}

LOG(INFO) << Format("Writing tablet $0 to sys catalog as part of a migration.", tablet_id);
LOG_WITH_FUNC(INFO) << Format(
"Writing tablet $0 to sys catalog as part of a migration.", tablet_id);
auto l = (*tablet_res)->LockForWrite();
WARN_NOT_OK(sys_catalog_->ForceUpsert(leader_ready_term(), *tablet_res),
"Failed to upsert migrated colocated tablet into sys catalog.");
Expand Down
3 changes: 3 additions & 0 deletions src/yb/master/catalog_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -955,6 +955,9 @@ class CatalogManager : public tserver::TabletPeerLookupIf,
virtual HybridTime AllowedHistoryCutoffProvider(tablet::RaftGroupMetadata* metadata) {
return HybridTime();
}

void WriteTableToSysCatalog(const TableId& table_id);

void WriteTabletToSysCatalog(const TabletId& tablet_id);

Result<boost::optional<ReplicationInfoPB>> GetTablespaceReplicationInfoWithRetry(
Expand Down
48 changes: 48 additions & 0 deletions src/yb/master/sys_catalog-test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -791,5 +791,53 @@ TEST_F(SysCatalogTest, TestCatalogManagerTasksTracker) {
table->AbortTasksAndClose();
}

// Test migration of the TableInfo namespace_name field.
TEST_F(SysCatalogTest, TestNamespaceNameMigration) {
// First create a new namespace to add our table to.
unique_ptr<TestNamespaceLoader> ns_loader(new TestNamespaceLoader());
ASSERT_OK(sys_catalog_->Visit(ns_loader.get()));
ASSERT_EQ(kNumSystemNamespaces, ns_loader->namespaces.size());

scoped_refptr<NamespaceInfo> ns(new NamespaceInfo("deadbeafdeadbeafdeadbeafdeadbeaf"));
{
auto l = ns->LockForWrite();
l.mutable_data()->pb.set_name("test_ns");
ASSERT_OK(sys_catalog_->Upsert(kLeaderTerm, ns));
l.Commit();
}

// Now create a new table and add it to that namespace.
unique_ptr<TestTableLoader> loader(new TestTableLoader());
ASSERT_OK(sys_catalog_->Visit(loader.get()));
ASSERT_EQ(kNumSystemTables, loader->tables.size());
const std::string table_id = "testtableid";
scoped_refptr<TableInfo> table = CreateUncommittedTable(table_id);

// Only set the namespace id and clear the namespace name for this table.
{
auto* metadata = &table->mutable_metadata()->mutable_dirty()->pb;
metadata->clear_namespace_name();
metadata->set_namespace_id(ns->id());
}

// Add the table.
ASSERT_OK(sys_catalog_->Upsert(kLeaderTerm, table));
table->mutable_metadata()->CommitMutation();

// Restart the cluster and wait for the background task to update the persistent state of the
// table's namespace_name.
ASSERT_OK(RestartMaster());
ASSERT_OK(WaitFor(
[&]() -> Result<bool> {
loader->Reset();
RETURN_NOT_OK(sys_catalog_->Visit(loader.get()));
auto in_mem_ns_name = master_->catalog_manager()->GetTableInfo(table_id)->namespace_name();
auto persisted_ns_name = loader->tables[table_id]->LockForRead()->namespace_name();
return !in_mem_ns_name.empty() && in_mem_ns_name == ns->name() &&
!persisted_ns_name.empty() && persisted_ns_name == ns->name();
},
10s * kTimeMultiplier, "Wait for table's namespace_name to be set in memory and on disk."));
}

} // namespace master
} // namespace yb

0 comments on commit 531310d

Please sign in to comment.