tablets, mv: create tablets for a new materialized view

Before this patch, trying to create a materialized view when tablets
are enabled for a keyspace results in a failure: "Tablet map not found
for table <uuid>", with uuid referring to the new view.

When a table schema is created, the handler on_before_create_column_family()
is called - and this function creates the tablet map for the new table.
The bug was that we forgot to do the same when creating a materialized
view - which also a bona-fide table.

In this patch we call on_before_create_column_family() also when
creating the materialized view. I decided *not* to create a new
callback (e.g., on_before_create_view()) and rather call the existing
on_before_create_column_family() callback - after all, a view is
a column family too.

This patch also includes a test for this issue, which fails to create
the view before this patch, and passes with the patch. The test is
in the test/topology_experimental_raft suite, which runs Scylla with
the tablets experimental feature, and will also allow me to create
tests that need multiple nodes. However, the first test added here
only needs a single node to reproduce the bug and validate its fix.

Fixes #16194.

Signed-off-by: Nadav Har'El <nyh@scylladb.com>

Closes scylladb/scylladb#16205
This commit is contained in:
Nadav Har'El
2023-11-28 16:24:07 +02:00
committed by Tomasz Grabiec
parent 3582095b79
commit 88a5ddabce
2 changed files with 44 additions and 5 deletions

View File

@@ -857,16 +857,23 @@ future<std::vector<mutation>> prepare_new_view_announcement(storage_proxy& sp, v
#endif
auto& db = sp.local_db();
try {
auto&& keyspace = db.find_keyspace(view->ks_name()).metadata();
auto keyspace = db.find_keyspace(view->ks_name()).metadata();
if (keyspace->cf_meta_data().contains(view->cf_name())) {
throw exceptions::already_exists_exception(view->ks_name(), view->cf_name());
}
mlogger.info("Create new view: {}", view);
auto mutations = db::schema_tables::make_create_view_mutations(keyspace, std::move(view), ts);
co_return co_await include_keyspace(sp, *keyspace, std::move(mutations));
return seastar::async([&db, keyspace = std::move(keyspace), &sp, view = std::move(view), ts] {
auto mutations = db::schema_tables::make_create_view_mutations(keyspace, std::move(view), ts);
// We don't have a separate on_before_create_view() listener to
// call. But a view is also a column family, and we need to call
// the on_before_create_column_family listener - notably, to
// create tablets for the new view table.
db.get_notifier().before_create_column_family(*keyspace, *view, mutations, ts);
return include_keyspace(sp, *keyspace, std::move(mutations)).get();
});
} catch (const replica::no_such_keyspace& e) {
auto&& ex = std::make_exception_ptr(exceptions::configuration_exception(format("Cannot add view '{}' to non existing keyspace '{}'.", view->cf_name(), view->ks_name())));
co_return coroutine::exception(std::move(ex));
return make_exception_future<std::vector<mutation>>(
exceptions::configuration_exception(format("Cannot add view '{}' to non existing keyspace '{}'.", view->cf_name(), view->ks_name())));
}
}

View File

@@ -0,0 +1,32 @@
#
# Copyright (C) 2023-present ScyllaDB
#
# SPDX-License-Identifier: AGPL-3.0-or-later
#
# Tests for interaction of materialized views with *tablets*
from test.pylib.manager_client import ManagerClient
import pytest
import asyncio
import logging
logger = logging.getLogger(__name__)
@pytest.mark.asyncio
async def test_tablet_mv_create(manager: ManagerClient):
"""A basic test for creating a materialized view on a table stored
with tablets on a one-node cluster. We just create the view and
delete it - that's it, we don't read or write the table.
Reproduces issue #16194.
"""
servers = [await manager.server_add() for i in range(1)]
cql = manager.get_cql()
await cql.run_async("CREATE KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1, 'initial_tablets': 100}")
await cql.run_async("CREATE TABLE test.test (pk int PRIMARY KEY, c int)")
await cql.run_async("CREATE MATERIALIZED VIEW test.tv AS SELECT * FROM test.test WHERE c IS NOT NULL AND pk IS NOT NULL PRIMARY KEY (c, pk)")
await cql.run_async("DROP KEYSPACE test")