From e1c3dd75b818147b545f9a46acfcf3fc19291473 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 12 Feb 2024 13:25:54 -0800 Subject: [PATCH] [nexus] put DNS servers in DNS, so you can DNS while you DNS (#5033) Currently, the DNS propagation background task in Nexus uses the `services` table to enumerate the list of DNS servers that it's responsible for keeping up to date. However, we'd really like to get rid of said `services` table (see #4947), and the DNS propagation code is the only remaining user of the `services` table. Therefore, this branch changes how DNS servers are discovered for DNS propagation. Rather than discovering DNS server addresses from the `services` table, the `DnsWatcher` background task now discovers DNS servers...using internal DNS. As described in #4889, this may _seem_ like a cyclical dependency, but, because the initial set of internal DNS servers operate at known addresses -- by design -- so that they can always be discovered. And they have to be up and discoverable for Nexus to even come up and find CockroachDB. So, internal DNS can safely be assumed to be up if Nexus has come up at all. Now, the `services` table is no longer used, and This change breaks the existing tests `nexus::app::background::init::test_dns_propagation_basic` and `nexus::app::background::dns_servers::test_basic`. I've rewritten the `test_dns_propagation_basic` test to test the new expected behavior: - creating a new internal DNS server and adding a DNS record for it to the database's DNS config table results in that server's DNS records being propagated to the existing DNS serve - the `DnsWatcher` background task then picks up the DNS records for the new DNS server by querying the existing known DNS server - the current DNS config generation is then propagated to the new DNS server - a subsequent generation is propagated to both the initial and new DNS servers The `dns_servers::test_basic` test tested the discovery of DNS server addresses from the database. Because these no longer come from the db, and now come from internal DNS, this test would now end up exercising most of the functionality tested in `test_dns_propagation_basic`. I didn't think it was necessary to have two tests for this, so I made the judgement call to delete `dns_servers::test_basic`. If we think having a more isolated test that exercises only the DNS watcher task and not the DNS propagation task, we could put this back and create records on the DNS server by manually hitting its API with new configs, but I didn't think this was really worth the effort. I've also removed the `Datastore::upsert_service` method, which was used only for test code and is now dead. I considered deleting all code related to querying the `services` table in this branch as well. However, I noticed that it's still populated when initializing the rack, and that `omdb` has commands for querying that table. I wasn't sure if there were alternative data sources for the `omdb` debugging commands yet, so I didn't remove them. If the data provided by those commands is available elsewhere, or if their only real purpose is _just_ to print the state of this table, I'm happy to delete them in this branch, as well. Closes #4889 ![image](https://github.com/oxidecomputer/omicron/assets/2796466/c37a0d31-26f7-4a5d-9748-ef7212cde9a9) --- dev-tools/omdb/tests/env.out | 12 +- dev-tools/omdb/tests/successes.out | 8 +- internal-dns/src/names.rs | 2 +- nexus/src/app/background/dns_config.rs | 5 +- nexus/src/app/background/dns_servers.rs | 214 ++++------------------- nexus/src/app/background/init.rs | 223 +++++++++++++----------- nexus/src/app/sled.rs | 38 ---- 7 files changed, 168 insertions(+), 334 deletions(-) diff --git a/dev-tools/omdb/tests/env.out b/dev-tools/omdb/tests/env.out index 0600945194..3e6e89d508 100644 --- a/dev-tools/omdb/tests/env.out +++ b/dev-tools/omdb/tests/env.out @@ -57,11 +57,11 @@ task: "dns_propagation_internal" task: "dns_servers_external" - watches list of external DNS servers stored in CockroachDB + watches list of external DNS servers stored in internal DNS task: "dns_servers_internal" - watches list of internal DNS servers stored in CockroachDB + watches list of internal DNS servers stored in internal DNS task: "external_endpoints" @@ -147,11 +147,11 @@ task: "dns_propagation_internal" task: "dns_servers_external" - watches list of external DNS servers stored in CockroachDB + watches list of external DNS servers stored in internal DNS task: "dns_servers_internal" - watches list of internal DNS servers stored in CockroachDB + watches list of internal DNS servers stored in internal DNS task: "external_endpoints" @@ -224,11 +224,11 @@ task: "dns_propagation_internal" task: "dns_servers_external" - watches list of external DNS servers stored in CockroachDB + watches list of external DNS servers stored in internal DNS task: "dns_servers_internal" - watches list of internal DNS servers stored in CockroachDB + watches list of internal DNS servers stored in internal DNS task: "external_endpoints" diff --git a/dev-tools/omdb/tests/successes.out b/dev-tools/omdb/tests/successes.out index 1cd85262f6..3086c98f32 100644 --- a/dev-tools/omdb/tests/successes.out +++ b/dev-tools/omdb/tests/successes.out @@ -251,11 +251,11 @@ task: "dns_propagation_internal" task: "dns_servers_external" - watches list of external DNS servers stored in CockroachDB + watches list of external DNS servers stored in internal DNS task: "dns_servers_internal" - watches list of internal DNS servers stored in CockroachDB + watches list of internal DNS servers stored in internal DNS task: "external_endpoints" @@ -313,7 +313,7 @@ task: "dns_servers_internal" task: "dns_propagation_internal" configured period: every 1m currently executing: no - last completed activation: iter 5, triggered by a dependent task completing + last completed activation: iter 4, triggered by a dependent task completing started at (s ago) and ran for ms attempt to propagate generation: 1 @@ -341,7 +341,7 @@ task: "dns_servers_external" task: "dns_propagation_external" configured period: every 1m currently executing: no - last completed activation: iter 5, triggered by a dependent task completing + last completed activation: iter 4, triggered by a dependent task completing started at (s ago) and ran for ms attempt to propagate generation: 2 diff --git a/internal-dns/src/names.rs b/internal-dns/src/names.rs index e0c9b79555..bffe6e829a 100644 --- a/internal-dns/src/names.rs +++ b/internal-dns/src/names.rs @@ -65,7 +65,7 @@ impl ServiceName { /// Returns the DNS name for this service, ignoring the zone part of the DNS /// name - pub(crate) fn dns_name(&self) -> String { + pub fn dns_name(&self) -> String { match self { ServiceName::Clickhouse | ServiceName::ClickhouseKeeper diff --git a/nexus/src/app/background/dns_config.rs b/nexus/src/app/background/dns_config.rs index 959cf1843e..be18ac3612 100644 --- a/nexus/src/app/background/dns_config.rs +++ b/nexus/src/app/background/dns_config.rs @@ -159,7 +159,6 @@ impl BackgroundTask for DnsConfigWatcher { mod test { use crate::app::background::common::BackgroundTask; use crate::app::background::dns_config::DnsConfigWatcher; - use crate::app::background::init::test::read_internal_dns_zone_id; use crate::app::background::init::test::write_test_dns_generation; use assert_matches::assert_matches; use async_bb8_diesel::AsyncRunQueryDsl; @@ -197,9 +196,7 @@ mod test { // Now write generation 2, activate again, and verify that the update // was sent to the watcher. - let internal_dns_zone_id = - read_internal_dns_zone_id(&opctx, &datastore).await; - write_test_dns_generation(&datastore, internal_dns_zone_id).await; + write_test_dns_generation(&opctx, &datastore).await; assert_eq!(watcher.borrow().as_ref().unwrap().generation, 1); let value = task.activate(&opctx).await; assert_eq!(watcher.borrow().as_ref().unwrap().generation, 2); diff --git a/nexus/src/app/background/dns_servers.rs b/nexus/src/app/background/dns_servers.rs index 97fb3510b7..8f4cce4ee0 100644 --- a/nexus/src/app/background/dns_servers.rs +++ b/nexus/src/app/background/dns_servers.rs @@ -7,22 +7,15 @@ use super::common::BackgroundTask; use futures::future::BoxFuture; use futures::FutureExt; +use internal_dns::names::ServiceName; +use internal_dns::resolver::Resolver; use nexus_db_model::DnsGroup; -use nexus_db_model::ServiceKind; use nexus_db_queries::context::OpContext; -use nexus_db_queries::db::DataStore; -use omicron_common::api::external::DataPageParams; use serde::Serialize; use serde_json::json; -use std::net::{SocketAddr, SocketAddrV6}; -use std::num::NonZeroU32; -use std::sync::Arc; +use std::net::SocketAddr; use tokio::sync::watch; -// This constraint could be relaxed by paginating through the list of servers, -// but we don't expect to have this many servers any time soon. -const MAX_DNS_SERVERS: usize = 10; - #[derive(Debug, Clone, Eq, PartialEq, Serialize)] pub struct DnsServersList { pub addresses: Vec, @@ -31,20 +24,17 @@ pub struct DnsServersList { /// Background task that keeps track of the latest list of DNS servers for a DNS /// group pub struct DnsServersWatcher { - datastore: Arc, dns_group: DnsGroup, + resolver: Resolver, last: Option, tx: watch::Sender>, rx: watch::Receiver>, } impl DnsServersWatcher { - pub fn new( - datastore: Arc, - dns_group: DnsGroup, - ) -> DnsServersWatcher { + pub fn new(dns_group: DnsGroup, resolver: Resolver) -> DnsServersWatcher { let (tx, rx) = watch::channel(None); - DnsServersWatcher { datastore, dns_group, last: None, tx, rx } + DnsServersWatcher { dns_group, last: None, tx, rx, resolver } } /// Exposes the latest list of DNS servers for this DNS group @@ -75,58 +65,38 @@ impl BackgroundTask for DnsServersWatcher { }; // Read the latest service configuration for this DNS group. - let service_kind = match self.dns_group { - DnsGroup::Internal => ServiceKind::InternalDns, - DnsGroup::External => ServiceKind::ExternalDns, + let service_name = match self.dns_group { + DnsGroup::Internal => ServiceName::InternalDns, + DnsGroup::External => ServiceName::ExternalDns, }; - let pagparams = DataPageParams { - marker: None, - limit: NonZeroU32::try_from( - u32::try_from(MAX_DNS_SERVERS).unwrap(), - ) - .unwrap(), - direction: dropshot::PaginationOrder::Ascending, + let result = self.resolver.lookup_all_socket_v6(service_name).await; + let addresses = match result { + Err(error) => { + warn!( + &log, + "failed to lookup DNS servers"; + "error" => format!("{:#}", error) + ); + return json!({ + "error": + format!( + "failed to read list of DNS servers: {:#}", + error + ) + }); + } + Ok(addresses) => { + // TODO(eliza): it would be nicer if `Resolver` had a method + // returning an iterator instead of a `Vec`, so we didn't + // have to drain the Vec and then collect it into a new + // one... + addresses.into_iter().map(SocketAddr::V6).collect() + } }; - let result = self - .datastore - .services_list_kind(opctx, service_kind, &pagparams) - .await; - - if let Err(error) = result { - warn!( - &log, - "failed to read list of DNS servers"; - "error" => format!("{:#}", error) - ); - return json!({ - "error": - format!( - "failed to read list of DNS servers: {:#}", - error - ) - }); - } - - let services = result.unwrap(); - if services.len() >= MAX_DNS_SERVERS { - warn!( - &log, - "found {} servers, which is more than MAX_DNS_SERVERS \ - ({}). There may be more that will not be used.", - services.len(), - MAX_DNS_SERVERS - ); - } - - let new_config = DnsServersList { - addresses: services - .into_iter() - .map(|s| SocketAddrV6::new(*s.ip, *s.port, 0, 0).into()) - .collect(), - }; - let new_addrs_dbg = format!("{:?}", new_config); + let new_config = DnsServersList { addresses }; + let new_addrs_dbg = format!("{new_config:?}"); let rv = serde_json::to_value(&new_config).unwrap_or_else(|error| { json!({ @@ -177,119 +147,3 @@ impl BackgroundTask for DnsServersWatcher { .boxed() } } - -#[cfg(test)] -mod test { - use crate::app::background::common::BackgroundTask; - use crate::app::background::dns_servers::DnsServersList; - use crate::app::background::dns_servers::DnsServersWatcher; - use crate::app::background::dns_servers::MAX_DNS_SERVERS; - use assert_matches::assert_matches; - use async_bb8_diesel::AsyncRunQueryDsl; - use diesel::ExpressionMethods; - use diesel::QueryDsl; - use nexus_db_model::DnsGroup; - use nexus_db_queries::context::OpContext; - use nexus_db_queries::db::model::Service; - use nexus_db_queries::db::model::ServiceKind; - use nexus_test_utils_macros::nexus_test; - use std::net::Ipv6Addr; - use std::net::SocketAddrV6; - use uuid::Uuid; - - type ControlPlaneTestContext = - nexus_test_utils::ControlPlaneTestContext; - - #[nexus_test(server = crate::Server)] - async fn test_basic(cptestctx: &ControlPlaneTestContext) { - let nexus = &cptestctx.server.apictx().nexus; - let datastore = nexus.datastore(); - let opctx = OpContext::for_tests( - cptestctx.logctx.log.clone(), - datastore.clone(), - ); - - // Verify the initial state. - let mut task = - DnsServersWatcher::new(datastore.clone(), DnsGroup::Internal); - let watcher = task.watcher(); - assert_matches!(*watcher.borrow(), None); - - // The datastore from the ControlPlaneTestContext is initialized with - // one DNS server. - let _ = task.activate(&opctx).await; - assert_matches!(*watcher.borrow(), Some(DnsServersList { - ref addresses - }) if addresses.len() == 1); - - // If we add another server, we should see it. - { - use nexus_db_queries::db::schema::service::dsl; - diesel::insert_into(dsl::service) - .values(Service::new( - Uuid::new_v4(), - Uuid::new_v4(), - Some(Uuid::new_v4()), - SocketAddrV6::new(Ipv6Addr::LOCALHOST, 1, 0, 0), - ServiceKind::InternalDns, - )) - .execute_async( - &*datastore.pool_connection_for_tests().await.unwrap(), - ) - .await - .unwrap(); - } - - let _ = task.activate(&opctx).await; - assert_matches!(*watcher.borrow(), Some(DnsServersList { - ref addresses - }) if addresses.len() == 2); - - // If we add MAX_DNS_SERVERS more servers, we should see - // MAX_DNS_SERVERS. - { - use nexus_db_queries::db::schema::service::dsl; - let new_services = (0..u16::try_from(MAX_DNS_SERVERS).unwrap()) - .map(|i| { - Service::new( - Uuid::new_v4(), - Uuid::new_v4(), - Some(Uuid::new_v4()), - SocketAddrV6::new(Ipv6Addr::LOCALHOST, i + 2, 0, 0), - ServiceKind::InternalDns, - ) - }) - .collect::>(); - - diesel::insert_into(dsl::service) - .values(new_services) - .execute_async( - &*datastore.pool_connection_for_tests().await.unwrap(), - ) - .await - .unwrap(); - } - - let _ = task.activate(&opctx).await; - assert_matches!(*watcher.borrow(), Some(DnsServersList { - ref addresses - }) if addresses.len() == MAX_DNS_SERVERS); - - // Now delete all the servers and try again. - { - use nexus_db_queries::db::schema::service::dsl; - diesel::delete( - dsl::service.filter(dsl::kind.eq(ServiceKind::InternalDns)), - ) - .execute_async( - &*datastore.pool_connection_for_tests().await.unwrap(), - ) - .await - .unwrap(); - } - let _ = task.activate(&opctx).await; - assert_matches!(*watcher.borrow(), Some(DnsServersList { - ref addresses - }) if addresses.is_empty()); - } -} diff --git a/nexus/src/app/background/init.rs b/nexus/src/app/background/init.rs index 9d078f10d0..27e58a298c 100644 --- a/nexus/src/app/background/init.rs +++ b/nexus/src/app/background/init.rs @@ -101,6 +101,7 @@ impl BackgroundTasks { opctx, datastore.clone(), DnsGroup::Internal, + resolver.clone(), &config.dns_internal, ); let (task_external_dns_config, task_external_dns_servers) = init_dns( @@ -108,6 +109,7 @@ impl BackgroundTasks { opctx, datastore.clone(), DnsGroup::External, + resolver.clone(), &config.dns_external, ); @@ -301,6 +303,7 @@ fn init_dns( opctx: &OpContext, datastore: Arc, dns_group: DnsGroup, + resolver: internal_dns::resolver::Resolver, config: &DnsTasksConfig, ) -> (common::TaskHandle, common::TaskHandle) { let dns_group_name = dns_group.to_string(); @@ -321,13 +324,13 @@ fn init_dns( ); // Background task: DNS server list watcher - let dns_servers = dns_servers::DnsServersWatcher::new(datastore, dns_group); + let dns_servers = dns_servers::DnsServersWatcher::new(dns_group, resolver); let dns_servers_watcher = dns_servers.watcher(); let task_name_servers = format!("dns_servers_{}", dns_group); let task_servers = driver.register( task_name_servers.clone(), format!( - "watches list of {} DNS servers stored in CockroachDB", + "watches list of {} DNS servers stored in internal DNS", dns_group, ), config.period_secs_servers, @@ -361,22 +364,17 @@ fn init_dns( #[cfg(test)] pub mod test { - use async_bb8_diesel::AsyncRunQueryDsl; use dropshot::HandlerTaskMode; use nexus_db_model::DnsGroup; - use nexus_db_model::Generation; use nexus_db_queries::context::OpContext; + use nexus_db_queries::db::datastore::DnsVersionUpdateBuilder; use nexus_db_queries::db::DataStore; use nexus_test_utils_macros::nexus_test; use nexus_types::internal_api::params as nexus_params; - use nexus_types::internal_api::params::ServiceKind; - use omicron_common::api::external::DataPageParams; use omicron_test_utils::dev::poll; use std::net::SocketAddr; - use std::num::NonZeroU32; use std::time::Duration; use tempfile::TempDir; - use uuid::Uuid; type ControlPlaneTestContext = nexus_test_utils::ControlPlaneTestContext; @@ -423,12 +421,23 @@ pub mod test { .expect("failed to get initial DNS server config"); assert_eq!(config.generation, 1); - // We'll need the id of the internal DNS zone. - let internal_dns_zone_id = - read_internal_dns_zone_id(&opctx, datastore).await; + let internal_dns_srv_name = + internal_dns::ServiceName::InternalDns.dns_name(); + + let initial_srv_record = { + let zone = + config.zones.get(0).expect("DNS config must have a zone"); + let Some(record) = zone.records.get(&internal_dns_srv_name) else { + panic!("zone must have a record for {internal_dns_srv_name}") + }; + match record.get(0) { + Some(dns_service_client::types::DnsRecord::Srv(srv)) => srv, + record => panic!("expected a SRV record for {internal_dns_srv_name}, found {record:?}"), + } + }; // Now spin up another DNS server, add it to the list of servers, and - // make sure that DNS gets propagated to it. Note that we shouldn't + // make sure that DNS gets propagated to it. Note that we shouldn't // have to explicitly activate the background task because inserting a // new service ought to do that for us. let log = &cptestctx.logctx.log; @@ -467,29 +476,76 @@ pub mod test { SocketAddr::V4(_) => panic!("expected v6 address"), SocketAddr::V6(a) => a, }; + + // In order to test that DNS gets propagated to a newly-added server, we + // first need to update the source of truth about DNS (the database). + // Then we need to wait for that to get propagated (by this same + // mechanism) to the existing DNS servers. Only then would we expect + // the mechanism to see the new DNS server and then propagate + // configuration to it. + let update = { + use nexus_params::{DnsRecord, Srv}; + + let target = "my-great-dns-server.host"; + + let mut update = test_dns_update_builder(); + update.remove_name(internal_dns_srv_name.clone()).unwrap(); + update + .add_name( + internal_dns_srv_name, + vec![ + DnsRecord::Srv(Srv { + prio: 0, + weight: 0, + port: new_dns_addr.port(), + target: format!( + "{target}.control-plane.oxide.internal" + ), + }), + DnsRecord::Srv(initial_srv_record.clone()), + ], + ) + .unwrap(); + update + .add_name( + target.to_string(), + vec![DnsRecord::Aaaa(*new_dns_addr.ip())], + ) + .unwrap(); + update + }; + write_dns_update(&opctx, datastore, update).await; + info!(&cptestctx.logctx.log, "updated new dns records"); + + // Activate the internal DNS propagation pipeline. nexus - .upsert_service( - &opctx, - Uuid::new_v4(), - cptestctx.sled_agent.sled_agent.id, - Some(Uuid::new_v4()), - new_dns_addr, - ServiceKind::InternalDns.into(), - ) - .await - .unwrap(); + .background_tasks + .activate(&nexus.background_tasks.task_internal_dns_config); + + wait_propagate_dns( + &cptestctx.logctx.log, + "initial", + initial_dns_dropshot_server.local_addr(), + 2, + ) + .await; + + // Discover the new internal DNS server from internal DNS. + nexus + .background_tasks + .activate(&nexus.background_tasks.task_internal_dns_servers); wait_propagate_dns( &cptestctx.logctx.log, "new", new_dns_dropshot_server.local_addr(), - 1, + 2, ) .await; - // Now, write version 2 of the internal DNS configuration with one + // Now, write version 3 of the internal DNS configuration with one // additional record. - write_test_dns_generation(datastore, internal_dns_zone_id).await; + write_test_dns_generation(&opctx, datastore).await; // Activate the internal DNS propagation pipeline. nexus @@ -501,7 +557,7 @@ pub mod test { &cptestctx.logctx.log, "initial", initial_dns_dropshot_server.local_addr(), - 2, + 3, ) .await; @@ -509,7 +565,7 @@ pub mod test { &cptestctx.logctx.log, "new", new_dns_dropshot_server.local_addr(), - 2, + 3, ) .await; } @@ -522,15 +578,16 @@ pub mod test { generation: u64, ) { println!( - "waiting for propagation of generation {} to {} DNS server ({})", - generation, label, addr + "waiting for propagation of generation {generation} to {label} \ + DNS server ({addr})", ); let client = dns_service_client::Client::new( &format!("http://{}", addr), log.clone(), ); - poll::wait_for_condition( + let poll_max = Duration::from_secs(30); + let result = poll::wait_for_condition( || async { match client.dns_config_get().await { Err(error) => { @@ -548,87 +605,51 @@ pub mod test { } }, &Duration::from_millis(50), - &Duration::from_secs(30), + &poll_max, ) - .await - .expect("DNS config not propagated in expected time"); + .await; + if let Err(err) = result { + panic!( + "DNS generation {generation} not propagated to \ + {label} DNS server ({addr}) within {poll_max:?}: {err}" + ); + } else { + println!( + "DNS generation {generation} propagated to {label} \ + DNS server ({addr}) successfully." + ); + } } - pub(crate) async fn write_test_dns_generation( + pub(crate) async fn write_dns_update( + opctx: &OpContext, datastore: &DataStore, - internal_dns_zone_id: Uuid, + update: DnsVersionUpdateBuilder, ) { - { - let conn = datastore.pool_connection_for_tests().await.unwrap(); - let _: Result<(), _> = datastore - .transaction_retry_wrapper("write_test_dns_generation") - .transaction(&conn, |conn| async move { - { - use nexus_db_queries::db::model::DnsVersion; - use nexus_db_queries::db::schema::dns_version::dsl; - - diesel::insert_into(dsl::dns_version) - .values(DnsVersion { - dns_group: DnsGroup::Internal, - version: Generation(2u32.try_into().unwrap()), - time_created: chrono::Utc::now(), - creator: String::from("test suite"), - comment: String::from("test suite"), - }) - .execute_async(&conn) - .await - .unwrap(); - } - - { - use nexus_db_queries::db::model::DnsName; - use nexus_db_queries::db::schema::dns_name::dsl; - - diesel::insert_into(dsl::dns_name) - .values( - DnsName::new( - internal_dns_zone_id, - String::from("we-got-beets"), - Generation(2u32.try_into().unwrap()), - None, - vec![nexus_params::DnsRecord::Aaaa( - "fe80::3".parse().unwrap(), - )], - ) - .unwrap(), - ) - .execute_async(&conn) - .await - .unwrap(); - } - - Ok(()) - }) - .await; - } + let conn = datastore.pool_connection_for_tests().await.unwrap(); + info!(opctx.log, "writing DNS update..."); + datastore.dns_update(opctx, &conn, update).await.unwrap(); } - pub(crate) async fn read_internal_dns_zone_id( + pub(crate) async fn write_test_dns_generation( opctx: &OpContext, datastore: &DataStore, - ) -> Uuid { - let dns_zones = datastore - .dns_zones_list( - &opctx, - DnsGroup::Internal, - &DataPageParams { - marker: None, - direction: dropshot::PaginationOrder::Ascending, - limit: NonZeroU32::new(2).unwrap(), - }, + ) { + let mut update = test_dns_update_builder(); + update + .add_name( + "we-got-beets".to_string(), + vec![nexus_params::DnsRecord::Aaaa("fe80::3".parse().unwrap())], ) - .await .unwrap(); - assert_eq!( - dns_zones.len(), - 1, - "expected exactly one internal DNS zone" - ); - dns_zones[0].id + write_dns_update(opctx, datastore, update).await + } + + fn test_dns_update_builder() -> DnsVersionUpdateBuilder { + DnsVersionUpdateBuilder::new( + DnsGroup::Internal, + "test suite DNS update".to_string(), + "test suite".to_string(), + ) } } diff --git a/nexus/src/app/sled.rs b/nexus/src/app/sled.rs index 943490ac04..738dae1d6d 100644 --- a/nexus/src/app/sled.rs +++ b/nexus/src/app/sled.rs @@ -23,9 +23,6 @@ use std::net::SocketAddrV6; use std::sync::Arc; use uuid::Uuid; -#[cfg(test)] -use nexus_db_queries::db::model::ServiceKind; - impl super::Nexus { // Sleds pub fn sled_lookup<'a>( @@ -276,41 +273,6 @@ impl super::Nexus { Ok(()) } - // Services - - /// Upserts a Service into the database, updating it if it already exists. - #[cfg(test)] - pub(crate) async fn upsert_service( - &self, - opctx: &OpContext, - id: Uuid, - sled_id: Uuid, - zone_id: Option, - address: SocketAddrV6, - kind: ServiceKind, - ) -> Result<(), Error> { - info!( - self.log, - "upserting service"; - "sled_id" => sled_id.to_string(), - "service_id" => id.to_string(), - "address" => address.to_string(), - ); - let service = - db::model::Service::new(id, sled_id, zone_id, address, kind); - self.db_datastore.service_upsert(opctx, service).await?; - - if kind == ServiceKind::ExternalDns { - self.background_tasks - .activate(&self.background_tasks.task_external_dns_servers); - } else if kind == ServiceKind::InternalDns { - self.background_tasks - .activate(&self.background_tasks.task_internal_dns_servers); - } - - Ok(()) - } - /// Ensure firewall rules for internal services get reflected on all the relevant sleds. pub(crate) async fn plumb_service_firewall_rules( &self,